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 15:05:07 UTC

[01/33] incubator-ignite git commit: Added tests on EVT_CACHE_REBALANCE_PART_DATA_LOST event.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-959 769d051cf -> cb4f10d01 (forced update)


Added tests on EVT_CACHE_REBALANCE_PART_DATA_LOST event.


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

Branch: refs/heads/ignite-959
Commit: 8c2553eda7ea68498c1d0235060b20f31ef55c8a
Parents: 01eee2d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jun 17 16:12:48 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jun 17 16:12:48 2015 +0300

----------------------------------------------------------------------
 ...ridCachePartitionNotLoadedEventSelfTest.java | 82 ++++++++++++++++++++
 .../ignite/util/TestTcpCommunicationSpi.java    | 21 +++++
 2 files changed, 103 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c2553ed/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
index 6da27d5..baa6d89 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePartitionNotLoadedEventSelfTest.java
@@ -22,15 +22,19 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 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 org.apache.ignite.util.*;
 import org.eclipse.jetty.util.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  *
@@ -150,6 +154,84 @@ public class GridCachePartitionNotLoadedEventSelfTest extends GridCommonAbstract
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testStableTopology() throws Exception {
+        backupCnt = 1;
+
+        startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        startGrid(0);
+
+        PartitionNotFullyLoadedListener lsnr = new PartitionNotFullyLoadedListener();
+
+        grid(1).events().localListen(lsnr, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);
+
+        IgniteCache<Integer, Integer> cache0 = jcache(0);
+
+        int key = primaryKey(cache0);
+
+        jcache(1).put(key, key);
+
+        assert cache0.containsKey(key);
+
+        TestTcpCommunicationSpi.stop(ignite(0));
+
+        stopGrid(0, true);
+
+        awaitPartitionMapExchange();
+
+        assert jcache(1).containsKey(key);
+
+        assert lsnr.lostParts.isEmpty();
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMapPartitioned() throws Exception {
+        backupCnt = 0;
+
+        startGrid(0);
+
+        startGrid(1);
+
+        PartitionNotFullyLoadedListener lsnr = new PartitionNotFullyLoadedListener();
+
+        grid(1).events().localListen(lsnr, EventType.EVT_CACHE_REBALANCE_PART_DATA_LOST);
+
+        TestTcpCommunicationSpi.skipMsgType(ignite(0), GridDhtPartitionsFullMessage.class);
+
+        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        boolean timeout = false;
+
+        try {
+            fut.get(1, TimeUnit.SECONDS);
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            timeout = true;
+        }
+
+        assert timeout;
+
+        stopGrid(0, true);
+
+        awaitPartitionMapExchange();
+
+        assert !lsnr.lostParts.isEmpty();
+    }
+
+    /**
      *
      */
     private static class PartitionNotFullyLoadedListener implements IgnitePredicate<Event> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8c2553ed/modules/core/src/test/java/org/apache/ignite/util/TestTcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/TestTcpCommunicationSpi.java b/modules/core/src/test/java/org/apache/ignite/util/TestTcpCommunicationSpi.java
index ad2a262..6e4e50b 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/TestTcpCommunicationSpi.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/TestTcpCommunicationSpi.java
@@ -19,6 +19,7 @@ package org.apache.ignite.util;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.tcp.*;
@@ -30,11 +31,17 @@ public class TestTcpCommunicationSpi extends TcpCommunicationSpi {
     /** */
     private volatile boolean stopped;
 
+    /** */
+    private Class ignoreMsg;
+
     /** {@inheritDoc} */
     @Override public void sendMessage(final ClusterNode node, final Message msg) throws IgniteSpiException {
         if (stopped)
             return;
 
+        if (ignoreMsg != null && ((GridIoMessage)msg).message().getClass().equals(ignoreMsg))
+            return;
+
         super.sendMessage(node, msg);
     }
 
@@ -46,9 +53,23 @@ public class TestTcpCommunicationSpi extends TcpCommunicationSpi {
     }
 
     /**
+     *
+     */
+    public void stop(Class ignoreMsg) {
+        this.ignoreMsg = ignoreMsg;
+    }
+
+    /**
      * Stop SPI, messages will not send anymore.
      */
     public static void stop(Ignite ignite) {
         ((TestTcpCommunicationSpi)ignite.configuration().getCommunicationSpi()).stop();
     }
+
+    /**
+     * Skip messages will not send anymore.
+     */
+    public static void skipMsgType(Ignite ignite, Class clazz) {
+        ((TestTcpCommunicationSpi)ignite.configuration().getCommunicationSpi()).stop(clazz);
+    }
 }


[21/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-1048

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


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

Branch: refs/heads/ignite-959
Commit: a2d2c9b96e9bd86f2f711df5da835c95af57b612
Parents: be881a3 c7ba154
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 12:56:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 12:56:44 2015 +0300

----------------------------------------------------------------------
 .../replicated/GridCacheReplicatedFailoverSelfTest.java         | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------



[16/33] incubator-ignite git commit: ignite-1048 create/lock entries before topology read lock

Posted by sb...@apache.org.
ignite-1048 create/lock entries before topology read lock


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

Branch: refs/heads/ignite-959
Commit: be881a367ef02a724c2b84aba8f72c6effadc2b2
Parents: 1c66078
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 10:52:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 10:52:47 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtLocalPartition.java        | 3 ++-
 .../cache/distributed/dht/atomic/GridDhtAtomicCache.java    | 9 ++++-----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/be881a36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 3a577a7..87c7f0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -236,7 +236,8 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
     void onAdded(GridDhtCacheEntry entry) {
         GridDhtPartitionState state = state();
 
-        assert state != EVICTED : "Adding entry to invalid partition: " + this;
+        if (state == EVICTED)
+            throw new GridDhtInvalidPartitionException(id, "Adding entry to invalid partition [part=" + id + ']');
 
         map.put(entry.key(), entry);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/be881a36/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 8630421..7bec302 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1028,7 +1028,10 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         IgniteCacheExpiryPolicy expiry = null;
 
         try {
-            List<GridDhtCacheEntry> locked = null;
+            // If batch store update is enabled, we need to lock all entries.
+            // First, need to acquire locks on cache entries, then check filter.
+            List<GridDhtCacheEntry> locked = lockEntries(keys, req.topologyVersion());
+
             Collection<IgniteBiTuple<GridDhtCacheEntry, GridCacheVersion>> deleted = null;
 
             try {
@@ -1058,10 +1061,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                             return;
                         }
 
-                        // If batch store update is enabled, we need to lock all entries.
-                        // First, need to acquire locks on cache entries, then check filter.
-                        locked = lockEntries(keys, req.topologyVersion());
-
                         boolean hasNear = ctx.discovery().cacheNearNode(node, name());
 
                         GridCacheVersion ver = req.updateVersion();


[08/33] incubator-ignite git commit: # ignite-695: Fix (the rest of todos)

Posted by sb...@apache.org.
# ignite-695: Fix (the rest of todos)


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

Branch: refs/heads/ignite-959
Commit: 27fbf022fce03b1577f1ea7afe2bd3ea8fa11c0f
Parents: edc9a1c
Author: ashutak <as...@gridgain.com>
Authored: Fri Jun 19 12:54:09 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Jun 19 12:54:09 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/IgniteCacheTxMessageRecoveryTest.java     | 5 +++++
 .../near/GridCacheNearOnlyMultiNodeFullApiSelfTest.java         | 2 +-
 .../java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java | 4 ++--
 .../IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java     | 5 +++++
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java | 2 +-
 .../internal/websession/IgniteWebSessionSelfTestSuite.java      | 2 +-
 6 files changed, 15 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java
index f26948a..fab1bf3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java
@@ -26,6 +26,11 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  */
 public class IgniteCacheTxMessageRecoveryTest extends IgniteCacheMessageRecoveryAbstractTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-795");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return TRANSACTIONAL;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/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 e78b782..8343c64 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
@@ -194,7 +194,7 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
 
     /** {@inheritDoc} */
     @Override public void testNearDhtKeySize() throws Exception {
-        // TODO fix this test for client mode.
+        fail("https://issues.apache.org/jira/browse/IGNITE-1029");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 6245308..0a31f49 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -113,7 +113,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         GridTestUtils.addTestIfNeeded(suite, GridCacheMvccSelfTest.class, ignoredTests);
         suite.addTestSuite(GridCacheMvccPartitionedSelfTest.class);
         suite.addTestSuite(GridCacheMvccManagerSelfTest.class);
-//        suite.addTestSuite(GridCacheP2PUndeploySelfTest.class); TODO uncomment in DR branch.
+        suite.addTestSuite(GridCacheP2PUndeploySelfTest.class);
         suite.addTestSuite(GridCacheConfigurationValidationSelfTest.class);
         suite.addTestSuite(GridCacheConfigurationConsistencySelfTest.class);
         suite.addTestSuite(GridCacheJdbcBlobStoreSelfTest.class);
@@ -151,7 +151,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class);
         suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class);
         suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class);
-        // suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class); TODO IGNITE-795
+        suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionAtomicSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredAtomicSelfTest.class, ignoredTests);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
index dc25af5..e131470 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.java
@@ -22,6 +22,11 @@ package org.apache.ignite.internal.processors.cache;
  */
 public class IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest extends IgniteCacheQueryOffheapMultiThreadedSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-971");
+    }
+
+    /** {@inheritDoc} */
     @Override protected boolean evictsEnabled() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/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 dee3078..ae4acc2 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
@@ -62,7 +62,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27fbf022/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java
index f7e5a2b..e53b784 100644
--- a/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java
+++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/IgniteWebSessionSelfTestSuite.java
@@ -55,7 +55,7 @@ public class IgniteWebSessionSelfTestSuite extends TestSuite {
 
         /** {@inheritDoc} */
         @Override public void testRestarts() throws Exception {
-            // TODO IGNITE-810, enable when fixed.
+            fail("https://issues.apache.org/jira/browse/IGNITE-810");
         }
     }
 


[30/33] 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-959
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(-)
----------------------------------------------------------------------



[31/33] incubator-ignite git commit: #ignite-1043: Add tests for read through flag.

Posted by sb...@apache.org.
#ignite-1043: Add tests for read through flag.


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

Branch: refs/heads/ignite-959
Commit: bf1ea21b2f2ff6964b8e13e821b5194243855ff4
Parents: b3e2bd4
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jun 24 11:30:15 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jun 24 11:30:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../CacheReadThroughAtomicRestartSelfTest.java  |  32 +++++
 ...heReadThroughLocalAtomicRestartSelfTest.java |  32 +++++
 .../CacheReadThroughLocalRestartSelfTest.java   |  32 +++++
 ...dThroughReplicatedAtomicRestartSelfTest.java |  32 +++++
 ...cheReadThroughReplicatedRestartSelfTest.java |  32 +++++
 .../cache/CacheReadThroughRestartSelfTest.java  | 133 +++++++++++++++++++
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../IgniteCacheTxStoreSessionTest.java          |   4 +
 .../testsuites/IgniteCacheTestSuite4.java       |   7 +
 11 files changed, 307 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 8a4e3b9..9b7801c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1377,7 +1377,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
-     * @return {@code True} if store read-through mode is enabled.
+     * @return {@code True} if {@link CacheConfiguration#isLoadPreviousValue()} flag is set.
      */
     public boolean loadPreviousValue() {
         return cacheCfg.isLoadPreviousValue();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index bdaa552..e160529 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -958,7 +958,7 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
      *
      */
     private void loadMissingFromStore() {
-        if (!skipStore && cctx.loadPreviousValue() && cctx.readThrough() && (needReturnVal || read)) {
+        if (!skipStore && (read || cctx.loadPreviousValue()) && cctx.readThrough() && (needReturnVal || read)) {
             final Map<KeyCacheObject, GridDhtCacheEntry> loadMap = new LinkedHashMap<>();
 
             final GridCacheVersion ver = version();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughAtomicRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughAtomicRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughAtomicRestartSelfTest.java
new file mode 100644
index 0000000..3ac7c3c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughAtomicRestartSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughAtomicRestartSelfTest extends CacheReadThroughRestartSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalAtomicRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalAtomicRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalAtomicRestartSelfTest.java
new file mode 100644
index 0000000..f0892ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalAtomicRestartSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughLocalAtomicRestartSelfTest extends CacheReadThroughLocalRestartSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalRestartSelfTest.java
new file mode 100644
index 0000000..7561edf
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughLocalRestartSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughLocalRestartSelfTest extends CacheReadThroughRestartSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return LOCAL;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedAtomicRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedAtomicRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedAtomicRestartSelfTest.java
new file mode 100644
index 0000000..0e179ff
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedAtomicRestartSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughReplicatedAtomicRestartSelfTest extends CacheReadThroughReplicatedRestartSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedRestartSelfTest.java
new file mode 100644
index 0000000..5f1419c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughReplicatedRestartSelfTest.java
@@ -0,0 +1,32 @@
+/*
+ * 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.cache.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughReplicatedRestartSelfTest extends CacheReadThroughRestartSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return REPLICATED;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
new file mode 100644
index 0000000..7d0cd4c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadThroughRestartSelfTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.configuration.*;
+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.transactions.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for read through store.
+ */
+public class CacheReadThroughRestartSelfTest extends GridCacheAbstractSelfTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TransactionConfiguration txCfg = new TransactionConfiguration();
+
+        txCfg.setTxSerializableEnabled(true);
+
+        cfg.setTransactionConfiguration(txCfg);
+
+        CacheConfiguration cc = cacheConfiguration(gridName);
+
+        cc.setLoadPreviousValue(false);
+
+        cfg.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadThroughInTx() throws Exception {
+        IgniteCache<String, Integer> cache = grid(1).cache(null);
+
+        for (int k = 0; k < 1000; k++)
+            cache.put("key" + k, k);
+
+        stopAllGrids();
+
+        startGrids(2);
+
+        Ignite ignite = grid(1);
+
+        cache = ignite.cache(null);
+
+        for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) {
+            for (TransactionIsolation txIsolation : TransactionIsolation.values()) {
+                try (Transaction tx = ignite.transactions().txStart(txConcurrency, txIsolation, 100000, 1000)) {
+                    for (int k = 0; k < 1000; k++) {
+                        String key = "key" + k;
+
+                        assertNotNull("Null value for key: " + key, cache.get(key));
+                    }
+
+                    tx.commit();
+                }
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadThrough() throws Exception {
+        IgniteCache<String, Integer> cache = grid(1).cache(null);
+
+        for (int k = 0; k < 1000; k++)
+            cache.put("key" + k, k);
+
+        stopAllGrids();
+
+        startGrids(2);
+
+        Ignite ignite = grid(1);
+
+        cache = ignite.cache(null);
+
+        for (int k = 0; k < 1000; k++) {
+            String key = "key" + k;
+
+            assertNotNull("Null value for key: " + key, cache.get(key));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index 3f14f49..468aec1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -531,7 +531,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     /**
      * Serializable factory.
      */
-    private static class TestStoreFactory implements Factory<CacheStore> {
+    protected static class TestStoreFactory implements Factory<CacheStore> {
         @Override public CacheStore create() {
             return cacheStore();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionTest.java
index 269b9ae..c97fe29 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/integration/IgniteCacheTxStoreSessionTest.java
@@ -135,9 +135,13 @@ public class IgniteCacheTxStoreSessionTest extends IgniteCacheStoreSessionAbstra
 
         try (Transaction tx = startTx(concurrency, isolation)) {
             log.info("Do tx get.");
+            expData.add(new ExpectedData(false, "load", new HashMap(), cache.getName()));
+            expData.add(new ExpectedData(true, "sessionEnd", F.<Object, Object>asMap(0, "load"), cache.getName()));
 
             cache.get(key1);
 
+            expData.clear();
+
             log.info("Do tx put.");
 
             cache.put(key1, key1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf1ea21b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 6295a4d..682965a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -145,6 +145,13 @@ public class IgniteCacheTestSuite4 extends TestSuite {
 
         suite.addTestSuite(IgniteCacheManyClientsTest.class);
 
+        suite.addTestSuite(CacheReadThroughRestartSelfTest.class);
+        suite.addTestSuite(CacheReadThroughReplicatedRestartSelfTest.class);
+        suite.addTestSuite(CacheReadThroughReplicatedAtomicRestartSelfTest.class);
+        suite.addTestSuite(CacheReadThroughLocalRestartSelfTest.class);
+        suite.addTestSuite(CacheReadThroughLocalAtomicRestartSelfTest.class);
+        suite.addTestSuite(CacheReadThroughAtomicRestartSelfTest.class);
+
         return suite;
     }
 }


[13/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' into ignite-sprint-7

Conflicts:
	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java


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

Branch: refs/heads/ignite-959
Commit: cd06e1d317e3f828cb0f64ef2e2903be8d1fea8b
Parents: ec9cfca 1605996
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 16:02:08 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 16:02:08 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../ignite/internal/util/nio/GridNioServer.java |  64 +++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  92 ++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |  19 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  80 +++++---
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 ++++++++++++++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 ++-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 32 files changed, 916 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cd06e1d3/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index ae4acc2,b9205a9..14093ba
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -62,7 -62,8 +62,8 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
 -        // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
++        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
          suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);


[07/33] incubator-ignite git commit: # ignite-sprint-7

Posted by sb...@apache.org.
# 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/6974a8e3
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/6974a8e3
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/6974a8e3

Branch: refs/heads/ignite-959
Commit: 6974a8e3d356a1e1b777064ae79cb0f67a50a077
Parents: 9458f12
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 12:16:49 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 12:16:49 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/task/GridTaskProcessor.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6974a8e3/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index bb9ff50..d59a51d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -1175,7 +1175,8 @@ public class GridTaskProcessor extends GridProcessorAdapter {
 
             ctx.closure().runLocalSafe(new Runnable() {
                 @Override public void run() {
-                    lock.readLock();
+                    if (!lock.tryReadLock())
+                        return;
 
                     try {
                         for (GridTaskWorker<?, ?> task : tasks.values())


[12/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' into ignite-sprint-7

Conflicts:
	examples/pom.xml
	modules/aop/pom.xml
	modules/aws/pom.xml
	modules/clients/pom.xml
	modules/cloud/pom.xml
	modules/codegen/pom.xml
	modules/core/pom.xml
	modules/core/src/main/resources/ignite.properties
	modules/extdata/p2p/pom.xml
	modules/extdata/uri/pom.xml
	modules/gce/pom.xml
	modules/geospatial/pom.xml
	modules/hadoop/pom.xml
	modules/hibernate/pom.xml
	modules/indexing/pom.xml
	modules/jcl/pom.xml
	modules/jta/pom.xml
	modules/log4j/pom.xml
	modules/mesos/pom.xml
	modules/rest-http/pom.xml
	modules/scalar-2.10/pom.xml
	modules/scalar/pom.xml
	modules/schedule/pom.xml
	modules/schema-import/pom.xml
	modules/slf4j/pom.xml
	modules/spark-2.10/pom.xml
	modules/spark/pom.xml
	modules/spring/pom.xml
	modules/ssh/pom.xml
	modules/tools/pom.xml
	modules/urideploy/pom.xml
	modules/visor-console-2.10/pom.xml
	modules/visor-console/pom.xml
	modules/visor-plugins/pom.xml
	modules/web/pom.xml
	modules/yardstick/pom.xml
	pom.xml


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

Branch: refs/heads/ignite-959
Commit: ec9cfcaf8b245be6fe4723b1e2d08526d3a6d4cb
Parents: 8fa9d3d d699faa
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 19 15:32:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 19 15:32:21 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 ++++++++++++++++++--
 1 file changed, 41 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[29/33] 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-959
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)


[33/33] incubator-ignite git commit: ignite-959 Avoid sending key-value class definitions to servers when starting a cache

Posted by sb...@apache.org.
ignite-959 Avoid sending key-value class definitions to servers when starting a 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/cb4f10d0
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/cb4f10d0
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/cb4f10d0

Branch: refs/heads/ignite-959
Commit: cb4f10d01b611c2109e4a8a211e355a9fb10c096
Parents: c270bd8
Author: agura <ag...@gridgain.com>
Authored: Wed Jun 24 16:04:36 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Wed Jun 24 16:04:36 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheTypeMetadata.java  |  30 ++
 .../configuration/CacheConfiguration.java       |   2 +-
 .../processors/cache/GridCacheProcessor.java    |  18 +-
 .../processors/query/GridQueryProcessor.java    | 375 ++++++++++++++++---
 .../CacheMetricsForClusterGroupSelfTest.java    |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 6 files changed, 368 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
index 20129b7..d6b0c79 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheTypeMetadata.java
@@ -71,6 +71,10 @@ public class CacheTypeMetadata implements Serializable {
     @GridToStringInclude
     private Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> grps;
 
+    /** Aliases for fields. */
+    @GridToStringInclude
+    private Map<String, String> aliases;
+
     /**
      * Default constructor.
      */
@@ -88,6 +92,8 @@ public class CacheTypeMetadata implements Serializable {
         txtFlds = new LinkedHashSet<>();
 
         grps = new LinkedHashMap<>();
+
+        aliases = new HashMap<>();
     }
 
     /**
@@ -115,6 +121,8 @@ public class CacheTypeMetadata implements Serializable {
         txtFlds = new LinkedHashSet<>(src.getTextFields());
 
         grps = new LinkedHashMap<>(src.getGroups());
+
+        aliases = new HashMap<>(src.aliases);
     }
 
     /**
@@ -333,6 +341,28 @@ public class CacheTypeMetadata implements Serializable {
         this.grps = grps;
     }
 
+    /**
+     * Returns fields aliases.
+     *
+     * @return fields aliases.
+     */
+    public Map<String, String> getAliases() {
+        return aliases;
+    }
+
+    /**
+     * Adds field alias.
+     *
+     * @param alias Alias.
+     * @param fieldName Field name.
+     */
+    public void addAlias(String alias, String fieldName) {
+        A.notNull(alias, "alias");
+        A.notNull(fieldName, "fieldName");
+
+        aliases.put(alias, fieldName);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(CacheTypeMetadata.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 7af4974..4ec3e4c 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -298,7 +298,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     private boolean sqlEscapeAll;
 
     /** */
-    private Class<?>[] indexedTypes;
+    private transient Class<?>[] indexedTypes;
 
     /** */
     private int sqlOnheapRowCacheSize = DFLT_SQL_ONHEAP_ROW_CACHE_SIZE;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/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..428063b 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
@@ -583,6 +583,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             // Initialize defaults.
             initialize(cfg, cacheObjCtx);
 
+            ctx.query().generateTypeMetadata(cfg);
+
             cfgs[i] = cfg; // Replace original configuration value.
 
             String masked = maskNull(cfg.getName());
@@ -594,8 +596,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     throw new IgniteCheckedException("Duplicate cache name found (check configuration and " +
                         "assign unique name to each cache): " + U.maskName(cacheName));
                 else
-                    throw new IgniteCheckedException("Default cache has already been configured (check configuration and " +
-                        "assign unique name to each cache).");
+                    throw new IgniteCheckedException("Default cache has already been configured " +
+                        "(check configuration and assign unique name to each cache).");
             }
 
             CacheType cacheType;
@@ -691,7 +693,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         if (rmtCfg != null) {
                             CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n, desc);
+                            checkCache(locCfg, rmtCfg, n);
 
                             // Check plugin cache configurations.
                             CachePluginManager pluginMgr = desc.pluginManager();
@@ -1625,7 +1627,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         for (DynamicCacheDescriptor desc : registeredCaches.values()) {
             if (!desc.cancelled()) {
-                DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
+                DynamicCacheChangeRequest req =
+                    new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
                 req.startCacheConfiguration(desc.cacheConfiguration());
 
@@ -1930,6 +1933,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 try {
                     CacheConfiguration cfg = new CacheConfiguration(ccfg);
 
+                    ctx.query().generateTypeMetadata(cfg);
+
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
 
                     initialize(cfg, cacheObjCtx);
@@ -2260,11 +2265,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param locCfg Local configuration.
      * @param rmtCfg Remote configuration.
      * @param rmtNode Remote node.
-     * @param desc Cache descriptor.
      * @throws IgniteCheckedException If check failed.
      */
-    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode,
-        DynamicCacheDescriptor desc) throws IgniteCheckedException {
+    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode)
+        throws IgniteCheckedException {
         ClusterNode locNode = ctx.discovery().localNode();
 
         UUID rmt = rmtNode.id();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e080c6d..a1bb019 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -117,67 +117,304 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param ccfg Cache configuration.
      * @throws IgniteCheckedException If failed.
      */
+    /**
+     * @param ccfg Ccfg.
+     */
     public void initializeCache(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
         idx.registerCache(ccfg);
 
         try {
-            if (!F.isEmpty(ccfg.getTypeMetadata())) {
-                for (CacheTypeMetadata meta : ccfg.getTypeMetadata()) {
-                    if (F.isEmpty(meta.getValueType()))
-                        throw new IgniteCheckedException("Value type is not set: " + meta);
+            //generateTypeMetadata(ccfg);
+
+            if (!F.isEmpty(ccfg.getTypeMetadata()))
+                processTypeMetadata(ccfg);
+        }
+        catch (IgniteCheckedException | RuntimeException e) {
+            idx.unregisterCache(ccfg);
+
+            throw e;
+        }
+    }
+
+    /**
+     * Generates cache type metadata if indexed types are defined.
+     *
+     * @param ccfg Cache configuration.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public void generateTypeMetadata(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
+        if (ccfg.getTypeMetadata() != null)
+            throw new IgniteCheckedException("Cache type metadata already exists for cache " + ccfg.getName());
+
+        Class<?>[] clss = ccfg.getIndexedTypes();
+
+        if (F.isEmpty(clss))
+            return;
+
+        List<CacheTypeMetadata> metadata = new ArrayList<>(clss.length / 2);
+
+        for (int i = 0; i < clss.length; i += 2) {
+            Class<?> keyCls = clss[i];
+            Class<?> valCls = clss[i + 1];
+
+            CacheTypeMetadata meta = new CacheTypeMetadata();
+
+            meta.setKeyType(keyCls);
+            meta.setValueType(valCls);
+
+            Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups = new HashMap<>();
+
+            processClassAnnotations(keyCls, meta, null, orderedGroups);
+            processClassAnnotations(valCls, meta, null, orderedGroups);
+
+            fillOrderedGroups(meta, orderedGroups);
+
+            metadata.add(meta);
+        }
+
+        ccfg.setTypeMetadata(metadata);
+    }
+
+    /**
+     * @param meta Cache type metadata.
+     * @param orderedGroups Ordered groups.
+     * @throws IgniteCheckedException In case or error.
+     */
+    private static void fillOrderedGroups(CacheTypeMetadata meta,
+        Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups)
+        throws IgniteCheckedException
+    {
+        for (Map.Entry<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> ordGrp : orderedGroups.entrySet()) {
+            String grpName = ordGrp.getKey();
+
+            Map<String, LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>>> groups = meta.getGroups();
+
+            LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields = groups.get(grpName);
+
+            if (fields == null)
+                groups.put(grpName, fields = new LinkedHashMap<>());
+
+            for (Map.Entry<Integer, T3<String, Class<?>, Boolean>> grpFields : ordGrp.getValue().entrySet()) {
+                String name = grpFields.getValue().get1();
+
+                Class<?> cls = grpFields.getValue().get2();
 
-                    TypeDescriptor desc = new TypeDescriptor();
+                Boolean desc = grpFields.getValue().get3();
 
-                    Class<?> valCls = U.classForName(meta.getValueType(), null);
+                if (fields.put(name, new T2<Class<?>, Boolean>(cls, desc)) != null)
+                    throw new IgniteCheckedException("Field " + name + " already exists in group " + grpName);
+            }
+        }
+    }
+
+    /**
+     * @param cls Class.
+     * @param meta Type metadata.
+     * @param parentField Parent field name.
+     */
+    private void processClassAnnotations(Class<?> cls, CacheTypeMetadata meta,
+        String parentField, Map<String, TreeMap<Integer, T3<String, Class<?>, Boolean>>> orderedGroups)
+        throws IgniteCheckedException
+    {
+        if (U.isJdk(cls) || idx.isGeometryClass(cls))
+            return;
+
+        for (Class<?> c = cls; c != null && !c.equals(Object.class); c = c.getSuperclass()) {
+            for (Field field : c.getDeclaredFields()) {
+                QueryTextField txtAnn = field.getAnnotation(QueryTextField.class);
+
+                if (txtAnn != null) {
+                    String fieldName = parentField == null ? field.getName() : parentField + '.' + field.getName();
+
+                    meta.getTextFields().add(fieldName);
+
+                    meta.getQueryFields().put(fieldName, field.getType());
+                }
+
+                QuerySqlField sqlAnn = field.getAnnotation(QuerySqlField.class);
 
-                    desc.name(valCls != null ? typeName(valCls) : meta.getValueType());
+                if (sqlAnn != null) {
+                    String name = sqlAnn.name().isEmpty() ? field.getName() : sqlAnn.name();
 
-                    desc.valueClass(valCls != null ? valCls : Object.class);
-                    desc.keyClass(
-                        meta.getKeyType() == null ?
-                            Object.class :
-                            U.classForName(meta.getKeyType(), Object.class));
+                    String pathStr = parentField == null ? name : parentField + '.' + name;
 
-                    TypeId typeId;
+                    processClassAnnotations(field.getType(), meta, pathStr, orderedGroups);
 
-                    if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
-                        processPortableMeta(meta, desc);
+                    if (sqlAnn.index()) {
+                        Map<String, Class<?>> fields =
+                            sqlAnn.descending() ? meta.getDescendingFields() : meta.getAscendingFields();
 
-                        typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
+                        fields.put(pathStr, field.getType());
                     }
-                    else {
-                        processClassMeta(meta, desc);
 
-                        typeId = new TypeId(ccfg.getName(), valCls);
+                    meta.getQueryFields().put(pathStr, field.getType());
+
+                    if (!sqlAnn.name().isEmpty())
+                        meta.addAlias(sqlAnn.name(), field.getName());
+
+                    if (!F.isEmpty(sqlAnn.groups())) {
+                        for (String grp : sqlAnn.groups()) {
+                            LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields = meta.getGroups().get(grp);
+
+                            if (fields == null)
+                                meta.getGroups().put(grp, fields = new LinkedHashMap<>());
+
+                            if (fields.put(pathStr, new IgniteBiTuple<Class<?>, Boolean>(field.getType(), false)) != null)
+                                throw new IgniteCheckedException("Field " + pathStr + " already exists in group " + grp);
+                        }
                     }
 
-                    addTypeByName(ccfg, desc);
-                    types.put(typeId, desc);
+                    if (!F.isEmpty(sqlAnn.orderedGroups())) {
+                        for (QuerySqlField.Group idx : sqlAnn.orderedGroups()) {
+                            TreeMap<Integer, T3<String, Class<?>, Boolean>> orderedFields = orderedGroups.get(idx.name());
+
+                            if (orderedFields == null)
+                                orderedGroups.put(idx.name(), orderedFields = new TreeMap<>());
+
+                            T3<String, Class<?>, Boolean> grp =
+                                new T3<String, Class<?>, Boolean>(pathStr, field.getType(), idx.descending());
 
-                    desc.registered(idx.registerType(ccfg.getName(), desc));
+                            if (orderedFields.put(idx.order(), grp) != null)
+                                throw new IgniteCheckedException("Field " + pathStr + " has duplicated order " +
+                                    idx.order() + " in group " + idx.name());
+                        }
+                    }
                 }
             }
 
-            Class<?>[] clss = ccfg.getIndexedTypes();
+            for (Method mtd : c.getDeclaredMethods()) {
+                String mtdName = mtd.getName().startsWith("get") && mtd.getName().length() > 3 ?
+                    mtd.getName().substring(3) : mtd.getName();
+
+                QuerySqlField sqlAnn = mtd.getAnnotation(QuerySqlField.class);
+                QueryTextField txtAnn = mtd.getAnnotation(QueryTextField.class);
+
+                if (sqlAnn != null || txtAnn != null) {
+                    if (mtd.getParameterTypes().length != 0)
+                        throw new IgniteCheckedException("Getter with QuerySqlField " +
+                            "annotation cannot have parameters: " + mtd);
 
-            if (!F.isEmpty(clss)) {
-                for (int i = 0; i < clss.length; i += 2) {
-                    Class<?> keyCls = clss[i];
-                    Class<?> valCls = clss[i + 1];
+                    Class<?> type = mtd.getReturnType();
 
-                    TypeDescriptor desc = processKeyAndValueClasses(keyCls, valCls);
+                    if (txtAnn != null) {
+                        String pathStr = parentField == null ? mtdName : parentField + '.' + mtdName;
 
-                    addTypeByName(ccfg, desc);
-                    types.put(new TypeId(ccfg.getName(), valCls), desc);
+                        meta.getTextFields().add(pathStr);
 
-                    desc.registered(idx.registerType(ccfg.getName(), desc));
+                        meta.getQueryFields().put(pathStr, type);
+                    }
+
+                    if (sqlAnn != null) {
+                        String name = sqlAnn.name().isEmpty() ? mtdName : sqlAnn.name();
+
+                        name = parentField == null ? name : parentField + '.' + name;
+
+                        processClassAnnotations(mtd.getReturnType(), meta, name, orderedGroups);
+
+                        if (sqlAnn.index()) {
+                            Map<String, Class<?>> fields =
+                                sqlAnn.descending() ? meta.getDescendingFields() : meta.getAscendingFields();
+
+                            fields.put(name, type);
+                        }
+
+                        meta.getQueryFields().put(name, type);
+
+                        if (!sqlAnn.name().isEmpty())
+                            meta.addAlias(sqlAnn.name(), mtdName);
+
+                        if (!F.isEmpty(sqlAnn.groups())) {
+                            for (String grp : sqlAnn.groups()) {
+                                LinkedHashMap<String, IgniteBiTuple<Class<?>, Boolean>> fields = meta.getGroups().get(grp);
+
+                                if (fields == null) {
+                                    fields = new LinkedHashMap<>();
+
+                                    meta.getGroups().put(grp, fields);
+                                }
+
+                                if (fields.put(name, new IgniteBiTuple<Class<?>, Boolean>(mtd.getReturnType(), false)) != null)
+                                    throw new IgniteCheckedException("Field " + name + " already exists in group " + grp);
+                            }
+                        }
+
+                        if (!F.isEmpty(sqlAnn.orderedGroups())) {
+                            for (QuerySqlField.Group idx : sqlAnn.orderedGroups()) {
+                                TreeMap<Integer, T3<String, Class<?>, Boolean>> orderedFields = orderedGroups.get(idx.name());
+
+                                if (orderedFields == null)
+                                    orderedGroups.put(idx.name(), orderedFields = new TreeMap<>());
+
+                                T3<String, Class<?>, Boolean> grp =
+                                    new T3<String, Class<?>, Boolean>(name, mtd.getReturnType(), idx.descending());
+
+                                if (orderedFields.put(idx.order(), grp) != null)
+                                    throw new IgniteCheckedException("Field " + name + " has duplicated order " +
+                                        idx.order() + " in group " + idx.name());
+                            }
+                        }
+                    }
                 }
             }
         }
-        catch (IgniteCheckedException | RuntimeException e) {
-            idx.unregisterCache(ccfg);
+    }
 
-            throw e;
+    private void processIndexedTypes(CacheConfiguration<?, ?> ccfg, Class<?>[] clss) throws IgniteCheckedException {
+        for (int i = 0; i < clss.length; i += 2) {
+            Class<?> keyCls = clss[i];
+            Class<?> valCls = clss[i + 1];
+
+            TypeDescriptor desc = processKeyAndValueClasses(keyCls, valCls);
+
+            addTypeByName(ccfg, desc);
+            types.put(new TypeId(ccfg.getName(), valCls), desc);
+
+            desc.registered(idx.registerType(ccfg.getName(), desc));
+        }
+    }
+
+    private void processTypeMetadata(CacheConfiguration<?, ?> ccfg) throws IgniteCheckedException {
+        for (CacheTypeMetadata meta : ccfg.getTypeMetadata()) {
+            if (F.isEmpty(meta.getValueType()))
+                throw new IgniteCheckedException("Value type is not set: " + meta);
+
+            TypeDescriptor desc = new TypeDescriptor();
+
+            Class<?> valCls = U.classForName(meta.getValueType(), null);
+
+            desc.name(valCls != null ? typeName(valCls) : meta.getValueType());
+
+            desc.valueClass(valCls != null ? valCls : Object.class);
+            desc.keyClass(
+                meta.getKeyType() == null ?
+                    Object.class :
+                    U.classForName(meta.getKeyType(), Object.class));
+
+            TypeId typeId;
+
+            if (valCls == null || ctx.cacheObjects().isPortableEnabled()) {
+                processPortableMeta(meta, desc);
+
+                typeId = new TypeId(ccfg.getName(), ctx.cacheObjects().typeId(meta.getValueType()));
+            }
+            else {
+                processClassMeta(meta, desc);
+
+                typeId = new TypeId(ccfg.getName(), valCls);
+
+                if ((U.isJdk(valCls) || idx.isGeometryClass(valCls)) && idx.isSqlType(valCls)) { // We have to index primitive _val.
+                    String idxName = "_val_idx";
+
+                    desc.addIndex(idxName, idx.isGeometryClass(valCls) ? GEO_SPATIAL : SORTED);
+
+                    desc.addFieldToIndex(idxName, "_VAL", 0, false);
+                }
+            }
+
+            addTypeByName(ccfg, desc);
+            types.put(typeId, desc);
+
+            desc.registered(idx.registerType(ccfg.getName(), desc));
         }
     }
 
@@ -1144,12 +1381,26 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert keyCls != null;
         assert valCls != null;
 
-        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
             ClassProperty prop = buildClassProperty(
                 keyCls,
                 valCls,
                 entry.getKey(),
-                entry.getValue());
+                entry.getValue(),
+                meta.getAliases());
+
+            d.addProperty(prop, false);
+        }
+
+        for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
+            String name = entry.getKey();
+
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                name,
+                entry.getValue(),
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1165,7 +1416,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 keyCls,
                 valCls,
                 entry.getKey(),
-                entry.getValue());
+                entry.getValue(),
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1181,7 +1433,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 keyCls,
                 valCls,
                 txtIdx,
-                String.class);
+                String.class,
+                meta.getAliases());
 
             d.addProperty(prop, false);
 
@@ -1203,7 +1456,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                         keyCls,
                         valCls,
                         idxField.getKey(),
-                        idxField.getValue().get1());
+                        idxField.getValue().get1(),
+                        meta.getAliases());
 
                     d.addProperty(prop, false);
 
@@ -1215,16 +1469,25 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
+    }
 
-        for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(
-                keyCls,
-                valCls,
-                entry.getKey(),
-                entry.getValue());
+    private String resolveAliases(String key, CacheTypeMetadata meta) {
+        Map<String, String> aliases = meta.getAliases();
 
-            d.addProperty(prop, false);
+        String[] tokens = key.split("\\.");
+
+        StringBuilder sb = new StringBuilder();
+
+        for (String token : tokens) {
+            String name = aliases.get(token);
+
+            if (sb.length() > 0)
+                sb.append('.');
+
+            sb.append(name == null ? token : name);
         }
+
+        return sb.toString();
     }
 
     /**
@@ -1327,16 +1590,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Class property.
      * @throws IgniteCheckedException If failed.
      */
-    private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType)
+    private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType, Map<String, String> aliases)
         throws IgniteCheckedException {
         ClassProperty res = buildClassProperty(
             true,
             keyCls,
             pathStr,
-            resType);
+            resType,
+            aliases);
 
         if (res == null) // We check key before value consistently with PortableProperty.
-            res = buildClassProperty(false, valCls, pathStr, resType);
+            res = buildClassProperty(false, valCls, pathStr, resType, aliases);
 
         if (res == null)
             throw new IgniteCheckedException("Failed to initialize property '" + pathStr + "' for " +
@@ -1354,7 +1618,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Property instance corresponding to the given path.
      * @throws IgniteCheckedException If property cannot be created.
      */
-    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType)
+    static ClassProperty buildClassProperty(boolean key, Class<?> cls, String pathStr, Class<?> resType, Map<String, String> aliases)
         throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
@@ -1363,10 +1627,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         for (String prop : path) {
             ClassProperty tmp;
 
+            String fieldName = aliases.get(prop);
+
+            if (fieldName == null)
+                fieldName = prop;
+
             try {
                 StringBuilder bld = new StringBuilder("get");
 
-                bld.append(prop);
+                bld.append(fieldName);
 
                 bld.setCharAt(3, Character.toUpperCase(bld.charAt(3)));
 
@@ -1374,13 +1643,15 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             }
             catch (NoSuchMethodException ignore) {
                 try {
-                    tmp = new ClassProperty(cls.getDeclaredField(prop), key);
+                    tmp = new ClassProperty(cls.getDeclaredField(fieldName), key);
                 }
                 catch (NoSuchFieldException ignored) {
                     return null;
                 }
             }
 
+            tmp.name(prop);
+
             tmp.parent(res);
 
             cls = tmp.type();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
index 1ba24e3..455e306 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheMetricsForClusterGroupSelfTest.java
@@ -99,7 +99,7 @@ public class CacheMetricsForClusterGroupSelfTest extends GridCommonAbstractTest
         Collection<ClusterNode> nodes = grid(0).cluster().forRemotes().nodes();
 
         for (ClusterNode node : nodes) {
-            Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode) node).cacheMetrics();
+            Map<Integer, CacheMetrics> metrics = ((TcpDiscoveryNode)node).cacheMetrics();
             assertNotNull(metrics);
             assertFalse(metrics.isEmpty());
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cb4f10d0/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 06c0961..138a15a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -632,6 +632,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         PreparedStatement stmt;
 
         try {
+//            System.out.println("!!! SQL: " + sql);
+
             stmt = conn.prepareStatement(sql);
         }
         catch (SQLException e) {


[22/33] incubator-ignite git commit: # ignite-sprint-7 minor

Posted by sb...@apache.org.
# ignite-sprint-7 minor


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

Branch: refs/heads/ignite-959
Commit: 36433bc3d96a67975ce99a0774e608ccf50c2a21
Parents: c7ba154
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 13:31:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 13:31:34 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/testframework/junits/GridAbstractTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/36433bc3/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 9c42920..9a63b3a 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -606,7 +606,7 @@ public abstract class GridAbstractTest extends TestCase {
                 Thread.sleep(1000);
         }
 
-        throw new Exception("Failed to wait for proper topology.");
+        throw new Exception("Failed to wait for proper topology: " + cnt);
     }
 
     /** */


[11/33] incubator-ignite git commit: # ignite-sprint-7 do not call ping from exchange worker

Posted by sb...@apache.org.
# ignite-sprint-7 do not call ping from exchange worker


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

Branch: refs/heads/ignite-959
Commit: 8fa9d3dd9214e85748a6b7b900fa2433079ce246
Parents: d006389
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 14:37:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 14:37:37 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 64 ++++++++++++------
 .../GridCachePartitionExchangeManager.java      | 70 +++++++++-----------
 .../GridCacheAbstractFailoverSelfTest.java      |  6 +-
 3 files changed, 79 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index eef9fde..74a4512 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -323,7 +323,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * Processes failed messages.
      * @param nodeId niode id.
      * @param msg message.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
@@ -511,6 +511,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
      */
+    @SuppressWarnings("unchecked")
     private IgniteInternalFuture<Object> startFuture(GridCacheMessage cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
@@ -574,6 +575,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      * @throws ClusterTopologyCheckedException If receiver left.
      */
@@ -734,6 +736,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param nodeId ID of node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      */
     public void send(UUID nodeId, GridCacheMessage msg, GridIoPolicy plc) throws IgniteCheckedException {
@@ -795,8 +798,41 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Sends message without retries and node ping in case of error.
+     *
+     * @param node Node to send message to.
+     * @param msg Message.
+     * @param plc IO policy.
+     * @throws IgniteCheckedException If send failed.
+     */
+    public void sendNoRetry(ClusterNode node,
+        GridCacheMessage msg,
+        GridIoPolicy plc)
+        throws IgniteCheckedException
+    {
+        assert node != null;
+        assert msg != null;
+
+        onSend(msg, null);
+
+        try {
+            cctx.gridIO().send(node, TOPIC_CACHE, msg, plc);
+
+            if (log.isDebugEnabled())
+                log.debug("Sent cache message [msg=" + msg + ", node=" + U.toShortString(node) + ']');
+        }
+        catch (IgniteCheckedException e) {
+            if (!cctx.discovery().alive(node.id()))
+                throw new ClusterTopologyCheckedException("Node left grid while sending message to: " + node.id(), e);
+            else
+                throw e;
+        }
+    }
+
+    /**
      * Adds message handler.
      *
+     * @param cacheId Cache ID.
      * @param type Type of message.
      * @param c Handler.
      */
@@ -846,29 +882,15 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         idxClsHandlers.remove(cacheId);
 
-        for (Iterator<ListenerKey> iterator = clsHandlers.keySet().iterator(); iterator.hasNext(); ) {
-            ListenerKey key = iterator.next();
+        for (Iterator<ListenerKey> iter = clsHandlers.keySet().iterator(); iter.hasNext(); ) {
+            ListenerKey key = iter.next();
 
             if (key.cacheId == cacheId)
-                iterator.remove();
+                iter.remove();
         }
     }
 
     /**
-     * @param lsnr Listener to add.
-     */
-    public void addDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().addDisconnectListener(lsnr);
-    }
-
-    /**
-     * @param lsnr Listener to remove.
-     */
-    public void removeDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().removeDisconnectListener(lsnr);
-    }
-
-    /**
      * @param msgCls Message class to check.
      * @return Message index.
      */
@@ -1029,11 +1051,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = cacheId;
+            int res = cacheId;
 
-            result = 31 * result + msgCls.hashCode();
+            res = 31 * res + msgCls.hashCode();
 
-            return result;
+            return res;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 3df45cb..81019e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -567,27 +567,21 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         Collection<ClusterNode> rmts = null;
 
-        try {
-            // If this is the oldest node.
-            if (oldest.id().equals(cctx.localNodeId())) {
-                rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
-
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
+        // If this is the oldest node.
+        if (oldest.id().equals(cctx.localNodeId())) {
+            rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
 
-                sendAllPartitions(rmts);
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing local partitions from non-oldest node: " +
-                        cctx.localNodeId());
+            if (log.isDebugEnabled())
+                log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
 
-                sendLocalPartitions(oldest, null);
-            }
+            sendAllPartitions(rmts);
         }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to refresh partition map [oldest=" + oldest.id() + ", rmts=" + U.nodeIds(rmts) +
-                ", loc=" + cctx.localNodeId() + ']', e);
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Refreshing local partitions from non-oldest node: " +
+                    cctx.localNodeId());
+
+            sendLocalPartitions(oldest, null);
         }
     }
 
@@ -616,10 +610,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param nodes Nodes.
      * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes)
-        throws IgniteCheckedException {
+    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes) {
         GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE);
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
@@ -634,7 +626,19 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
-        cctx.io().safeSend(nodes, m, SYSTEM_POOL, null);
+        for (ClusterNode node : nodes) {
+            try {
+                cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
+            }
+            catch (ClusterTopologyCheckedException ignore) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
+                        node.id() + ", msg=" + m + ']');
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send partitions full message [node=" + node + ']', e);
+            }
+        }
 
         return true;
     }
@@ -642,11 +646,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param node Node.
      * @param id ID.
-     * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
-        throws IgniteCheckedException {
+    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
             cctx.kernalContext().clientNode(),
             cctx.versions().last());
@@ -669,16 +670,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
 
         try {
-            cctx.io().send(node, m, SYSTEM_POOL);
-
-            return true;
+            cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
                     node.id() + ", msg=" + m + ']');
-
-            return false;
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send local partition map to node [node=" + node + ", exchId=" + id + ']', e);
         }
     }
 
@@ -902,13 +902,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
 
         try {
-            try {
-                sendLocalPartitions(node, msg.exchangeId());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send local partition map to node [nodeId=" + node.id() + ", exchId=" +
-                    msg.exchangeId() + ']', e);
-            }
+            sendLocalPartitions(node, msg.exchangeId());
         }
         finally {
             leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index b6cd88e..3c74674 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -74,9 +74,11 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
-        discoSpi.setSocketTimeout(10_000);
-        discoSpi.setAckTimeout(10_000);
+        discoSpi.setSocketTimeout(30_000);
+        discoSpi.setAckTimeout(30_000);
         discoSpi.setNetworkTimeout(60_000);
+        discoSpi.setHeartbeatFrequency(30_000);
+        discoSpi.setReconnectCount(2);
 
         return cfg;
     }


[28/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-7' into ignite-1045

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


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

Branch: refs/heads/ignite-959
Commit: a2141c6a7012f10e8f8003160752047046ffdf0d
Parents: a6cda33 359b431
Author: sboikov <se...@inria.fr>
Authored: Tue Jun 23 22:14:51 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Tue Jun 23 22:14:51 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteReflectionFactory.java  |  81 ++++++-
 .../distributed/dht/GridDhtLocalPartition.java  |   3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   9 +-
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 modules/hadoop/pom.xml                          |  78 ------
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +--
 .../internal/processors/hadoop/HadoopUtils.java | 237 ------------------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 +++++++++++++++++++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 ++++++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 ++-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 28 files changed, 561 insertions(+), 389 deletions(-)
----------------------------------------------------------------------



[27/33] incubator-ignite git commit: ignite-1045 properly handle flag DynamicCacheChangeRequest.clientStartOnly

Posted by sb...@apache.org.
ignite-1045 properly handle flag DynamicCacheChangeRequest.clientStartOnly


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

Branch: refs/heads/ignite-959
Commit: a6cda33a8b76925d09c21262e88467421025fa77
Parents: c7ba154
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 16:47:13 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 17:29:49 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |  46 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++++++++++
 .../distributed/IgniteCache150ClientsTest.java  | 189 ++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../testsuites/IgniteClientTestSuite.java       |  38 ++
 6 files changed, 733 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 7c780b0..38a0d55 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -153,6 +153,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** Skip preload flag. */
     private boolean skipPreload;
 
+    /** */
+    private boolean clientOnlyExchange;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -524,6 +527,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                         if (exchId.isLeft())
                             cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
 
+                        rmtIds = Collections.emptyList();
+                        rmtNodes = Collections.emptyList();
+
                         onDone(exchId.topologyVersion());
 
                         skipPreload = cctx.kernalContext().clientNode();
@@ -532,8 +538,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     }
                 }
 
-                if (cctx.kernalContext().clientNode()) {
-                    skipPreload = true;
+                clientOnlyExchange = clientNodeEvt || cctx.kernalContext().clientNode();
+
+                if (clientOnlyExchange) {
+                    skipPreload = cctx.kernalContext().clientNode();
 
                     for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                         if (cacheCtx.isLocal())
@@ -551,23 +559,45 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                         initTopology(cacheCtx);
                     }
 
-                    if (oldestNode.get() != null) {
+                    if (oldest != null) {
                         rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteServerNodesWithCaches(cctx,
                             exchId.topologyVersion()));
 
                         rmtIds = Collections.unmodifiableSet(new HashSet<>(F.nodeIds(rmtNodes)));
 
-                        ready.set(true);
-
                         initFut.onDone(true);
 
                         if (log.isDebugEnabled())
                             log.debug("Initialized future: " + this);
 
-                        sendPartitions();
+                        if (cctx.localNode().equals(oldest)) {
+                            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                                boolean updateTop = !cacheCtx.isLocal() &&
+                                    exchId.topologyVersion().equals(cacheCtx.startTopologyVersion());
+
+                                if (updateTop) {
+                                    for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
+                                        if (top.cacheId() == cacheCtx.cacheId()) {
+                                            cacheCtx.topology().update(exchId, top.partitionMap(true));
+
+                                            break;
+                                        }
+                                    }
+
+                                }
+                            }
+
+                            onDone(exchId.topologyVersion());
+                        }
+                        else
+                            sendPartitions();
                     }
-                    else
+                    else {
+                        rmtIds = Collections.emptyList();
+                        rmtNodes = Collections.emptyList();
+
                         onDone(exchId.topologyVersion());
+                    }
 
                     return;
                 }
@@ -885,7 +915,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      */
     private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) throws IgniteCheckedException {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
-            cctx.kernalContext().clientNode(),
+            clientOnlyExchange,
             cctx.versions().last());
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
new file mode 100644
index 0000000..5011e5f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
@@ -0,0 +1,466 @@
+/*
+ * 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.cache.affinity.fair.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+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 org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheStartNoExchangeTimeoutTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        if (gridName.equals(getTestGridName(NODES - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrids(NODES);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultinodeCacheStart() throws Exception {
+        for (int i = 0; i < 10; i++) {
+            log.info("Iteration: " + i);
+
+            final String name = "cache-" + i;
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Ignite ignite = ignite(idx.getAndIncrement());
+
+                    CacheConfiguration ccfg = new CacheConfiguration();
+
+                    ccfg.setName(name);
+
+                    assertNotNull(ignite.getOrCreateCache(ccfg));
+
+                    return null;
+                }
+            }, 2, "create-cache").get(15_000);
+
+            awaitPartitionMapExchange();
+
+            checkCache(name);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestNotAffinityNode1() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            oldestNotAffinityNode1(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void oldestNotAffinityNode1(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite = grid(0);
+
+        assertEquals(1L, ignite.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite.getOrCreateCache(ccfg));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestNotAffinityNode2() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            oldestNotAffinityNode2(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void oldestNotAffinityNode2(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite0 = grid(0);
+        IgniteEx ignite1 = grid(1);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite1.getOrCreateCache(ccfg));
+
+        assertNotNull(ignite0.cache(ccfg.getName()));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotAffinityNode1() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            notAffinityNode1(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void notAffinityNode1(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite = grid(1);
+
+        assertEquals(2, ignite.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeNode(2));
+
+        assertNotNull(ignite.getOrCreateCache(ccfg));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotAffinityNode2() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            notAffinityNode2(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void notAffinityNode2(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite0 = grid(0);
+        IgniteEx ignite1 = grid(1);
+
+        assertEquals(2L, ignite1.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeNode(2));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        assertNotNull(ignite1.cache(ccfg.getName()));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged1() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx client = grid(NODES - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged2() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterIncludeNode(3));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx ingite1 = grid(1);
+
+        assertNotNull(ingite1.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged3() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterIncludeNode(3));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx client = grid(NODES - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @param name Cache name.
+     */
+    private void checkCache(@Nullable String name) {
+        int key = 0;
+
+        for (Ignite ignite : G.allGrids()) {
+            IgniteCache<Object, Object> cache = ignite.cache(name);
+
+            assertNotNull(cache);
+
+            for (int i = 0; i < 100; i++) {
+                cache.put(key, key);
+
+                assertEquals(key, cache.get(key));
+
+                key++;
+            }
+        }
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    private List<CacheConfiguration> cacheConfigurations() {
+        List<CacheConfiguration> res = new ArrayList<>();
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-1");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(0);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-2");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(1);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-3");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(1);
+            ccfg.setAffinity(new FairAffinityFunction());
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-4");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(0);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-5");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(1);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-4");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(1);
+            ccfg.setAffinity(new FairAffinityFunction());
+
+            res.add(ccfg);
+        }
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterExcludeOldest implements IgnitePredicate<ClusterNode> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() > 1;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterExcludeNode implements IgnitePredicate<ClusterNode> {
+        /** */
+        private final long excludeOrder;
+
+        /**
+         * @param excludeOrder Node order to exclude.
+         */
+        public TestFilterExcludeNode(long excludeOrder) {
+            this.excludeOrder = excludeOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() != excludeOrder;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterIncludeNode implements IgnitePredicate<ClusterNode> {
+        /** */
+        private final long includeOrder;
+
+        /**
+         * @param includeOrder Node order to exclude.
+         */
+        public TestFilterIncludeNode(long includeOrder) {
+            this.includeOrder = includeOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() == includeOrder;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg)
+            throws IgniteSpiException {
+            Object msg0 = ((GridIoMessage)msg).message();
+
+            if (msg0 instanceof GridDhtPartitionsSingleRequest) // Sent in case of exchange timeout.
+                fail("Unexpected message: " + msg0);
+
+            super.sendMessage(node, msg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
new file mode 100644
index 0000000..282c7c8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.communication.tcp.*;
+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 org.jsr166.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class IgniteCache150ClientsTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int CACHES = 10;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setNetworkTimeout(30_000);
+        cfg.setConnectorConfiguration(null);
+        cfg.setPeerClassLoadingEnabled(false);
+        cfg.setTimeServerPortRange(200);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSocketWriteTimeout(200);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setLocalPortRange(200);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(0);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setMaxMissedClientHeartbeats(200);
+
+        cfg.setClientMode(!gridName.equals(getTestGridName(0)));
+
+        CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES];
+
+        for (int i = 0 ; i < ccfgs.length; i++) {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setCacheMode(PARTITIONED);
+            ccfg.setAtomicityMode(i % 2 == 0 ? ATOMIC : TRANSACTIONAL);
+            ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+            ccfg.setBackups(1);
+
+            ccfg.setName("cache-" + i);
+
+            ccfgs[i] = ccfg;
+        }
+
+        cfg.setCacheConfiguration(ccfgs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void test150Clients() throws Exception {
+        Ignite srv = startGrid(0);
+
+        assertFalse(srv.configuration().isClientMode());
+
+        final int CLIENTS = 150;
+
+        final AtomicInteger idx = new AtomicInteger(1);
+
+        final CountDownLatch latch = new CountDownLatch(CLIENTS);
+
+        final List<String> cacheNames = new ArrayList<>();
+
+        for (int i = 0; i < CACHES; i++)
+            cacheNames.add("cache-" + i);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                boolean cnt = false;
+
+                try {
+                    Ignite ignite = startGrid(idx.getAndIncrement());
+
+                    assertTrue(ignite.configuration().isClientMode());
+                    assertTrue(ignite.cluster().localNode().isClient());
+
+                    latch.countDown();
+
+                    cnt = true;
+
+                    log.info("Started [node=" + ignite.name() + ", left=" + latch.getCount() + ']');
+
+                    ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+
+                    while (latch.getCount() > 0) {
+                        Thread.sleep(1000);
+
+                        IgniteCache<Object, Object> cache = ignite.cache(cacheNames.get(rnd.nextInt(0, CACHES)));
+
+                        Integer key = rnd.nextInt(0, 100_000);
+
+                        cache.put(key, 0);
+
+                        assertNotNull(cache.get(key));
+                    }
+
+                    return null;
+                }
+                finally {
+                    if (!cnt)
+                        latch.countDown();
+                }
+            }
+        }, CLIENTS, "start-client");
+
+        fut.get();
+
+        log.info("Started all clients.");
+
+        checkNodes(CLIENTS + 1);
+    }
+
+    /**
+     * @param expCnt Expected number of nodes.
+     */
+    private void checkNodes(int expCnt) {
+        assertEquals(expCnt, G.allGrids().size());
+
+        long topVer = -1L;
+
+        for (Ignite ignite : G.allGrids()) {
+            log.info("Check node: " + ignite.name());
+
+            if (topVer == -1L)
+                topVer = ignite.cluster().topologyVersion();
+            else
+                assertEquals("Unexpected topology version for node: " + ignite.name(),
+                    topVer,
+                    ignite.cluster().topologyVersion());
+
+            assertEquals("Unexpected number of nodes for node: " + ignite.name(),
+                expCnt,
+                ignite.cluster().nodes().size());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index e5d30b6..d60a0c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -464,6 +464,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
     /**
      * @param nearCache If {@code true} creates near cache on client.
+     * @param srvNode If {@code true} creates client cache on server node.
      * @throws Exception If failed.
      */
     private void clientOnlyCacheStart(boolean nearCache, boolean srvNode) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 6295a4d..d13a99d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -102,6 +102,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class);
         suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class);
         suite.addTestSuite(IgniteDynamicClientCacheStartSelfTest.class);
+        suite.addTestSuite(IgniteDynamicCacheStartNoExchangeTimeoutTest.class);
 
         suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
new file mode 100644
index 0000000..98f9181
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+
+/**
+ *
+ */
+public class IgniteClientTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Client Test Suite");
+
+        suite.addTestSuite(IgniteCache150ClientsTest.class);
+
+        return suite;
+    }
+}


[19/33] 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-959
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));


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

Posted by sb...@apache.org.
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-959
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);
     }
 


[25/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1048' into ignite-sprint-7

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

Branch: refs/heads/ignite-959
Commit: dfe95bdfb98663134809ca456f63e100fb1e7c26
Parents: 2bb6e0f a2d2c9b
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 16:38:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 16:38:38 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtLocalPartition.java        | 3 ++-
 .../cache/distributed/dht/atomic/GridDhtAtomicCache.java    | 9 ++++-----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[14/33] incubator-ignite git commit: # ignite-sprint-7: fix compilation after merge.

Posted by sb...@apache.org.
# ignite-sprint-7: fix compilation after merge.


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

Branch: refs/heads/ignite-959
Commit: 5d8e404f8d871f96a11238d865c6a7c656080638
Parents: cd06e1d
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 16:28:29 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 16:28:29 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d8e404f/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 14093ba..181ff0c 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
@@ -62,7 +62,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);


[02/33] incubator-ignite git commit: 1.1.4-SNAPSHOT

Posted by sb...@apache.org.
1.1.4-SNAPSHOT


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

Branch: refs/heads/ignite-959
Commit: 489ab0f1a6b24460bf1896798780b2203a1f5d27
Parents: 8c2553e
Author: Ignite Teamcity <ig...@apache.org>
Authored: Wed Jun 17 16:54:34 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Wed Jun 17 16:54:34 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/pom.xml                               | 2 +-
 modules/aws/pom.xml                               | 2 +-
 modules/clients/pom.xml                           | 2 +-
 modules/cloud/pom.xml                             | 2 +-
 modules/codegen/pom.xml                           | 2 +-
 modules/core/pom.xml                              | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/extdata/p2p/pom.xml                       | 2 +-
 modules/extdata/uri/pom.xml                       | 2 +-
 modules/gce/pom.xml                               | 2 +-
 modules/geospatial/pom.xml                        | 2 +-
 modules/hadoop/pom.xml                            | 2 +-
 modules/hibernate/pom.xml                         | 2 +-
 modules/indexing/pom.xml                          | 2 +-
 modules/jcl/pom.xml                               | 2 +-
 modules/jta/pom.xml                               | 2 +-
 modules/log4j/pom.xml                             | 2 +-
 modules/mesos/pom.xml                             | 2 +-
 modules/rest-http/pom.xml                         | 2 +-
 modules/scalar-2.10/pom.xml                       | 2 +-
 modules/scalar/pom.xml                            | 2 +-
 modules/schedule/pom.xml                          | 2 +-
 modules/schema-import/pom.xml                     | 2 +-
 modules/slf4j/pom.xml                             | 2 +-
 modules/spark-2.10/pom.xml                        | 2 +-
 modules/spark/pom.xml                             | 2 +-
 modules/spring/pom.xml                            | 2 +-
 modules/ssh/pom.xml                               | 2 +-
 modules/tools/pom.xml                             | 2 +-
 modules/urideploy/pom.xml                         | 2 +-
 modules/visor-console-2.10/pom.xml                | 2 +-
 modules/visor-console/pom.xml                     | 2 +-
 modules/visor-plugins/pom.xml                     | 2 +-
 modules/web/pom.xml                               | 2 +-
 modules/yardstick/pom.xml                         | 2 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 016b3ce..834b5b4 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index e04f183..e9cc980 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index f7fdfbc..1d3b1f3 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 7842c47..6b28630 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 579c6a0..0c9cd3d 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 9d375cc..5b7f518 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index a0ee32e..d84cef0 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 5171bd6..1be362f 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.1.2-SNAPSHOT
+ignite.version=1.1.4-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 4546112..d849117 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index be0c73b..e62738e 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index e0a5d1d..ff582bd 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index d9bd236..bcad0b0 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index dd7c9b7..1884271 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index d7e098c..dac4b3b 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 18a3579..91811d2 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 516df8a..d100d4a 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index ca5ef52..d44f9a8 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 533f592..7cb43cc 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 173fe7c..1c44422 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 243ac06..686a3ce 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 63bfd36..2ce2de1 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 3fb2470..5863f4a 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 049fe65..86d0ab6 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 8523d34..59c5d63 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 5284182..90e7031 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 46cc4e7..f664aa0 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index e62ca24..577ae57 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index d56cb2b..d43d337 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index d65d173..3e9ac77 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 0bb221f..7b07260 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 70a23b0..5d7bf4c 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index bb7aefc..c3d9045 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index ef6abc0..f8627da 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index ef217ed..dd5cbe0 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 6b69c63..4e8a50f 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 39c7f5a..84a8336 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/489ab0f1/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 68d610e..64f9e25 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.2-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[18/33] 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-959
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));


[10/33] incubator-ignite git commit: 1.1.6-SNAPSHOT

Posted by sb...@apache.org.
1.1.6-SNAPSHOT


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

Branch: refs/heads/ignite-959
Commit: d0063897052b97398a93bd3ac32fd573d0d7f3d3
Parents: 188cd82
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Jun 19 12:56:32 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Jun 19 12:56:32 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/pom.xml                               | 2 +-
 modules/aws/pom.xml                               | 2 +-
 modules/clients/pom.xml                           | 2 +-
 modules/cloud/pom.xml                             | 2 +-
 modules/codegen/pom.xml                           | 2 +-
 modules/core/pom.xml                              | 2 +-
 modules/core/src/main/resources/ignite.properties | 2 +-
 modules/extdata/p2p/pom.xml                       | 2 +-
 modules/extdata/uri/pom.xml                       | 2 +-
 modules/gce/pom.xml                               | 2 +-
 modules/geospatial/pom.xml                        | 2 +-
 modules/hadoop/pom.xml                            | 2 +-
 modules/hibernate/pom.xml                         | 2 +-
 modules/indexing/pom.xml                          | 2 +-
 modules/jcl/pom.xml                               | 2 +-
 modules/jta/pom.xml                               | 2 +-
 modules/log4j/pom.xml                             | 2 +-
 modules/mesos/pom.xml                             | 2 +-
 modules/rest-http/pom.xml                         | 2 +-
 modules/scalar-2.10/pom.xml                       | 2 +-
 modules/scalar/pom.xml                            | 2 +-
 modules/schedule/pom.xml                          | 2 +-
 modules/schema-import/pom.xml                     | 2 +-
 modules/slf4j/pom.xml                             | 2 +-
 modules/spark-2.10/pom.xml                        | 2 +-
 modules/spark/pom.xml                             | 2 +-
 modules/spring/pom.xml                            | 2 +-
 modules/ssh/pom.xml                               | 2 +-
 modules/tools/pom.xml                             | 2 +-
 modules/urideploy/pom.xml                         | 2 +-
 modules/visor-console-2.10/pom.xml                | 2 +-
 modules/visor-console/pom.xml                     | 2 +-
 modules/visor-plugins/pom.xml                     | 2 +-
 modules/web/pom.xml                               | 2 +-
 modules/yardstick/pom.xml                         | 2 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 834b5b4..bcff11e 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index e9cc980..5eea18d 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 1d3b1f3..05ecbc1 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 6b28630..7c4d2c8 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 0c9cd3d..bb32689 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 5b7f518..1ea5fe5 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index d84cef0..79fbc2f 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 1be362f..b25b2ca 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.1.4-SNAPSHOT
+ignite.version=1.1.6-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index d849117..26f2e50 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index e62738e..1690ab9 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index ff582bd..127c7a4 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index bcad0b0..e28c139 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 1884271..637f097 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index dac4b3b..d001b02 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 91811d2..b044092 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index d100d4a..b764c96 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index d44f9a8..531e6f2 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 7cb43cc..af62389 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 1c44422..07bee9e 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 686a3ce..2d7fe6d 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 2ce2de1..c4b8d60 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 5863f4a..a048939 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 86d0ab6..735035f 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 59c5d63..2898327 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 90e7031..fb7c86e 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index f664aa0..1b8b04e 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 577ae57..8c00ec8 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index d43d337..35638fe 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 3e9ac77..23ae1df 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 7b07260..9e5685d 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 5d7bf4c..0add261 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index c3d9045..8806438 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index f8627da..8c6e4c9 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index dd5cbe0..e4162c0 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 4e8a50f..e6a23bd 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 84a8336..5307447 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 64f9e25..44cb0ce 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[15/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite 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/1c660781
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/1c660781
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/1c660781

Branch: refs/heads/ignite-959
Commit: 1c660781355f870271760dd6dc0392741fdb15f6
Parents: 5d8e404 285d790
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 23 10:00:32 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 23 10:00:32 2015 +0700

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 49 +++++++++++++++-----
 .../communication/tcp/TcpCommunicationSpi.java  | 12 +++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |  1 -
 3 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[05/33] incubator-ignite git commit: # ignite-sprint-7 disabled shmem for test

Posted by sb...@apache.org.
# ignite-sprint-7 disabled shmem for 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/edc9a1c4
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/edc9a1c4
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/edc9a1c4

Branch: refs/heads/ignite-959
Commit: edc9a1c4b13afb8ef36a961f4d5b252135b916b6
Parents: 044f17d
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 16:31:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 16:31:57 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/distributed/IgniteCacheManyClientsTest.java    | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/edc9a1c4/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 c3223a2..48884de 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
@@ -63,6 +63,7 @@ public class IgniteCacheManyClientsTest extends GridCommonAbstractTest {
         cfg.setTimeServerPortRange(200);
 
         ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setLocalPortRange(200);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(2 * 60_000);


[06/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite 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/9458f125
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/9458f125
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/9458f125

Branch: refs/heads/ignite-959
Commit: 9458f12554579fd71a26f56d7bf191d0ec51173e
Parents: edc9a1c ad0a026
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 19 09:55:02 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 19 09:55:02 2015 +0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |  9 +++++
 .../processors/cache/IgniteCacheProxy.java      |  5 +++
 .../processors/query/GridQueryProcessor.java    |  7 ++++
 .../communication/tcp/TcpCommunicationSpi.java  | 42 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 35 ++++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 42 ++++++++++++++------
 .../tcp/internal/TcpDiscoveryNode.java          | 18 +++++++++
 .../apache/ignite/internal/GridSelfTest.java    | 20 ++++++----
 .../DataStreamerMultiThreadedSelfTest.java      |  3 ++
 9 files changed, 132 insertions(+), 49 deletions(-)
----------------------------------------------------------------------



[03/33] incubator-ignite git commit: Merge branch 'ignite-sprint-6' into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' into ignite-sprint-7

Conflicts:
	examples/pom.xml
	modules/aop/pom.xml
	modules/aws/pom.xml
	modules/clients/pom.xml
	modules/cloud/pom.xml
	modules/codegen/pom.xml
	modules/core/pom.xml
	modules/core/src/main/resources/ignite.properties
	modules/extdata/p2p/pom.xml
	modules/extdata/uri/pom.xml
	modules/gce/pom.xml
	modules/geospatial/pom.xml
	modules/hadoop/pom.xml
	modules/hibernate/pom.xml
	modules/indexing/pom.xml
	modules/jcl/pom.xml
	modules/jta/pom.xml
	modules/log4j/pom.xml
	modules/mesos/pom.xml
	modules/rest-http/pom.xml
	modules/scalar-2.10/pom.xml
	modules/scalar/pom.xml
	modules/schedule/pom.xml
	modules/schema-import/pom.xml
	modules/slf4j/pom.xml
	modules/spark-2.10/pom.xml
	modules/spark/pom.xml
	modules/spring/pom.xml
	modules/ssh/pom.xml
	modules/tools/pom.xml
	modules/urideploy/pom.xml
	modules/visor-console-2.10/pom.xml
	modules/visor-console/pom.xml
	modules/visor-plugins/pom.xml
	modules/web/pom.xml
	modules/yardstick/pom.xml
	pom.xml


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

Branch: refs/heads/ignite-959
Commit: c6a0f24adf62c58eb0b739183bbd8d572827ad14
Parents: 489ab0f d874b00
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Thu Jun 18 11:21:44 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Thu Jun 18 11:21:44 2015 +0300

----------------------------------------------------------------------
 .../s3/S3CheckpointManagerSelfTest.java         |   2 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   4 +-
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |   2 +-
 .../s3/S3SessionCheckpointSelfTest.java         |   2 +-
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |   2 +-
 .../client/GridClientConfiguration.java         |   4 +-
 .../affinity/AffinityTopologyVersion.java       |   7 -
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/IgniteCacheProxy.java      |   7 +
 .../distributed/dht/GridDhtLocalPartition.java  |  56 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../dht/GridDhtPartitionsReservation.java       | 292 +++++++++
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../cache/distributed/dht/GridReservable.java   |  35 +
 .../dht/preloader/GridDhtPartitionMap.java      |  26 +-
 .../cache/query/GridCacheQueryManager.java      |  33 -
 .../cache/query/GridCacheTwoStepQuery.java      |  22 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../processors/query/GridQueryIndexing.java     |  14 +-
 .../processors/query/GridQueryProcessor.java    |  14 +-
 .../messages/GridQueryNextPageResponse.java     |  34 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 111 +++-
 .../apache/ignite/internal/util/GridDebug.java  |  19 +
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 +++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 +++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 +
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 +
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 +++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 +
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |  24 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |  30 +-
 .../IgniteCacheAtomicStopBusySelfTest.java      |   8 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  19 +-
 ...gniteCacheTransactionalStopBusySelfTest.java |   8 +-
 .../junits/GridTestKernalContext.java           |   2 +-
 .../junits/common/GridCommonAbstractTest.java   |   8 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   4 +
 .../processors/query/h2/IgniteH2Indexing.java   |  79 ++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  49 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 332 +++++++---
 .../query/h2/twostep/GridMergeIndex.java        |  17 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   7 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 650 ++++++++++++++++---
 .../query/h2/twostep/GridResultPage.java        |  21 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   3 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++++
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  36 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 383 +++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 53 files changed, 3238 insertions(+), 341 deletions(-)
----------------------------------------------------------------------



[20/33] incubator-ignite git commit: # ignite-sprint-7 disabled hanging test

Posted by sb...@apache.org.
# ignite-sprint-7 disabled hanging 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/c7ba1541
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/c7ba1541
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/c7ba1541

Branch: refs/heads/ignite-959
Commit: c7ba1541959b78451e81fb701550894b09fe6eb9
Parents: 1c66078
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 12:55:17 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 12:55:17 2015 +0300

----------------------------------------------------------------------
 .../replicated/GridCacheReplicatedFailoverSelfTest.java         | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7ba1541/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
index 3461dd4..4269ea6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
@@ -28,6 +28,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheReplicatedFailoverSelfTest extends GridCacheAbstractFailoverTxSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return REPLICATED;
     }


[04/33] incubator-ignite git commit: # ignite-sprint-7 do not run TaskDiscoveryListener in discovery thread

Posted by sb...@apache.org.
# ignite-sprint-7 do not run TaskDiscoveryListener in discovery thread


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

Branch: refs/heads/ignite-959
Commit: 044f17deae5a16c0d463633b75640d04a125f5d2
Parents: c6a0f24
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jun 18 15:36:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jun 18 15:36:44 2015 +0300

----------------------------------------------------------------------
 .../processors/task/GridTaskProcessor.java      |  22 ++-
 .../GridTaskFailoverAffinityRunTest.java        | 170 +++++++++++++++++++
 .../testsuites/IgniteComputeGridTestSuite.java  |   1 +
 3 files changed, 184 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index 88713a1..bb9ff50 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -1171,17 +1171,21 @@ public class GridTaskProcessor extends GridProcessorAdapter {
         @Override public void onEvent(Event evt) {
             assert evt.type() == EVT_NODE_FAILED || evt.type() == EVT_NODE_LEFT;
 
-            UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
+            final UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
-            lock.readLock();
+            ctx.closure().runLocalSafe(new Runnable() {
+                @Override public void run() {
+                    lock.readLock();
 
-            try {
-                for (GridTaskWorker<?, ?> task : tasks.values())
-                    task.onNodeLeft(nodeId);
-            }
-            finally {
-                lock.readUnlock();
-            }
+                    try {
+                        for (GridTaskWorker<?, ?> task : tasks.values())
+                            task.onNodeLeft(nodeId);
+                    }
+                    finally {
+                        lock.readUnlock();
+                    }
+                }
+            }, false);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
new file mode 100644
index 0000000..7ddd966
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFailoverAffinityRunTest.java
@@ -0,0 +1,170 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+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.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class GridTaskFailoverAffinityRunTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean clientMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        boolean client = clientMode && gridName.equals(getTestGridName(0));
+
+        if (client) {
+            cfg.setClientMode(true);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        }
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeRestart() throws Exception {
+        clientMode = false;
+
+        nodeRestart();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeRestartClient() throws Exception {
+        clientMode = true;
+
+        nodeRestart();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void nodeRestart() throws Exception {
+        startGridsMultiThreaded(4);
+
+        assertEquals((Boolean)clientMode, grid(0).configuration().isClientMode());
+
+        IgniteCompute comp = grid(0).compute().withAsync();
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final AtomicInteger gridIdx = new AtomicInteger(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int grid = gridIdx.getAndIncrement();
+
+                while (!stop.get()) {
+                    stopGrid(grid);
+
+                    startGrid(grid);
+                }
+
+                return null;
+            }
+        }, 2, "restart-thread");
+
+        try {
+            long stopTime = System.currentTimeMillis() + 60_000;
+
+            while (System.currentTimeMillis() < stopTime) {
+                Collection<IgniteFuture<?>> futs = new ArrayList<>(1000);
+
+                for (int i = 0; i < 1000; i++) {
+                    comp.affinityCall(null, i, new TestJob());
+
+                    IgniteFuture<?> fut0 = comp.future();
+
+                    assertNotNull(fut0);
+
+                    futs.add(fut0);
+                }
+
+                for (IgniteFuture<?> fut0 : futs) {
+                    try {
+                        fut0.get();
+                    }
+                    catch (IgniteException ignore) {
+                        // No-op.
+                    }
+                }
+            }
+        }
+        finally {
+            stop.set(true);
+
+            fut.get();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestJob implements IgniteCallable<Object> {
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            Thread.sleep(1000);
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/044f17de/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 82fc5e0..baf425c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -93,6 +93,7 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(GridMultinodeRedeployIsolatedModeSelfTest.class);
         suite.addTestSuite(IgniteComputeEmptyClusterGroupTest.class);
         suite.addTestSuite(IgniteComputeTopologyExceptionTest.class);
+        suite.addTestSuite(GridTaskFailoverAffinityRunTest.class);
 
         return suite;
     }


[32/33] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1045' into ignite-sprint-7

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

Branch: refs/heads/ignite-959
Commit: c270bd812df9cdf43d55ce0f546657e2f0e8d1da
Parents: bf1ea21 a2141c6
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jun 24 12:17:57 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jun 24 12:17:57 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |  46 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++++++++++
 .../distributed/IgniteCache150ClientsTest.java  | 189 ++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../testsuites/IgniteClientTestSuite.java       |  38 ++
 6 files changed, 733 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


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


[23/33] incubator-ignite git commit: [IGNITE-980]: Investigate if we should close() the FileSystems in MapRed task implementations.

Posted by sb...@apache.org.
[IGNITE-980]: Investigate if we should close() the FileSystems in MapRed task implementations.


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

Branch: refs/heads/ignite-959
Commit: 3b49184b48ca55d5cf4dcdd7485ab8aba1bb9cff
Parents: 36433bc
Author: iveselovskiy <iv...@gridgain.com>
Authored: Tue Jun 23 13:58:06 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Tue Jun 23 13:58:06 2015 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 modules/hadoop/pom.xml                          |  78 ------
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +--
 .../internal/processors/hadoop/HadoopUtils.java | 237 -------------------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 ++++++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 ++-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 23 files changed, 237 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
index 51faf5d..d3735d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
@@ -55,12 +55,14 @@ public interface HadoopJobInfo extends Serializable {
      * This method will be called once for the same ID on one node, though it can be called on the same host
      * multiple times from different processes (in case of multiple nodes on the same host or external execution).
      *
+     * @param jobCls The job class.
      * @param jobId Job ID.
      * @param log Logger.
      * @return Job.
      * @throws IgniteCheckedException If failed.
      */
-    HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
+    public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+        HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
 
     /**
      * @return Number of reducers configured for job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
index ce67c57..f21a1e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
@@ -28,10 +28,9 @@ public interface HadoopCounterWriter {
     /**
      * Writes counters of given job to some statistics storage.
      *
-     * @param jobInfo Job info.
-     * @param jobId Job id.
+     * @param job The job.
      * @param cntrs Counters.
      * @throws IgniteCheckedException If failed.
      */
-    public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs) throws IgniteCheckedException;
+    public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 637f097..e300ba0 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -143,84 +143,6 @@
                 <artifactId>maven-surefire-plugin</artifactId>
                 <version>2.17</version>
                 <configuration>
-                    <classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-annotations</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-auth</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-codec:commons-codec</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.httpcomponents:httpclient</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.httpcomponents:httpcore</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-common</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.guava:guava</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-cli:commons-cli</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.commons:commons-math3</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>xmlenc:xmlenc</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-httpclient:commons-httpclient</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-net:commons-net</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.servlet:servlet-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.mortbay.jetty:jetty</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.mortbay.jetty:jetty-util</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-core</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-json</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jettison:jettison</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.xml.bind:jaxb-impl</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-jaxrs</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-xc</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-server</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>asm:asm</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>tomcat:jasper-compiler</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>tomcat:jasper-runtime</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.servlet.jsp:jsp-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-el:commons-el</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-logging:commons-logging</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>net.java.dev.jets3t:jets3t</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.jamesmurty.utils:java-xmlbuilder</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-core-asl</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-mapper-asl
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.avro:avro</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.thoughtworks.paranamer:paranamer</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.xerial.snappy:snappy-java</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.protobuf:protobuf-java</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.jcraft:jsch</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.code.findbugs:jsr305</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.zookeeper:zookeeper</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>org.apache.commons:commons-compress</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.tukaani:xz</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-hdfs</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-daemon:commons-daemon</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-mapreduce-client-common
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-common</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.xml.bind:jaxb-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.xml.stream:stax-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.activation:activation</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.inject:guice</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.inject:javax.inject</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey.contribs:jersey-guice</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-client</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-client</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-server-common
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.inject.extensions:guice-servlet
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>io.netty:netty</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-mapreduce-client-core
-                        </classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-beanutils:commons-beanutils</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.hamcrest:hamcrest-core</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-collections:commons-collections</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-http</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-io:commons-io</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-lang:commons-lang</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>commons-configuration:commons-configuration
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-server</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-util</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-io</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>aopalliance:aopalliance</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.beust:jcommander</classpathDependencyExcludes>
-                    </classpathDependencyExcludes>
                 </configuration>
             </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
index d910507..f76b354 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
@@ -25,6 +25,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -48,11 +49,15 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO;
 
     /** {@inheritDoc} */
-    @Override public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs)
+    @Override public void write(HadoopJob job, HadoopCounters cntrs)
         throws IgniteCheckedException {
 
         Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration();
 
+        final HadoopJobInfo jobInfo = job.info();
+
+        final HadoopJobId jobId = job.id();
+
         for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
             hadoopCfg.set(e.getKey(), e.getValue());
 
@@ -72,7 +77,7 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
         try {
             hadoopCfg.set(MRJobConfig.USER_NAME, user);
 
-            FileSystem fs = HadoopUtils.fileSystemForMrUser(jobStatPath.toUri(), hadoopCfg, true);
+            FileSystem fs = ((HadoopV2Job)job).fileSystem(jobStatPath.toUri(), hadoopCfg);
 
             fs.mkdirs(jobStatPath);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index eb98ff9..0988fe0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -67,6 +67,28 @@ public class HadoopClassLoader extends URLClassLoader {
     private final String name;
 
     /**
+     * Gets name for Job class loader. The name is specific for local node id.
+     * @param locNodeId The local node id.
+     * @return The class loader name.
+     */
+    public static String nameForJob(UUID locNodeId) {
+        return "hadoop-job-node-" + locNodeId.toString();
+    }
+
+    /**
+     * Gets name for the task class loader. Task class loader
+     * @param info The task info.
+     * @param prefix Get only prefix (without task type and number)
+     * @return The class loader name.
+     */
+    public static String nameForTask(HadoopTaskInfo info, boolean prefix) {
+        if (prefix)
+            return "hadoop-task-" + info.jobId() + "-";
+        else
+            return "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber();
+    }
+
+    /**
      * @param urls Urls.
      */
     public HadoopClassLoader(URL[] urls, String name) {
@@ -568,4 +590,11 @@ public class HadoopClassLoader extends URLClassLoader {
     @Override public String toString() {
         return S.toString(HadoopClassLoader.class, this);
     }
+
+    /**
+     * Getter for name field.
+     */
+    public String name() {
+        return name;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index 2e855d0..95e03c8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -48,9 +47,6 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     /** User name. */
     private String user;
 
-    /** */
-    private static volatile Class<?> jobCls;
-
     /**
      * Default constructor required by {@link Externalizable}.
      */
@@ -82,24 +78,15 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
-        try {
-            Class<?> jobCls0 = jobCls;
+    @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+            HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        assert jobCls != null;
 
-            if (jobCls0 == null) { // It is enough to have only one class loader with only Hadoop classes.
-                synchronized (HadoopDefaultJobInfo.class) {
-                    if ((jobCls0 = jobCls) == null) {
-                        HadoopClassLoader ldr = new HadoopClassLoader(null, "hadoop-job");
-
-                        jobCls = jobCls0 = ldr.loadClass(HadoopV2Job.class.getName());
-                    }
-                }
-            }
-
-            Constructor<?> constructor = jobCls0.getConstructor(HadoopJobId.class, HadoopDefaultJobInfo.class,
-                IgniteLogger.class);
+        try {
+            Constructor<? extends HadoopJob> constructor = jobCls.getConstructor(HadoopJobId.class,
+                HadoopDefaultJobInfo.class, IgniteLogger.class);
 
-            return (HadoopJob)constructor.newInstance(jobId, this, log);
+            return constructor.newInstance(jobId, this, log);
         }
         // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call.
         catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
index 68a9ef6..f87e610 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
@@ -26,16 +26,10 @@ import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.ignite.*;
-import org.apache.ignite.hadoop.fs.v1.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
 
 import java.io.*;
-import java.net.*;
 import java.util.*;
 
 /**
@@ -63,34 +57,6 @@ public class HadoopUtils {
     /** Old reducer class attribute. */
     private static final String OLD_REDUCE_CLASS_ATTR = "mapred.reducer.class";
 
-    /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
-    private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
-        new HadoopLazyConcurrentMap.ValueFactory<FsCacheKey, FileSystem>() {
-            @Override public FileSystem createValue(FsCacheKey key) {
-                try {
-                    assert key != null;
-
-                    // Explicitly disable FileSystem caching:
-                    URI uri = key.uri();
-
-                    String scheme = uri.getScheme();
-
-                    // Copy the configuration to avoid altering the external object.
-                    Configuration cfg = new Configuration(key.configuration());
-
-                    String prop = HadoopUtils.disableFsCachePropertyName(scheme);
-
-                    cfg.setBoolean(prop, true);
-
-                    return FileSystem.get(uri, cfg, key.user());
-                }
-                catch (IOException | InterruptedException ioe) {
-                    throw new IgniteException(ioe);
-                }
-            }
-        }
-    );
-
     /**
      * Constructor.
      */
@@ -378,207 +344,4 @@ public class HadoopUtils {
         }
     }
 
-    /**
-     * Gets non-null user name as per the Hadoop viewpoint.
-     * @param cfg the Hadoop job configuration, may be null.
-     * @return the user name, never null.
-     */
-    private static String getMrHadoopUser(Configuration cfg) throws IOException {
-        String user = cfg.get(MRJobConfig.USER_NAME);
-
-        if (user == null)
-            user = IgniteHadoopFileSystem.getFsHadoopUser();
-
-        return user;
-    }
-
-    /**
-     * Common method to get the V1 file system in MapRed engine.
-     * It creates the filesystem for the user specified in the
-     * configuration with {@link MRJobConfig#USER_NAME} property.
-     * @param uri the file system uri.
-     * @param cfg the configuration.
-     * @return the file system
-     * @throws IOException
-     */
-    public static FileSystem fileSystemForMrUser(@Nullable URI uri, Configuration cfg, boolean doCacheFs) throws IOException {
-        final String usr = getMrHadoopUser(cfg);
-
-        assert usr != null;
-
-        if (uri == null)
-            uri = FileSystem.getDefaultUri(cfg);
-
-        final FileSystem fs;
-
-        if (doCacheFs) {
-            try {
-                fs = getWithCaching(uri, cfg, usr);
-            }
-            catch (IgniteException ie) {
-                throw new IOException(ie);
-            }
-        }
-        else {
-            try {
-                fs = FileSystem.get(uri, cfg, usr);
-            }
-            catch (InterruptedException ie) {
-                Thread.currentThread().interrupt();
-
-                throw new IOException(ie);
-            }
-        }
-
-        assert fs != null;
-        assert !(fs instanceof IgniteHadoopFileSystem) || F.eq(usr, ((IgniteHadoopFileSystem)fs).user());
-
-        return fs;
-    }
-
-    /**
-     * Note that configuration is not a part of the key.
-     * It is used solely to initialize the first instance
-     * that is created for the key.
-     */
-    public static final class FsCacheKey {
-        /** */
-        private final URI uri;
-
-        /** */
-        private final String usr;
-
-        /** */
-        private final String equalityKey;
-
-        /** */
-        private final Configuration cfg;
-
-        /**
-         * Constructor
-         */
-        public FsCacheKey(URI uri, String usr, Configuration cfg) {
-            assert uri != null;
-            assert usr != null;
-            assert cfg != null;
-
-            this.uri = fixUri(uri, cfg);
-            this.usr = usr;
-            this.cfg = cfg;
-
-            this.equalityKey = createEqualityKey();
-        }
-
-        /**
-         * Creates String key used for equality and hashing.
-         */
-        private String createEqualityKey() {
-            GridStringBuilder sb = new GridStringBuilder("(").a(usr).a(")@");
-
-            if (uri.getScheme() != null)
-                sb.a(uri.getScheme().toLowerCase());
-
-            sb.a("://");
-
-            if (uri.getAuthority() != null)
-                sb.a(uri.getAuthority().toLowerCase());
-
-            return sb.toString();
-        }
-
-        /**
-         * The URI.
-         */
-        public URI uri() {
-            return uri;
-        }
-
-        /**
-         * The User.
-         */
-        public String user() {
-            return usr;
-        }
-
-        /**
-         * The Configuration.
-         */
-        public Configuration configuration() {
-            return cfg;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("SimplifiableIfStatement")
-        @Override public boolean equals(Object obj) {
-            if (obj == this)
-                return true;
-
-            if (obj == null || getClass() != obj.getClass())
-                return false;
-
-            return equalityKey.equals(((FsCacheKey)obj).equalityKey);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return equalityKey.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return equalityKey;
-        }
-    }
-
-    /**
-     * Gets FileSystem caching it in static Ignite cache. The cache is a singleton
-     * for each class loader.
-     *
-     * <p/>Note that the file systems in the cache are keyed by a triplet {scheme, authority, user}.
-     * The Configuration is not a part of the key. This means that for the given key file system is
-     * initialized only once with the Configuration passed in upon the file system creation.
-     *
-     * @param uri The file system URI.
-     * @param cfg The configuration.
-     * @param usr The user to create file system for.
-     * @return The file system: either created, or taken from the cache.
-     */
-    private static FileSystem getWithCaching(URI uri, Configuration cfg, String usr) {
-        FsCacheKey key = new FsCacheKey(uri, usr, cfg);
-
-        return fileSysLazyMap.getOrCreate(key);
-    }
-
-    /**
-     * Gets the property name to disable file system cache.
-     * @param scheme The file system URI scheme.
-     * @return The property name. If scheme is null,
-     * returns "fs.null.impl.disable.cache".
-     */
-    public static String disableFsCachePropertyName(@Nullable String scheme) {
-        return String.format("fs.%s.impl.disable.cache", scheme);
-    }
-
-    /**
-     * Takes Fs URI using logic similar to that used in FileSystem#get(1,2,3).
-     * @param uri0 The uri.
-     * @param cfg The cfg.
-     * @return Correct URI.
-     */
-    public static URI fixUri(URI uri0, Configuration cfg) {
-        if (uri0 == null)
-            return FileSystem.getDefaultUri(cfg);
-
-        String scheme = uri0.getScheme();
-        String authority = uri0.getAuthority();
-
-        if (authority == null) {
-            URI dfltUri = FileSystem.getDefaultUri(cfg);
-
-            if (scheme == null || (scheme.equals(dfltUri.getScheme()) && dfltUri.getAuthority() != null))
-                return dfltUri;
-        }
-
-        return uri0;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
index dd679de..ef04b0f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.hadoop;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.security.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -76,7 +77,7 @@ public class SecondaryFileSystemProvider {
         }
 
         // Disable caching:
-        String prop = HadoopUtils.disableFsCachePropertyName(uri.getScheme());
+        String prop = HadoopFileSystemsUtils.disableFsCachePropertyName(uri.getScheme());
 
         cfg.setBoolean(prop, true);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
index d90bc28..382bbd0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.hadoop.fs;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
+import org.jetbrains.annotations.*;
 
 /**
  * Utilities for configuring file systems to support the separate working directory per each thread.
@@ -37,4 +38,14 @@ public class HadoopFileSystemsUtils {
         cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
                 HadoopLocalFileSystemV2.class.getName());
     }
+
+    /**
+     * Gets the property name to disable file system cache.
+     * @param scheme The file system URI scheme.
+     * @return The property name. If scheme is null,
+     * returns "fs.null.impl.disable.cache".
+     */
+    public static String disableFsCachePropertyName(@Nullable String scheme) {
+        return String.format("fs.%s.impl.disable.cache", scheme);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
index 71b38c4..c7565d3 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -51,6 +51,8 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
      */
     public HadoopLazyConcurrentMap(ValueFactory<K, V> factory) {
         this.factory = factory;
+
+        assert getClass().getClassLoader() == Ignite.class.getClassLoader();
     }
 
     /**
@@ -105,6 +107,9 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
         closeLock.writeLock().lock();
 
         try {
+            if (closed)
+                return;
+
             closed = true;
 
             Exception err = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 2f07817..194ae33 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -82,6 +83,9 @@ public class HadoopJobTracker extends HadoopComponent {
     /** Component busy lock. */
     private GridSpinReadWriteLock busyLock;
 
+    /** Class to create HadoopJob instances from. */
+    private Class<? extends HadoopJob> jobCls;
+
     /** Closure to check result of async transform of system cache. */
     private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
         @Override public void apply(IgniteInternalFuture<?> gridFut) {
@@ -95,12 +99,27 @@ public class HadoopJobTracker extends HadoopComponent {
     };
 
     /** {@inheritDoc} */
-    @Override public void start(HadoopContext ctx) throws IgniteCheckedException {
+    @SuppressWarnings("unchecked")
+    @Override public void start(final HadoopContext ctx) throws IgniteCheckedException {
         super.start(ctx);
 
         busyLock = new GridSpinReadWriteLock();
 
         evtProcSvc = Executors.newFixedThreadPool(1);
+
+        UUID nodeId = ctx.localNodeId();
+
+        assert jobCls == null;
+
+        HadoopClassLoader ldr = new HadoopClassLoader(null, HadoopClassLoader.nameForJob(nodeId));
+
+        try {
+            jobCls = (Class<HadoopV2Job>)ldr.loadClass(HadoopV2Job.class.getName());
+        }
+        catch (Exception ioe) {
+            throw new IgniteCheckedException("Failed to load job class [class="
+                + HadoopV2Job.class.getName() + ']', ioe);
+        }
     }
 
     /**
@@ -838,7 +857,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                             HadoopCounters cntrs = meta.counters();
 
-                            writer.write(job.info(), jobId, cntrs);
+                            writer.write(job, cntrs);
                         }
                     }
                     catch (Exception e) {
@@ -986,7 +1005,7 @@ public class HadoopJobTracker extends HadoopComponent {
                 jobInfo = meta.jobInfo();
             }
 
-            job = jobInfo.createJob(jobId, log);
+            job = jobInfo.createJob(jobCls, jobId, log);
 
             job.initialize(false, ctx.localNodeId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index 040552a..b0b0b8c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.hadoop.shuffle.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.unsafe.*;
@@ -115,7 +116,7 @@ public class HadoopChildProcessRunner {
 
                 assert job == null;
 
-                job = req.jobInfo().createJob(req.jobId(), log);
+                job = req.jobInfo().createJob(HadoopV2Job.class, req.jobId(), log);
 
                 job.initialize(true, nodeDesc.processId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
index d754039..33a218d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop.v2;
 
+import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
@@ -30,15 +31,18 @@ import org.apache.ignite.internal.processors.hadoop.v1.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
 import java.io.*;
 import java.lang.reflect.*;
+import java.net.*;
 import java.util.*;
 import java.util.Queue;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.*;
 
 /**
  * Hadoop job implementation for v2 API.
@@ -70,7 +74,10 @@ public class HadoopV2Job implements HadoopJob {
     private final Queue<Class<? extends HadoopTaskContext>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
 
     /** All created contexts. */
-    private final Queue<Class<?>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
+    private final Queue<Class<? extends HadoopTaskContext>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
+
+    /** File system cache map. */
+    private final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap = createHadoopLazyConcurrentMap();
 
     /** Local node ID */
     private volatile UUID locNodeId;
@@ -103,7 +110,7 @@ public class HadoopV2Job implements HadoopJob {
 
         jobCtx = new JobContextImpl(jobConf, hadoopJobID);
 
-        rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log);
+        rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log, this);
     }
 
     /** {@inheritDoc} */
@@ -134,7 +141,7 @@ public class HadoopV2Job implements HadoopJob {
             Path jobDir = new Path(jobDirPath);
 
             try {
-                FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf, true);
+                FileSystem fs = fileSystem(jobDir.toUri(), jobConf);
 
                 JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
                     jobDir);
@@ -180,6 +187,7 @@ public class HadoopV2Job implements HadoopJob {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException {
         T2<HadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
 
@@ -201,7 +209,7 @@ public class HadoopV2Job implements HadoopJob {
                 // Note that the classloader identified by the task it was initially created for,
                 // but later it may be reused for other tasks.
                 HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(),
-                    "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
+                    HadoopClassLoader.nameForTask(info, false));
 
                 cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 
@@ -243,8 +251,13 @@ public class HadoopV2Job implements HadoopJob {
 
     /** {@inheritDoc} */
     @Override public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException {
+        assert locNodeId != null;
+
         this.locNodeId = locNodeId;
 
+        assert ((HadoopClassLoader)getClass().getClassLoader()).name()
+            .equals(HadoopClassLoader.nameForJob(this.locNodeId));
+
         Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
 
         try {
@@ -274,17 +287,26 @@ public class HadoopV2Job implements HadoopJob {
             // Stop the daemon threads that have been created
             // with the task class loaders:
             while (true) {
-                Class<?> cls = fullCtxClsQueue.poll();
+                Class<? extends HadoopTaskContext> cls = fullCtxClsQueue.poll();
 
                 if (cls == null)
                     break;
 
                 try {
-                    Class<?> daemonCls = cls.getClassLoader().loadClass(HadoopClassLoader.HADOOP_DAEMON_CLASS_NAME);
+                    final ClassLoader ldr = cls.getClassLoader();
 
-                    Method m = daemonCls.getMethod("dequeueAndStopAll");
+                    try {
+                        // Stop Hadoop daemons for this *task*:
+                        stopHadoopFsDaemons(ldr);
+                    }
+                    catch (Exception e) {
+                        if (err == null)
+                            err = e;
+                    }
 
-                    m.invoke(null);
+                    // Also close all the FileSystems cached in
+                    // HadoopLazyConcurrentMap for this *task* class loader:
+                    closeCachedTaskFileSystems(ldr);
                 }
                 catch (Throwable e) {
                     if (err == null)
@@ -297,11 +319,46 @@ public class HadoopV2Job implements HadoopJob {
 
             assert fullCtxClsQueue.isEmpty();
 
+            try {
+                // Close all cached file systems for this *Job*:
+                fsMap.close();
+            }
+            catch (Exception e) {
+                if (err == null)
+                    err = e;
+            }
+
             if (err != null)
                 throw U.cast(err);
         }
     }
 
+    /**
+     * Stops Hadoop Fs daemon threads.
+     * @param ldr The task ClassLoader to stop the daemons for.
+     * @throws Exception On error.
+     */
+    private void stopHadoopFsDaemons(ClassLoader ldr) throws Exception {
+        Class<?> daemonCls = ldr.loadClass(HadoopClassLoader.HADOOP_DAEMON_CLASS_NAME);
+
+        Method m = daemonCls.getMethod("dequeueAndStopAll");
+
+        m.invoke(null);
+    }
+
+    /**
+     * Closes all the file systems user by task
+     * @param ldr The task class loader.
+     * @throws Exception On error.
+     */
+    private void closeCachedTaskFileSystems(ClassLoader ldr) throws Exception {
+        Class<?> clazz = ldr.loadClass(HadoopV2TaskContext.class.getName());
+
+        Method m = clazz.getMethod("close");
+
+        m.invoke(null);
+    }
+
     /** {@inheritDoc} */
     @Override public void prepareTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException {
         rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info));
@@ -331,4 +388,15 @@ public class HadoopV2Job implements HadoopJob {
     public JobConf jobConf() {
         return jobConf;
     }
+
+    /**
+     * Gets file system for this job.
+     * @param uri The uri.
+     * @param cfg The configuration.
+     * @return The file system.
+     * @throws IOException On error.
+     */
+    public FileSystem fileSystem(@Nullable URI uri, Configuration cfg) throws IOException {
+        return fileSystemForMrUserWithCaching(uri, cfg, fsMap);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
index 2f64e77..912cc3f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
@@ -39,9 +39,10 @@ import java.util.*;
  * Provides all resources are needed to the job execution. Downloads the main jar, the configuration and additional
  * files are needed to be placed on local files system.
  */
-public class HadoopV2JobResourceManager {
+class HadoopV2JobResourceManager {
     /** File type Fs disable caching property name. */
-    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME = HadoopUtils.disableFsCachePropertyName("file");
+    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME =
+        HadoopFileSystemsUtils.disableFsCachePropertyName("file");
 
     /** Hadoop job context. */
     private final JobContextImpl ctx;
@@ -61,16 +62,20 @@ public class HadoopV2JobResourceManager {
     /** Staging directory to delivery job jar and config to the work nodes. */
     private Path stagingDir;
 
+    /** The job. */
+    private final HadoopV2Job job;
+
     /**
      * Creates new instance.
      * @param jobId Job ID.
      * @param ctx Hadoop job context.
      * @param log Logger.
      */
-    public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteLogger log) {
+    public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteLogger log, HadoopV2Job job) {
         this.jobId = jobId;
         this.ctx = ctx;
         this.log = log.getLogger(HadoopV2JobResourceManager.class);
+        this.job = job;
     }
 
     /**
@@ -115,7 +120,7 @@ public class HadoopV2JobResourceManager {
                 stagingDir = new Path(new URI(mrDir));
 
                 if (download) {
-                    FileSystem fs = HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), cfg, true);
+                    FileSystem fs = job.fileSystem(stagingDir.toUri(), cfg);
 
                     if (!fs.exists(stagingDir))
                         throw new IgniteCheckedException("Failed to find map-reduce submission " +
@@ -210,7 +215,7 @@ public class HadoopV2JobResourceManager {
 
             FileSystem dstFs = FileSystem.getLocal(cfg);
 
-            FileSystem srcFs = HadoopUtils.fileSystemForMrUser(srcPath.toUri(), cfg, true);
+            FileSystem srcFs = job.fileSystem(srcPath.toUri(), cfg);
 
             if (extract) {
                 File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
@@ -292,8 +297,11 @@ public class HadoopV2JobResourceManager {
      */
     public void cleanupStagingDirectory() {
         try {
-            if (stagingDir != null)
-                HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), ctx.getJobConf(), true).delete(stagingDir, true);
+            if (stagingDir != null) {
+                FileSystem fs = job.fileSystem(stagingDir.toUri(), ctx.getJobConf());
+
+                fs.delete(stagingDir, true);
+            }
         }
         catch (Exception e) {
             log.error("Failed to remove job staging directory [path=" + stagingDir + ", jobId=" + jobId + ']' , e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
index e89feba..6eff475 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
@@ -33,6 +33,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v1.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -44,6 +45,7 @@ import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
@@ -54,6 +56,22 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     /** */
     private static final boolean COMBINE_KEY_GROUPING_SUPPORTED;
 
+    /** Lazy per-user file system cache used by the Hadoop task. */
+    private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap
+        = createHadoopLazyConcurrentMap();
+
+    /**
+     * This method is called with reflection upon Job finish with class loader of each task.
+     * This will clean up all the Fs created for specific task.
+     * Each class loader sees uses its own instance of <code>fsMap<code/> since the class loaders
+     * are different.
+     *
+     * @throws IgniteCheckedException On error.
+     */
+    public static void close() throws IgniteCheckedException {
+        fsMap.close();
+    }
+
     /**
      * Check for combiner grouping support (available since Hadoop 2.3).
      */
@@ -91,7 +109,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private volatile HadoopTask task;
 
     /** Local node ID */
-    private UUID locNodeId;
+    private final UUID locNodeId;
 
     /** Counters for task. */
     private final HadoopCounters cntrs = new HadoopCountersImpl();
@@ -423,7 +441,22 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private Object readExternalSplit(HadoopExternalSplit split) throws IgniteCheckedException {
         Path jobDir = new Path(jobConf().get(MRJobConfig.MAPREDUCE_JOB_DIR));
 
-        try (FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf(), false);
+        FileSystem fs;
+
+        try {
+            // This assertion uses .startsWith() instead of .equals() because task class loaders may
+            // be reused between tasks of the same job.
+            assert ((HadoopClassLoader)getClass().getClassLoader()).name()
+                .startsWith(HadoopClassLoader.nameForTask(taskInfo(), true));
+
+            // We also cache Fs there, all them will be cleared explicitly upon the Job end.
+            fs = fileSystemForMrUserWithCaching(jobDir.toUri(), jobConf(), fsMap);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        try (
             FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
 
             in.seek(split.offset());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
index fb21e2d..e7c7f8a 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
@@ -21,7 +21,6 @@ import junit.framework.*;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.jetbrains.annotations.*;
@@ -38,7 +37,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suite() throws Exception {
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite FS Events Test Suite");
 
@@ -58,7 +57,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suiteNoarchOnly() throws Exception {
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite IGFS Events Test Suite Noarch Only");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
index 66c14b5..da6f9c7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@ -272,7 +272,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             @Override public boolean apply() {
                 return igfs.exists(statPath);
             }
-        }, 10000);
+        }, 20_000);
 
         final long apiEvtCnt0 = apiEvtCnt;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
index 48e83cc..f59be19 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.hadoop.mapred.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.io.*;
 import java.util.*;
@@ -44,9 +45,11 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
 
         HadoopDefaultJobInfo jobInfo = createJobInfo(jobConf);
 
-        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+        UUID uuid = new UUID(0, 0);
 
-        return jobInfo.createJob(jobId, log);
+        HadoopJobId jobId = new HadoopJobId(uuid, 0);
+
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
index e73fae3..1570807 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.*;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.util.*;
 
@@ -62,9 +63,11 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
 
         HadoopDefaultJobInfo jobInfo = createJobInfo(hadoopJob.getConfiguration());
 
-        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+        UUID uuid = new UUID(0, 0);
 
-        return jobInfo.createJob(jobId, log);
+        HadoopJobId jobId = new HadoopJobId(uuid, 0);
+
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
index f3b9307..b8f62e6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
@@ -68,9 +68,11 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
 
         HadoopDefaultJobInfo info = createJobInfo(cfg);
 
-        HadoopJobId id = new HadoopJobId(UUID.randomUUID(), 1);
+        final UUID uuid = UUID.randomUUID();
 
-        HadoopJob job = info.createJob(id, log);
+        HadoopJobId id = new HadoopJobId(uuid, 1);
+
+        HadoopJob job = info.createJob(HadoopV2Job.class, id, log);
 
         HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
             null));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
index 9395c5e..b5e2ab5 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
@@ -136,7 +136,8 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+                HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
             assert false;
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 4be5d72..2ab3e8c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -50,7 +50,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         downloadHadoop();
         downloadHive();
 
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        final ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
index 8982d83..22beea6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
@@ -20,7 +20,6 @@ package org.apache.ignite.testsuites;
 import junit.framework.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
 
 import static org.apache.ignite.testsuites.IgniteHadoopTestSuite.*;
 
@@ -36,7 +35,7 @@ public class IgniteIgfsLinuxAndMacOSTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         downloadHadoop();
 
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite IGFS Test Suite For Linux And Mac OS");
 


[26/33] incubator-ignite git commit: # ignite-sprint-7: IgniteReflectionFactory javadoc

Posted by sb...@apache.org.
# ignite-sprint-7: IgniteReflectionFactory javadoc


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

Branch: refs/heads/ignite-959
Commit: 359b43144046bc683f9e28794eb7e7bc9c63ebc1
Parents: dfe95bd
Author: ashutak <as...@gridgain.com>
Authored: Tue Jun 23 16:59:44 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jun 23 16:59:44 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteReflectionFactory.java  | 81 ++++++++++++++++++--
 1 file changed, 76 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/359b4314/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
index 3222938..af73737 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
@@ -26,26 +26,91 @@ import java.lang.reflect.*;
 import java.util.*;
 
 /**
- * Convenience class for reflection-based object creation.
+ * Factory implementation that use reflection to create instance of given class.
+ * <p>
+ * There are 2 modes of factory: singleton and non-sigletton.
+ * <p>
+ * Class that should be created by {@link IgniteReflectionFactory} (component class) have to be
+ * public java POJO with public setters for field
+ * for which property injection will be used (see {@link #setProperties(Map)}).
+ * <p>
+ * <h1 class="header">Configuration</h1>
+ * <h2 class="header">Mandatory</h2>
+ * The following configuration parameters are mandatory:
+ * <ul>
+ * <li>Component class - class to be created (see {@link #setComponentClass(Class)}.
+ * It have to be public java POJO class with default constructor
+ * and public setters to be used by properties injection (see {@link #setProperties(Map)})</li>
+ * </ul>
+ * <h2 class="header">Optional</h2>
+ * The following configuration parameters are optional:
+ * <ul>
+ * </li>
+ * <li>Singleton mode (see {@link #setSingleton(boolean)})</li>
+ * <li>Properties map (see {@link #setProperties(Map)}</li>
+ * <li>With method (see {@link #setWithMethod(Object, String, Serializable)}</li>
+ * </ul>
+ * <h2 class="header">Java Example</h2>
+ * <pre name="code" class="java">
+ * Factory<CacheStoreSessionListener> factory =
+ *     new IgniteReflectionFactory<CacheStoreSessionListener>(MyCacheStoreSessionListener.class);
+ *
+ * CacheConfiguration cc = new CacheConfiguration()
+ *     .setCacheStoreSessionListenerFactories(factory);
+ *
+ * IgniteConfiguration cfg = new IgniteConfiguration()
+ *     .setCacheConfiguration(cc);
+ *
+ * // Start grid.
+ * Ignition.start(cfg);
+ * </pre>
+ * <h2 class="header">Spring Example</h2>
+ * TcpDiscoverySpi can be configured from Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
+ *     ...
+ *     &lt;property name="cacheConfiguration"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="org.apache.ignite.configuration.CacheConfiguration"&gt;
+ *                 ...
+ *                 &lt;property name="cacheStoreSessionListenerFactories"&gt;
+ *                     &lt;list&gt;
+ *                         &lt;bean class="org.apache.ignite.configuration.IgniteReflectionFactory"&gt;
+ *                             &lt;property name="componentClass" value="custom.project.MyCacheStoreSessionListener"/&gt;
+ *                         &lt;/bean&gt;
+ *                     &lt;/list&gt;
+ *                 &lt;/property&gt;
+ *                 ...
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ *     ...
+ * &lt;/bean&gt;
+ * </pre>
+ * <p>
+ * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
+ * <br>
+ * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
+ * @see Factory
  */
 public class IgniteReflectionFactory<T> implements Factory<T> {
     /** */
     private static final long serialVersionUID = 0L;
 
-    /** */
+    /** Singletom mode */
     private volatile boolean singleton;
 
-    /** */
+    /** Component class */
     private volatile Class<? extends T> cls;
 
-    /** */
+    /** Properties */
     private volatile Map<String, Serializable> props;
 
     /** */
     private transient T instance;
 
     /**
-     *
+     * Default constructor.
      */
     public IgniteReflectionFactory() {
         // No-op.
@@ -98,6 +163,9 @@ public class IgniteReflectionFactory<T> implements Factory<T> {
     }
 
     /**
+     * Gets a map of properties. Map contains entries of component class field name
+     * to value of the filed which will be used as initial value.
+     *
      * @return Properties.
      */
     public Map<String, Serializable> getProperties() {
@@ -105,6 +173,9 @@ public class IgniteReflectionFactory<T> implements Factory<T> {
     }
 
     /**
+     * Sets a map of properties. Map contains entries of component class field name
+     * to a value of the filed which will be used as initial value.
+     *
      * @param props Properties.
      */
     public void setProperties(Map<String, Serializable> props) {


[24/33] incubator-ignite git commit: [IGNITE-980]: added missing file

Posted by sb...@apache.org.
[IGNITE-980]: added missing file


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

Branch: refs/heads/ignite-959
Commit: 2bb6e0f887c99de70e479c48992f5a0a86cbf5ec
Parents: 3b49184
Author: iveselovskiy <iv...@gridgain.com>
Authored: Tue Jun 23 15:02:23 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Tue Jun 23 15:02:23 2015 +0300

----------------------------------------------------------------------
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 +++++++++++++++++++
 1 file changed, 241 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2bb6e0f8/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
new file mode 100644
index 0000000..f94b932
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemCacheUtils.java
@@ -0,0 +1,241 @@
+/*
+ * 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.hadoop.fs;
+
+import org.apache.hadoop.conf.*;
+import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.*;
+import org.apache.ignite.*;
+import org.apache.ignite.hadoop.fs.v1.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+
+/**
+ * File system cache utility methods used by Map-Reduce tasks and jobs.
+ */
+public class HadoopFileSystemCacheUtils {
+    /**
+     * A common static factory method. Creates new HadoopLazyConcurrentMap.
+     * @return a new HadoopLazyConcurrentMap.
+     */
+    public static HadoopLazyConcurrentMap<FsCacheKey, FileSystem> createHadoopLazyConcurrentMap() {
+        return new HadoopLazyConcurrentMap<>(
+            new HadoopLazyConcurrentMap.ValueFactory<FsCacheKey, FileSystem>() {
+                @Override public FileSystem createValue(FsCacheKey key) {
+                    try {
+                        assert key != null;
+
+                        // Explicitly disable FileSystem caching:
+                        URI uri = key.uri();
+
+                        String scheme = uri.getScheme();
+
+                        // Copy the configuration to avoid altering the external object.
+                        Configuration cfg = new Configuration(key.configuration());
+
+                        String prop = HadoopFileSystemsUtils.disableFsCachePropertyName(scheme);
+
+                        cfg.setBoolean(prop, true);
+
+                        return FileSystem.get(uri, cfg, key.user());
+                    }
+                    catch (IOException | InterruptedException ioe) {
+                        throw new IgniteException(ioe);
+                    }
+                }
+            }
+        );
+    }
+
+    /**
+     * Gets non-null user name as per the Hadoop viewpoint.
+     * @param cfg the Hadoop job configuration, may be null.
+     * @return the user name, never null.
+     */
+    private static String getMrHadoopUser(Configuration cfg) throws IOException {
+        String user = cfg.get(MRJobConfig.USER_NAME);
+
+        if (user == null)
+            user = IgniteHadoopFileSystem.getFsHadoopUser();
+
+        return user;
+    }
+
+    /**
+     * Common method to get the V1 file system in MapRed engine.
+     * It gets the filesystem for the user specified in the
+     * configuration with {@link MRJobConfig#USER_NAME} property.
+     * The file systems are created and cached in the given map upon first request.
+     *
+     * @param uri The file system uri.
+     * @param cfg The configuration.
+     * @param map The caching map.
+     * @return The file system.
+     * @throws IOException On error.
+     */
+    public static FileSystem fileSystemForMrUserWithCaching(@Nullable URI uri, Configuration cfg,
+        HadoopLazyConcurrentMap<FsCacheKey, FileSystem> map)
+            throws IOException {
+        assert map != null;
+        assert cfg != null;
+
+        final String usr = getMrHadoopUser(cfg);
+
+        assert usr != null;
+
+        if (uri == null)
+            uri = FileSystem.getDefaultUri(cfg);
+
+        final FileSystem fs;
+
+        try {
+            final FsCacheKey key = new FsCacheKey(uri, usr, cfg);
+
+            fs = map.getOrCreate(key);
+        }
+        catch (IgniteException ie) {
+            throw new IOException(ie);
+        }
+
+        assert fs != null;
+        assert !(fs instanceof IgniteHadoopFileSystem) || F.eq(usr, ((IgniteHadoopFileSystem)fs).user());
+
+        return fs;
+    }
+
+    /**
+     * Takes Fs URI using logic similar to that used in FileSystem#get(1,2,3).
+     * @param uri0 The uri.
+     * @param cfg The cfg.
+     * @return Correct URI.
+     */
+    private static URI fixUri(URI uri0, Configuration cfg) {
+        if (uri0 == null)
+            return FileSystem.getDefaultUri(cfg);
+
+        String scheme = uri0.getScheme();
+        String authority = uri0.getAuthority();
+
+        if (authority == null) {
+            URI dfltUri = FileSystem.getDefaultUri(cfg);
+
+            if (scheme == null || (scheme.equals(dfltUri.getScheme()) && dfltUri.getAuthority() != null))
+                return dfltUri;
+        }
+
+        return uri0;
+    }
+
+    /**
+     * Note that configuration is not a part of the key.
+     * It is used solely to initialize the first instance
+     * that is created for the key.
+     */
+    public static final class FsCacheKey {
+        /** */
+        private final URI uri;
+
+        /** */
+        private final String usr;
+
+        /** */
+        private final String equalityKey;
+
+        /** */
+        private final Configuration cfg;
+
+        /**
+         * Constructor
+         */
+        public FsCacheKey(URI uri, String usr, Configuration cfg) {
+            assert uri != null;
+            assert usr != null;
+            assert cfg != null;
+
+            this.uri = fixUri(uri, cfg);
+            this.usr = usr;
+            this.cfg = cfg;
+
+            this.equalityKey = createEqualityKey();
+        }
+
+        /**
+         * Creates String key used for equality and hashing.
+         */
+        private String createEqualityKey() {
+            GridStringBuilder sb = new GridStringBuilder("(").a(usr).a(")@");
+
+            if (uri.getScheme() != null)
+                sb.a(uri.getScheme().toLowerCase());
+
+            sb.a("://");
+
+            if (uri.getAuthority() != null)
+                sb.a(uri.getAuthority().toLowerCase());
+
+            return sb.toString();
+        }
+
+        /**
+         * The URI.
+         */
+        public URI uri() {
+            return uri;
+        }
+
+        /**
+         * The User.
+         */
+        public String user() {
+            return usr;
+        }
+
+        /**
+         * The Configuration.
+         */
+        public Configuration configuration() {
+            return cfg;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("SimplifiableIfStatement")
+        @Override public boolean equals(Object obj) {
+            if (obj == this)
+                return true;
+
+            if (obj == null || getClass() != obj.getClass())
+                return false;
+
+            return equalityKey.equals(((FsCacheKey)obj).equalityKey);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return equalityKey.hashCode();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return equalityKey;
+        }
+    }
+}


[09/33] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-7' into ignite-sprint-7

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

Branch: refs/heads/ignite-959
Commit: 188cd82327a41833c7884ef8b437652999a0e113
Parents: 27fbf02 6974a8e
Author: ashutak <as...@gridgain.com>
Authored: Fri Jun 19 12:54:39 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Fri Jun 19 12:54:39 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |  9 +++++
 .../processors/cache/IgniteCacheProxy.java      |  5 +++
 .../processors/query/GridQueryProcessor.java    |  7 ++++
 .../processors/task/GridTaskProcessor.java      |  3 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 42 ++++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 35 ++++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 42 ++++++++++++++------
 .../tcp/internal/TcpDiscoveryNode.java          | 18 +++++++++
 .../apache/ignite/internal/GridSelfTest.java    | 20 ++++++----
 .../DataStreamerMultiThreadedSelfTest.java      |  3 ++
 10 files changed, 134 insertions(+), 50 deletions(-)
----------------------------------------------------------------------