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/10 16:11:22 UTC

[01/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-545 cd9638373 -> 1652fd181


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..c30e216
--- /dev/null
+++ b/modules/hibernate/src/test/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListenerSelfTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+import org.hibernate.cfg.Configuration;
+
+import javax.cache.Cache;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import javax.persistence.*;
+import java.io.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheHibernateStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheHibernateStoreSessionListener lsnr = new CacheHibernateStoreSessionListener();
+
+                SessionFactory sesFactory = new Configuration().
+                    setProperty("hibernate.connection.url", URL).
+                    addAnnotatedClass(Table1.class).
+                    addAnnotatedClass(Table2.class).
+                    buildSessionFactory();
+
+                lsnr.setSessionFactory(sesFactory);
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkSession();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkSession();
+
+            if (write.get()) {
+                Session hibSes = ses.attachment();
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        hibSes.save(new Table1(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        hibSes.save(new Table2(entry.getKey(), entry.getValue()));
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkSession();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkSession() {
+            Session hibSes = ses.attachment();
+
+            assertNotNull(hibSes);
+
+            assertTrue(hibSes.isOpen());
+
+            Transaction tx = hibSes.getTransaction();
+
+            assertNotNull(tx);
+
+            if (ses.isWithinTransaction())
+                assertTrue(tx.isActive());
+            else
+                assertFalse(tx.isActive());
+
+            verifySameInstance(hibSes);
+        }
+
+        /**
+         * @param hibSes Session.
+         */
+        private void verifySameInstance(Session hibSes) {
+            Map<String, Session> props = ses.properties();
+
+            Session sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, hibSes);
+            else {
+                assertSame(hibSes, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table1")
+    private static class Table1 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table1(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
+
+    /**
+     */
+    @Entity
+    @Table(name = "Table2")
+    private static class Table2 implements Serializable {
+        /** */
+        @Id @GeneratedValue
+        @Column(name = "id")
+        private Integer id;
+
+        /** */
+        @Column(name = "key")
+        private int key;
+
+        /** */
+        @Column(name = "value")
+        private int value;
+
+        /**
+         * @param key Key.
+         * @param value Value.
+         */
+        private Table2(int key, int value) {
+            this.key = key;
+            this.value = value;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
index da741f8..655e801 100644
--- a/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
+++ b/modules/hibernate/src/test/java/org/apache/ignite/testsuites/IgniteHibernateTestSuite.java
@@ -41,6 +41,8 @@ public class IgniteHibernateTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheHibernateBlobStoreNodeRestartTest.class);
 
+        suite.addTestSuite(CacheHibernateStoreSessionListenerSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
new file mode 100644
index 0000000..8afd746
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexEntryEvictTest.java
@@ -0,0 +1,200 @@
+/*
+ * 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.query.*;
+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.spi.swapspace.file.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class GridCacheOffheapIndexEntryEvictTest extends GridCommonAbstractTest {
+    /** */
+    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setNetworkTimeout(2000);
+
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setBackups(1);
+        cacheCfg.setOffHeapMaxMemory(0);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setMemoryMode(OFFHEAP_TIERED);
+        cacheCfg.setEvictionPolicy(null);
+        cacheCfg.setSqlOnheapRowCacheSize(10);
+        cacheCfg.setIndexedTypes(Integer.class, TestValue.class);
+        cacheCfg.setNearConfiguration(null);
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(1);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryWhenLocked() throws Exception {
+        IgniteCache<Integer, TestValue> cache = grid(0).cache(null);
+
+        List<Lock> locks = new ArrayList<>();
+
+        final int ENTRIES = 1000;
+
+        try {
+            for (int i = 0; i < ENTRIES; i++) {
+                cache.put(i, new TestValue(i));
+
+                Lock lock = cache.lock(i);
+
+                lock.lock(); // Lock entry so that it should not be evicted.
+
+                locks.add(lock);
+
+                for (int j = 0; j < 3; j++)
+                    assertNotNull(cache.get(i));
+            }
+
+            checkQuery(cache, "_key >= 100", ENTRIES - 100);
+        }
+        finally {
+            for (Lock lock : locks)
+                lock.unlock();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdates() throws Exception {
+        final int ENTRIES = 500;
+
+        IgniteCache<Integer, TestValue> cache = grid(0).cache(null);
+
+        for (int i = 0; i < ENTRIES; i++) {
+            for (int j = 0; j < 3; j++) {
+                cache.getAndPut(i, new TestValue(i));
+
+                assertNotNull(cache.get(i));
+
+                assertNotNull(cache.localPeek(i));
+            }
+
+            checkQuery(cache, "_key >= 0", i + 1);
+        }
+
+        for (int i = 0; i < ENTRIES; i++) {
+            if (i % 2 == 0)
+                cache.getAndRemove(i);
+            else
+                cache.remove(i);
+
+            checkQuery(cache, "_key >= 0", ENTRIES - (i + 1));
+        }
+    }
+
+    /**
+     * @param cache Cache.
+     * @param sql Query.
+     * @param expCnt Number of expected entries.
+     */
+    private void checkQuery(IgniteCache<Integer, TestValue> cache, String sql, int expCnt) {
+        SqlQuery<Integer, TestValue> qry = new SqlQuery<>(TestValue.class, sql);
+
+        List<Cache.Entry<Integer, TestValue>> res = cache.query(qry).getAll();
+
+        assertEquals(expCnt, res.size());
+
+        for (Cache.Entry<Integer, TestValue> e : res) {
+            assertNotNull(e.getKey());
+
+            assertEquals((int)e.getKey(), e.getValue().val);
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestValue implements Externalizable {
+        /** */
+        private int val;
+
+        /**
+         *
+         */
+        public TestValue() {
+            // No-op.
+        }
+
+        /**
+         * @param val Value.
+         */
+        public TestValue(int val) {
+            this.val = val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void writeExternal(ObjectOutput out) throws IOException {
+            out.writeInt(val);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+            val = in.readInt();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
index 4e40040..4e613ae 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffheapIndexGetSelfTest.java
@@ -18,12 +18,19 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cache.query.annotations.*;
 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.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
 import static org.apache.ignite.cache.CacheMemoryMode.*;
@@ -67,8 +74,7 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
         cacheCfg.setAtomicityMode(TRANSACTIONAL);
         cacheCfg.setMemoryMode(OFFHEAP_TIERED);
         cacheCfg.setEvictionPolicy(null);
-        cacheCfg.setOffHeapMaxMemory(OFFHEAP_MEM);
-        cacheCfg.setIndexedTypes(Long.class, Long.class);
+        cacheCfg.setIndexedTypes(Long.class, Long.class, String.class, TestEntity.class);
 
         cfg.setCacheConfiguration(cacheCfg);
 
@@ -98,8 +104,6 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testGet() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-873");
-
         IgniteCache<Long, Long> cache = grid(0).cache(null);
 
         for (long i = 0; i < 100; i++)
@@ -107,5 +111,73 @@ public class GridCacheOffheapIndexGetSelfTest extends GridCommonAbstractTest {
 
         for (long i = 0; i < 100; i++)
             assertEquals((Long)i, cache.get(i));
+
+        SqlQuery<Long, Long> qry = new SqlQuery<>(Long.class, "_val >= 90");
+
+        List<Cache.Entry<Long, Long>> res = cache.query(qry).getAll();
+
+        assertEquals(10, res.size());
+
+        for (Cache.Entry<Long, Long> e : res) {
+            assertNotNull(e.getKey());
+            assertNotNull(e.getValue());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPutGet() throws Exception {
+        IgniteCache<Object, Object> cache = grid(0).cache(null);
+
+        Map map = new HashMap();
+
+        try (Transaction tx = grid(0).transactions().txStart(TransactionConcurrency.PESSIMISTIC,
+            TransactionIsolation.REPEATABLE_READ, 100000, 1000)) {
+
+            for (int i = 4; i < 400; i++) {
+                map.put("key" + i, new TestEntity("value"));
+                map.put(i, "value");
+            }
+
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        for (int i = 0; i < 100; i++) {
+            cache.get("key" + i);
+            cache.get(i);
+        }
+    }
+
+    /**
+     * Test entry class.
+     */
+    private static class TestEntity implements Serializable {
+        /** Value. */
+        @QuerySqlField(index = true)
+        private String val;
+
+        /**
+         * @param value Value.
+         */
+        public TestEntity(String value) {
+            this.val = value;
+        }
+
+        /**
+         * @return Value.
+         */
+        public String getValue() {
+            return val;
+        }
+
+        /**
+         * @param val Value
+         */
+        public void setValue(String val) {
+            this.val = val;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
index 81c2820..24011b4 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMetricsSelfTest.java
@@ -62,26 +62,35 @@ public class GridCacheQueryMetricsSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(disco);
 
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+        CacheConfiguration<String, Integer> cacheCfg1 = defaultCacheConfiguration();
 
-        cacheCfg.setCacheMode(CACHE_MODE);
-        cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
-        cacheCfg.setIndexedTypes(String.class, Integer.class);
+        cacheCfg1.setName("A");
+        cacheCfg1.setCacheMode(CACHE_MODE);
+        cacheCfg1.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg1.setIndexedTypes(String.class, Integer.class);
 
-        cfg.setCacheConfiguration(cacheCfg);
+        CacheConfiguration<String, Integer> cacheCfg2 = defaultCacheConfiguration();
+
+        cacheCfg2.setName("B");
+        cacheCfg2.setCacheMode(CACHE_MODE);
+        cacheCfg2.setWriteSynchronizationMode(FULL_SYNC);
+        cacheCfg2.setIndexedTypes(String.class, Integer.class);
+
+        cfg.setCacheConfiguration(cacheCfg1, cacheCfg2);
 
         return cfg;
     }
 
     /**
-     * JUnit.
+     * Test metrics for SQL queries.
      *
      * @throws Exception In case of error.
      */
-    public void testAccumulativeMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).cache(null);
+    public void testSqlFieldsQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
 
-        SqlQuery<String, Integer> qry = new SqlQuery(Integer.class, "_val >= 0");
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from Integer");
 
         cache.query(qry).getAll();
 
@@ -114,20 +123,22 @@ public class GridCacheQueryMetricsSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * JUnit.
+     * Test metrics for Scan queries.
      *
      * @throws Exception In case of error.
      */
-    public void testSingleQueryMetrics() throws Exception {
-        IgniteCache<String, Integer> cache = grid(0).cache(null);
+    public void testScanQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
 
-        SqlQuery<String, Integer> qry = new SqlQuery(Integer.class, "_val >= 0");
+        // Execute query.
+        ScanQuery<String, Integer> qry = new ScanQuery<>();
 
-        // Execute.
         cache.query(qry).getAll();
 
         QueryMetrics m = cache.queryMetrics();
 
+        assert m != null;
+
         info("Metrics: " + m);
 
         assertEquals(1, m.executions());
@@ -136,11 +147,54 @@ public class GridCacheQueryMetricsSelfTest extends GridCommonAbstractTest {
         assertTrue(m.maximumTime() >= 0);
         assertTrue(m.minimumTime() >= 0);
 
-        // Execute.
+        // Execute again with the same parameters.
         cache.query(qry).getAll();
 
         m = cache.queryMetrics();
 
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(2, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+    }
+
+    /**
+     * Test metrics for SQL cross cache queries.
+     *
+     * @throws Exception In case of error.
+     */
+    public void testSqlCrossCacheQueryMetrics() throws Exception {
+        IgniteCache<String, Integer> cache = grid(0).context().cache().jcache("A");
+
+        // Execute query.
+        SqlFieldsQuery qry = new SqlFieldsQuery("select * from \"B\".Integer");
+
+        cache.query(qry).getAll();
+
+        QueryMetrics m = cache.queryMetrics();
+
+        assert m != null;
+
+        info("Metrics: " + m);
+
+        assertEquals(1, m.executions());
+        assertEquals(0, m.fails());
+        assertTrue(m.averageTime() >= 0);
+        assertTrue(m.maximumTime() >= 0);
+        assertTrue(m.minimumTime() >= 0);
+
+        // Execute again with the same parameters.
+        cache.query(qry).getAll();
+
+        m = cache.queryMetrics();
+
+        assert m != null;
+
         info("Metrics: " + m);
 
         assertEquals(2, m.executions());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
index 2c0962b..5b623da 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridIndexingWithNoopSwapSelfTest.java
@@ -64,7 +64,11 @@ public class GridIndexingWithNoopSwapSelfTest extends GridCommonAbstractTest {
         cc.setRebalanceMode(SYNC);
         cc.setSwapEnabled(true);
         cc.setNearConfiguration(new NearCacheConfiguration());
-        cc.setEvictionPolicy(new FifoEvictionPolicy(1000));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setBackups(1);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setIndexedTypes(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index 1a60bbd..6224cb9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -99,11 +99,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
+        c.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder));
 
         // Otherwise noop swap space will be chosen on Windows.
         c.setSwapSpaceSpi(new FileSwapSpaceSpi());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationPrimitiveTypesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationPrimitiveTypesSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationPrimitiveTypesSelfTest.java
new file mode 100644
index 0000000..e90f10c
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationPrimitiveTypesSelfTest.java
@@ -0,0 +1,104 @@
+/*
+ * 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.query.*;
+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.testframework.junits.common.*;
+
+/**
+ *
+ */
+@SuppressWarnings("unchecked")
+public class IgniteCacheConfigurationPrimitiveTypesSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPrimitiveTypes() throws Exception {
+        Ignite ignite = startGrid(1);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>("c1");
+
+        ccfg.setIndexedTypes(
+            byte.class, byte.class,
+            short.class, short.class,
+            int.class, int.class,
+            long.class, long.class,
+            float.class, float.class,
+            double.class, double.class,
+            boolean.class, boolean.class);
+
+        IgniteCache<Object, Object> cache = ignite.getOrCreateCache(ccfg);
+
+        byte b = 1;
+        cache.put(b, b);
+
+        short s = 2;
+        cache.put(s, s);
+
+        int i = 3;
+        cache.put(i, i);
+
+        long l = 4;
+        cache.put(l, l);
+
+        float f = 5;
+        cache.put(f, f);
+
+        double d = 6;
+        cache.put(d, d);
+
+        boolean bool = true;
+        cache.put(bool, bool);
+
+        assert cache.query(new ScanQuery<>()).getAll().size() == 7;
+
+        assertEquals(cache.query(new SqlQuery<>(Byte.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Short.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Integer.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Long.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Float.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Double.class, "1 = 1")).getAll().size(), 1);
+        assertEquals(cache.query(new SqlQuery<>(Boolean.class, "1 = 1")).getAll().size(), 1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
index b2095a8..b171ead 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingQueryErrorTest.java
@@ -30,7 +30,8 @@ public class IgniteCacheP2pUnmarshallingQueryErrorTest extends IgniteCacheP2pUnm
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.getCacheConfiguration()[0].setIndexedTypes(TestKey.class, String.class);
+        if (cfg.getCacheConfiguration().length > 0)
+            cfg.getCacheConfiguration()[0].setIndexedTypes(TestKey.class, String.class);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
new file mode 100644
index 0000000..df4c01d
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+
+/**
+ * Test queries in off-heap tiered mode.
+ */
+public class IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest extends IgniteCacheQueryMultiThreadedSelfTest {
+    /** {@inheritDoc} */
+    @Override protected CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = super.cacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.REPLICATED);
+        ccfg.setMemoryMode(CacheMemoryMode.OFFHEAP_TIERED);
+        ccfg.setOffHeapMaxMemory(0);
+
+        return ccfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
index 23a97c9..1d6bbc8 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheQueryMultiThreadedSelfTest.java
@@ -104,7 +104,16 @@ public class IgniteCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTes
         cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setBackups(1);
-        cacheCfg.setEvictionPolicy(evictsEnabled() ? new LruEvictionPolicy(100) : null);
+
+        LruEvictionPolicy plc = null;
+
+        if (evictsEnabled()) {
+            plc = new LruEvictionPolicy();
+            plc.setMaxSize(100);
+        }
+
+        cacheCfg.setEvictionPolicy(plc);
+
         cacheCfg.setSqlOnheapRowCacheSize(128);
         cacheCfg.setIndexedTypes(
             Integer.class, Integer.class,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
index 3833576..4553fec 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ttl/CacheTtlAbstractSelfTest.java
@@ -67,7 +67,11 @@ public abstract class CacheTtlAbstractSelfTest extends GridCommonAbstractTest {
         ccfg.setAtomicityMode(atomicityMode());
         ccfg.setMemoryMode(memoryMode());
         ccfg.setOffHeapMaxMemory(0);
-        ccfg.setEvictionPolicy(new LruEvictionPolicy(MAX_CACHE_SIZE));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(MAX_CACHE_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
         ccfg.setIndexedTypes(Integer.class, Integer.class);
         ccfg.setBackups(2);
         ccfg.setWriteSynchronizationMode(FULL_SYNC);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 f42963a..2d5fed5 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
@@ -59,10 +59,10 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheLargeResultSelfTest.class);
         suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTiredSelfTest.class);
+        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(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
 //        suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class); TODO IGNITE-484

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index ae45120..67ebda9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -47,9 +47,12 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
         suite.addTestSuite(CacheTtlOnheapAtomicPartitionedSelfTest.class);
 
         suite.addTestSuite(GridCacheOffheapIndexGetSelfTest.class);
+        suite.addTestSuite(GridCacheOffheapIndexEntryEvictTest.class);
 
         suite.addTestSuite(CacheConfigurationP2PTest.class);
 
+        suite.addTestSuite(IgniteCacheConfigurationPrimitiveTypesSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index eca4fa9..c2bacff 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -23,7 +23,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
     <modelVersion>4.0.0</modelVersion>
 
-    <groupId>org.apache.ignite</groupId>
+    <parent>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-parent</artifactId>
+        <version>1</version>
+        <relativePath>../../parent</relativePath>
+    </parent>
+
     <artifactId>ignite-mesos</artifactId>
     <version>1.1.1-SNAPSHOT</version>
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index e922215..a6e61ac 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -77,6 +77,12 @@
         </dependency>
 
         <dependency>
+            <groupId>org.springframework</groupId>
+            <artifactId>spring-jdbc</artifactId>
+            <version>${spring.version}</version>
+        </dependency>
+
+        <dependency>
             <groupId>commons-logging</groupId>
             <artifactId>commons-logging</artifactId>
             <version>1.1.1</version>
@@ -103,6 +109,14 @@
             <type>test-jar</type>
             <scope>test</scope>
         </dependency>
+
+
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+            <version>1.3.175</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
new file mode 100644
index 0000000..0a32816
--- /dev/null
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListener.java
@@ -0,0 +1,207 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.transactions.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+import org.springframework.transaction.*;
+import org.springframework.transaction.support.*;
+
+import javax.cache.integration.*;
+import javax.sql.*;
+
+/**
+ * Cache store session listener based on Spring transaction management.
+ * <p>
+ * This listener starts a new DB transaction for each session and commits
+ * or rolls it back when session ends. If there is no ongoing
+ * cache transaction, this listener is no-op.
+ * <p>
+ * Store implementation can use any Spring APIs like {@link JdbcTemplate}
+ * and others. The listener will guarantee that if there is an
+ * ongoing cache transaction, all store operations within this
+ * transaction will be automatically enlisted in the same database
+ * transaction.
+ * <p>
+ * {@link CacheSpringStoreSessionListener} requires that either
+ * {@link #setTransactionManager(PlatformTransactionManager) transaction manager}
+ * or {@link #setDataSource(DataSource) data source} is configured. If non of them is
+ * provided, exception is thrown. Is both are provided, data source will be
+ * ignored.
+ * <p>
+ * If there is a transaction, a {@link TransactionStatus} object will be saved
+ * as a store session {@link CacheStoreSession#attachment() attachment}. It
+ * can be used to acquire current DB transaction status.
+ */
+public class CacheSpringStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Transaction manager. */
+    private PlatformTransactionManager txMgr;
+
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /**
+     * Sets transaction manager.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param txMgr Transaction manager.
+     */
+    public void setTransactionManager(PlatformTransactionManager txMgr) {
+        this.txMgr = txMgr;
+    }
+
+    /**
+     * Gets transaction manager.
+     *
+     * @return Transaction manager.
+     */
+    public PlatformTransactionManager getTransactionManager() {
+        return txMgr;
+    }
+
+    /**
+     * Sets data source.
+     * <p>
+     * Either transaction manager or data source is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (txMgr == null && dataSrc == null)
+            throw new IgniteException("Either transaction manager or data source is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (dataSrc != null) {
+            if (txMgr == null)
+                txMgr = new DataSourceTransactionManager(dataSrc);
+            else
+                U.warn(log, "Data source configured in " + getClass().getSimpleName() +
+                    " will be ignored (transaction manager is already set).");
+        }
+
+        assert txMgr != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.isWithinTransaction() && ses.attachment() == null) {
+            try {
+                TransactionDefinition def = definition(ses.transaction(), ses.cacheName());
+
+                ses.attach(txMgr.getTransaction(def));
+            }
+            catch (TransactionException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        if (ses.isWithinTransaction()) {
+            TransactionStatus tx = ses.attach(null);
+
+            if (tx != null) {
+                try {
+                    if (commit)
+                        txMgr.commit(tx);
+                    else
+                        txMgr.rollback(tx);
+                }
+                catch (TransactionException e) {
+                    throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+                }
+            }
+        }
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @return DB transaction isolation.
+     */
+    private TransactionDefinition definition(Transaction tx, String cacheName) {
+        assert tx != null;
+
+        DefaultTransactionDefinition def = new DefaultTransactionDefinition();
+
+        def.setName("Ignite Tx [cache=" + (cacheName != null ? cacheName : "<default>") + ", id=" + tx.xid() + ']');
+        def.setIsolationLevel(isolationLevel(tx.isolation()));
+
+        long timeoutSec = (tx.timeout() + 500) / 1000;
+
+        if (timeoutSec > 0 && timeoutSec < Integer.MAX_VALUE)
+            def.setTimeout((int)timeoutSec);
+
+        return def;
+    }
+
+    /**
+     * Gets DB transaction isolation level based on ongoing cache transaction isolation.
+     *
+     * @param isolation Cache transaction isolation.
+     * @return DB transaction isolation.
+     */
+    private int isolationLevel(TransactionIsolation isolation) {
+        switch (isolation) {
+            case READ_COMMITTED:
+                return TransactionDefinition.ISOLATION_READ_COMMITTED;
+
+            case REPEATABLE_READ:
+                return TransactionDefinition.ISOLATION_REPEATABLE_READ;
+
+            case SERIALIZABLE:
+                return TransactionDefinition.ISOLATION_SERIALIZABLE;
+
+            default:
+                throw new IllegalStateException(); // Will never happen.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..74f5c69
--- /dev/null
+++ b/modules/spring/src/test/java/org/apache/ignite/cache/store/spring/CacheSpringStoreSessionListenerSelfTest.java
@@ -0,0 +1,197 @@
+/*
+ * 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.cache.store.spring;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+import org.springframework.transaction.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheSpringStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** */
+    private static final DataSource DATA_SRC = new DriverManagerDataSource(URL);
+
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store(new JdbcTemplate(DATA_SRC));
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheSpringStoreSessionListener lsnr = new CacheSpringStoreSessionListener();
+
+                lsnr.setDataSource(DATA_SRC);
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        private final JdbcTemplate jdbc;
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /**
+         * @param jdbc JDBC template.
+         */
+        private Store(JdbcTemplate jdbc) {
+            this.jdbc = jdbc;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+
+            if (write.get()) {
+                String table;
+
+                switch (ses.cacheName()) {
+                    case "cache1":
+                        table = "Table1";
+
+                        break;
+
+                    case "cache2":
+                        if (fail.get())
+                            throw new CacheWriterException("Expected failure.");
+
+                        table = "Table2";
+
+                        break;
+
+                    default:
+                        throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                }
+
+                jdbc.update("INSERT INTO " + table + " (key, value) VALUES (?, ?)",
+                    entry.getKey(), entry.getValue());
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkTransaction();
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkTransaction() {
+            TransactionStatus tx = ses.attachment();
+
+            if (ses.isWithinTransaction()) {
+                assertNotNull(tx);
+                assertFalse(tx.isCompleted());
+            }
+            else
+                assertNull(tx);
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = DataSourceUtils.getConnection(jdbc.getDataSource());
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertEquals(!ses.isWithinTransaction(), conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
index 8251c18..12dd494 100644
--- a/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
+++ b/modules/spring/src/test/java/org/apache/ignite/testsuites/IgniteSpringTestSuite.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.*;
+import org.apache.ignite.cache.store.spring.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.p2p.*;
 import org.apache.ignite.spring.*;
@@ -47,6 +48,8 @@ public class IgniteSpringTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteStartFromStreamConfigurationTest.class));
 
+        suite.addTestSuite(CacheSpringStoreSessionListenerSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
index 4b66720..3aa2a19 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheScanCommand.scala
@@ -139,7 +139,7 @@ class VisorCacheScanCommand {
         val firstPage =
             try
                 executeRandom(groupForDataNode(node, cacheName),
-                    classOf[VisorQueryTask], new VisorQueryArg(cacheName, "SCAN", false, pageSize)) match {
+                    classOf[VisorQueryTask], new VisorQueryArg(cacheName, null, false, pageSize)) match {
                     case x if x.get1() != null =>
                         error(x.get1())
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/scripts/git-apply-patch.sh
----------------------------------------------------------------------
diff --git a/scripts/git-apply-patch.sh b/scripts/git-apply-patch.sh
index c5f686f..757cd26 100755
--- a/scripts/git-apply-patch.sh
+++ b/scripts/git-apply-patch.sh
@@ -56,18 +56,18 @@ do
         IGNITE_HOME="$2"
         shift
         ;;
-        
+
         -idb|--ignitedefbranch)
         IGNITE_DEFAULT_BRANCH="$2"
         shift
         ;;
-        
+
         -ph|--patchhome)
         PATCHES_HOME="$2"
         shift
         ;;
         *)
-        
+
         echo "Unknown parameter: ${key}"
         ;;
     esac
@@ -75,7 +75,7 @@ do
 done
 
 echo "IGNITE_HOME    : ${IGNITE_HOME}"
-echo "Master branch  : ${IGNITE_DEFAULT_BRANCH}"
+echo "Default branch : ${IGNITE_DEFAULT_BRANCH}"
 echo "Ignite task    : ${IGNITE_TASK}"
 echo
 echo "PATCHES_HOME   : ${PATCHES_HOME}"

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/scripts/git-format-patch.sh
----------------------------------------------------------------------
diff --git a/scripts/git-format-patch.sh b/scripts/git-format-patch.sh
index 970347f..b11c73d 100755
--- a/scripts/git-format-patch.sh
+++ b/scripts/git-format-patch.sh
@@ -20,9 +20,13 @@
 # Git patch-file maker.
 #
 echo 'Usage: scripts/git-format-patch.sh [-ih|--ignitehome <path>] [-idb|--ignitedefbranch <branch-name>] [-ph|--patchhome <path>]'
+echo 'It is a script to create patch between Current branch (branch with changes) and Default branche. The script is safe and do not broke or lose your changes.'
 echo "It should be called from IGNITE_HOME directory."
-echo "Patch will be created at PATCHES_HOME between Master branch (IGNITE_DEFAULT_BRANCH) and Current branch."
+echo "Patch will be created at PATCHES_HOME (= IGNITE_HOME, by default) between Default branch (IGNITE_DEFAULT_BRANCH) and Current branch."
 echo "Note: you can use ${IGNITE_HOME}/scripts/git-patch-prop-local.sh to set your own local properties (to rewrite settings at git-patch-prop-local.sh). "
+echo 'Examples:'
+echo '- Basic (with all defaults and properties from git-patch-prop.sh):  ./scripts/git-format-patch.sh'
+echo '- Rewrite some defaults (see Usage):                                ./scripts/git-format-patch.sh -ph /home/user_name/patches'
 echo
 
 #
@@ -51,17 +55,17 @@ do
         IGNITE_HOME="$2"
         shift
         ;;
-        
+
         -idb|--ignitedefbranch)
         IGNITE_DEFAULT_BRANCH="$2"
         shift
         ;;
-        
+
         -ph|--patchhome)
         PATCHES_HOME="$2"
         shift
         ;;
-        
+
         *)
         echo "Unknown parameter: ${key}"
         ;;
@@ -72,7 +76,7 @@ done
 IGNITE_CURRENT_BRANCH=$( determineCurrentBranch ${IGNITE_HOME} )
 
 echo "IGNITE_HOME    : ${IGNITE_HOME}"
-echo "Master branch  : ${IGNITE_DEFAULT_BRANCH}"
+echo "Default branch : ${IGNITE_DEFAULT_BRANCH}"
 echo "Current branch : ${IGNITE_CURRENT_BRANCH}"
 echo
 echo "PATCHES_HOME   : ${PATCHES_HOME}"
@@ -84,4 +88,4 @@ echo
 
 requireCleanWorkTree ${IGNITE_HOME}
 
-formatPatch ${IGNITE_HOME} ${IGNITE_DEFAULT_BRANCH} ${IGNITE_CURRENT_BRANCH} .patch
\ No newline at end of file
+formatPatch ${IGNITE_HOME} ${IGNITE_DEFAULT_BRANCH} ${IGNITE_CURRENT_BRANCH} .patch

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/scripts/git-patch-functions.sh
----------------------------------------------------------------------
diff --git a/scripts/git-patch-functions.sh b/scripts/git-patch-functions.sh
index 796bccd..3cc1bb0 100644
--- a/scripts/git-patch-functions.sh
+++ b/scripts/git-patch-functions.sh
@@ -39,6 +39,14 @@ formatPatch () {
     PATCHED_BRANCH=$3
     PATCH_SUFFIX=$4
 
+    if [ ${IGNITE_CURRENT_BRANCH} = ${IGNITE_DEFAULT_BRANCH} ]
+    then
+        echo $0", ERROR:"
+        echo "You are on Default branch. Please, checkout branch with changes."
+
+        exit 1
+    fi
+
     cd ${GIT_HOME}
 
     git checkout ${DEFAULT_BRANCH}
@@ -54,15 +62,15 @@ formatPatch () {
     echo "Patch file created."
 
     git checkout ${PATCHED_BRANCH}
-    
+
     git branch -D tmppatch # Delete tmp branch.
-    
-    echo 
+
+    echo
     echo "Patch created: ${PATCH_FILE}"
 }
 
 #
-# Determines current branch.
+# Determines Current branch.
 #
 # Params:
 # - Git home.
@@ -70,11 +78,11 @@ formatPatch () {
 #
 determineCurrentBranch () {
     GIT_HOME=$1
-    
+
     cd ${GIT_HOME}
-    
+
     CURRENT_BRANCH=`git rev-parse --abbrev-ref HEAD`
-    
+
     echo "$CURRENT_BRANCH"
 }
 
@@ -131,22 +139,22 @@ applyPatch () {
     PATCH_FILE=$3
 
     cd ${GIT_HOME}
-    
+
     if [ ! -f ${PATCH_FILE} ]
     then
         echo $0", ERROR:"
         echo "Expected patch file not found: $PATCH_FILE."
-        
+
         exit 1
     fi
 
     echo "Patch $PATCH_FILE will be applied to $DEFAULT_BRANCH branch."
-    
+
     git am ${PATCH_FILE}
 }
 
 #
-# Checks that given default branch and current branch are equal.
+# Checks that given Default branch and Current branch are equal.
 # Exit with code 1 in error case.
 #
 # Params:
@@ -160,12 +168,12 @@ currentAndDefaultBranchesShouldBeEqual () {
     cd ${GIT_HOME}
 
     CURRENT_BRANCH=$( determineCurrentBranch ${GIT_HOME} )
-    
+
     if [ "$CURRENT_BRANCH" != "$DEFAULT_BRANCH" ]
-    then 
+    then
         echo $0", ERROR:"
         echo "You are not on an expected branch. Your current branch at $GIT_HOME is $CURRENT_BRANCH, should be $DEFAULT_BRANCH."
-        
+
         exit 1
     fi
 }


[21/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 4f74303..44b7997 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -221,18 +221,19 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
         if (topVer != null) {
             tx.topologyVersion(topVer);
 
-            prepare0();
+            prepare0(false);
 
             return;
         }
 
-        prepareOnTopology();
+        prepareOnTopology(false, null);
     }
 
     /**
-     *
+     * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void prepareOnTopology() {
+    private void prepareOnTopology(final boolean remap, @Nullable final Runnable c) {
         GridDhtTopologyFuture topFut = topologyReadLock();
 
         try {
@@ -265,16 +266,22 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                     return;
                 }
 
-                tx.topologyVersion(topFut.topologyVersion());
+                if (remap)
+                    tx.onRemap(topFut.topologyVersion());
+                else
+                    tx.topologyVersion(topFut.topologyVersion());
+
+                prepare0(remap);
 
-                prepare0();
+                if (c != null)
+                    c.run();
             }
             else {
                 topFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
                         cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
                             @Override public void run() {
-                                prepareOnTopology();
+                                prepareOnTopology(remap, c);
                             }
                         });
                     }
@@ -346,10 +353,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
 
     /**
      * Initializes future.
+     *
+     * @param remap Remap flag.
      */
-    private void prepare0() {
+    private void prepare0(boolean remap) {
         try {
-            if (!tx.state(PREPARING)) {
+            boolean txStateCheck = remap ? tx.state() == PREPARING : tx.state(PREPARING);
+
+            if (!txStateCheck) {
                 if (tx.setRollbackOnly()) {
                     if (tx.timedOut())
                         onError(null, null, new IgniteTxTimeoutCheckedException("Transaction timed out and " +
@@ -366,7 +377,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             }
 
             // Make sure to add future before calling prepare.
-            cctx.mvcc().addFuture(this);
+            if (!remap)
+                cctx.mvcc().addFuture(this);
 
             prepare(
                 tx.optimistic() && tx.serializable() ? tx.readEntries() : Collections.<IgniteTxEntry>emptyList(),
@@ -502,7 +514,8 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             tx.implicitSingle(),
             m.explicitLock(),
             tx.subjectId(),
-            tx.taskNameHash());
+            tx.taskNameHash(),
+            m.clientFirst());
 
         for (IgniteTxEntry txEntry : m.writes()) {
             if (txEntry.op() == TRANSFORM)
@@ -560,13 +573,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
      * @param entry Transaction entry.
      * @param topVer Topology version.
      * @param cur Current mapping.
+     * @param waitLock Wait lock flag.
      * @throws IgniteCheckedException If transaction is group-lock and local node is not primary for key.
      * @return Mapping.
      */
     private GridDistributedTxMapping map(
         IgniteTxEntry entry,
         AffinityTopologyVersion topVer,
-        GridDistributedTxMapping cur,
+        @Nullable GridDistributedTxMapping cur,
         boolean waitLock
     ) throws IgniteCheckedException {
         GridCacheContext cacheCtx = entry.context();
@@ -599,10 +613,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
         }
 
         if (cur == null || !cur.node().id().equals(primary.id()) || cur.near() != cacheCtx.isNear()) {
+            boolean clientFirst = cur == null && cctx.kernalContext().clientNode();
+
             cur = new GridDistributedTxMapping(primary);
 
             // Initialize near flag right away.
             cur.near(cacheCtx.isNear());
+
+            cur.clientFirst(clientFirst);
         }
 
         cur.add(entry);
@@ -748,18 +766,47 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                     onError(nodeId, mappings, res.error());
                 }
                 else {
-                    onPrepareResponse(m, res);
+                    if (res.clientRemapVersion() != null) {
+                        assert cctx.kernalContext().clientNode();
+                        assert m.clientFirst();
+
+                        IgniteInternalFuture<?> affFut = cctx.exchange().affinityReadyFuture(res.clientRemapVersion());
+
+                        if (affFut != null && !affFut.isDone()) {
+                            affFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> fut) {
+                                    remap();
+                                }
+                            });
+                        }
+                        else
+                            remap();
+                    }
+                    else {
+                        onPrepareResponse(m, res);
 
-                    // Proceed prepare before finishing mini future.
-                    if (mappings != null)
-                        proceedPrepare(mappings);
+                        // Proceed prepare before finishing mini future.
+                        if (mappings != null)
+                            proceedPrepare(mappings);
 
-                    // Finish this mini future.
-                    onDone(tx);
+                        // Finish this mini future.
+                        onDone(tx);
+                    }
                 }
             }
         }
 
+        /**
+         *
+         */
+        private void remap() {
+            prepareOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(tx);
+                }
+            });
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(MiniFuture.class, this, "done", isDone(), "cancelled", isCancelled(), "err", error());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
index bce62c1..7006114 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearPessimisticTxPrepareFuture.java
@@ -84,6 +84,8 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
     /** {@inheritDoc} */
     @Override public void onResult(UUID nodeId, GridNearTxPrepareResponse res) {
         if (!isDone()) {
+            assert res.clientRemapVersion() == null : res;
+
             for (IgniteInternalFuture<IgniteInternalTx> fut : pending()) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -187,7 +189,8 @@ public class GridNearPessimisticTxPrepareFuture extends GridNearTxPrepareFutureA
                 tx.implicitSingle(),
                 m.explicitLock(),
                 tx.subjectId(),
-                tx.taskNameHash());
+                tx.taskNameHash(),
+                false);
 
             for (IgniteTxEntry txEntry : m.writes()) {
                 if (txEntry.op() == TRANSFORM)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index df7a65f..696acfb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -403,7 +403,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
         assert nodeId != null;
         assert res != null;
 
-        GridNearLockFuture<K, V> fut = (GridNearLockFuture<K, V>)ctx.mvcc().<Boolean>future(res.version(),
+        GridNearLockFuture fut = (GridNearLockFuture)ctx.mvcc().<Boolean>future(res.version(),
             res.futureId());
 
         if (fut != null)
@@ -423,7 +423,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
     ) {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        GridNearLockFuture<K, V> fut = new GridNearLockFuture<>(ctx,
+        GridNearLockFuture fut = new GridNearLockFuture(ctx,
             keys,
             (GridNearTxLocal)tx,
             isRead,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index c38965d..fa8877a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -56,8 +56,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     private static final long serialVersionUID = 0L;
 
     /** DHT mappings. */
-    private ConcurrentMap<UUID, GridDistributedTxMapping> mappings =
-        new ConcurrentHashMap8<>();
+    private ConcurrentMap<UUID, GridDistributedTxMapping> mappings = new ConcurrentHashMap8<>();
 
     /** Future. */
     @GridToStringExclude
@@ -65,13 +64,11 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
 
     /** */
     @GridToStringExclude
-    private final AtomicReference<GridNearTxFinishFuture> commitFut =
-        new AtomicReference<>();
+    private final AtomicReference<GridNearTxFinishFuture> commitFut = new AtomicReference<>();
 
     /** */
     @GridToStringExclude
-    private final AtomicReference<GridNearTxFinishFuture> rollbackFut =
-        new AtomicReference<>();
+    private final AtomicReference<GridNearTxFinishFuture> rollbackFut = new AtomicReference<>();
 
     /** Entries to lock on next step of prepare stage. */
     private Collection<IgniteTxEntry> optimisticLockEntries = Collections.emptyList();
@@ -85,6 +82,9 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     /** Info for entries accessed locally in optimistic transaction. */
     private Map<IgniteTxKey, IgniteCacheExpiryPolicy> accessMap;
 
+    /** */
+    private boolean hasRemoteLocks;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -97,6 +97,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
      * @param implicit Implicit flag.
      * @param implicitSingle Implicit with one key flag.
      * @param sys System flag.
+     * @param plc IO policy.
      * @param concurrency Concurrency.
      * @param isolation Isolation.
      * @param timeout Timeout.
@@ -1185,6 +1186,36 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public void onRemap(AffinityTopologyVersion topVer) {
+        assert cctx.kernalContext().clientNode();
+
+        mapped.set(false);
+        nearLocallyMapped = false;
+        colocatedLocallyMapped = false;
+        txNodes = null;
+        onePhaseCommit = false;
+        nearMap.clear();
+        dhtMap.clear();
+        mappings.clear();
+
+        this.topVer.set(topVer);
+    }
+
+    /**
+     * @param hasRemoteLocks {@code True} if tx has remote locks acquired.
+     */
+    public void hasRemoteLocks(boolean hasRemoteLocks) {
+        this.hasRemoteLocks = hasRemoteLocks;
+    }
+
+    /**
+     * @return {@code True} if tx has remote locks acquired.
+     */
+    public boolean hasRemoteLocks() {
+        return hasRemoteLocks;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNearTxLocal.class, this, "mappings", mappings.keySet(), "super", super.toString());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
index a08637d..b602a7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareRequest.java
@@ -75,6 +75,9 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
     /** Task name hash. */
     private int taskNameHash;
 
+    /** {@code True} if first optimistic tx prepare request sent from client node. */
+    private boolean firstClientReq;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -92,8 +95,13 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
      * @param txNodes Transaction nodes mapping.
      * @param last {@code True} if this last prepare request for node.
      * @param lastBackups IDs of backup nodes receiving last prepare request during this prepare.
+     * @param onePhaseCommit One phase commit flag.
+     * @param retVal Return value flag.
+     * @param implicitSingle Implicit single flag.
+     * @param explicitLock Explicit lock flag.
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash.
+     * @param firstClientReq {@code True} if first optimistic tx prepare request sent from client node.
      */
     public GridNearTxPrepareRequest(
         IgniteUuid futId,
@@ -110,11 +118,13 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         boolean implicitSingle,
         boolean explicitLock,
         @Nullable UUID subjId,
-        int taskNameHash
+        int taskNameHash,
+        boolean firstClientReq
     ) {
         super(tx, reads, writes, txNodes, onePhaseCommit);
 
         assert futId != null;
+        assert !firstClientReq || tx.optimistic() : tx;
 
         this.futId = futId;
         this.topVer = topVer;
@@ -126,6 +136,14 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
         this.explicitLock = explicitLock;
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
+        this.firstClientReq = firstClientReq;
+    }
+
+    /**
+     * @return {@code True} if first optimistic tx prepare request sent from client node.
+     */
+    public boolean firstClientRequest() {
+        return firstClientReq;
     }
 
     /**
@@ -273,60 +291,66 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 writer.incrementState();
 
             case 24:
-                if (!writer.writeIgniteUuid("futId", futId))
+                if (!writer.writeBoolean("firstClientReq", firstClientReq))
                     return false;
 
                 writer.incrementState();
 
             case 25:
-                if (!writer.writeBoolean("implicitSingle", implicitSingle))
+                if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 26:
-                if (!writer.writeBoolean("last", last))
+                if (!writer.writeBoolean("implicitSingle", implicitSingle))
                     return false;
 
                 writer.incrementState();
 
             case 27:
-                if (!writer.writeCollection("lastBackups", lastBackups, MessageCollectionItemType.UUID))
+                if (!writer.writeBoolean("last", last))
                     return false;
 
                 writer.incrementState();
 
             case 28:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeCollection("lastBackups", lastBackups, MessageCollectionItemType.UUID))
                     return false;
 
                 writer.incrementState();
 
             case 29:
-                if (!writer.writeBoolean("near", near))
+                if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 30:
-                if (!writer.writeBoolean("retVal", retVal))
+                if (!writer.writeBoolean("near", near))
                     return false;
 
                 writer.incrementState();
 
             case 31:
-                if (!writer.writeUuid("subjId", subjId))
+                if (!writer.writeBoolean("retVal", retVal))
                     return false;
 
                 writer.incrementState();
 
             case 32:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
+                if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
             case 33:
+                if (!writer.writeInt("taskNameHash", taskNameHash))
+                    return false;
+
+                writer.incrementState();
+
+            case 34:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -357,7 +381,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 24:
-                futId = reader.readIgniteUuid("futId");
+                firstClientReq = reader.readBoolean("firstClientReq");
 
                 if (!reader.isLastRead())
                     return false;
@@ -365,7 +389,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 25:
-                implicitSingle = reader.readBoolean("implicitSingle");
+                futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -373,7 +397,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 26:
-                last = reader.readBoolean("last");
+                implicitSingle = reader.readBoolean("implicitSingle");
 
                 if (!reader.isLastRead())
                     return false;
@@ -381,7 +405,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 27:
-                lastBackups = reader.readCollection("lastBackups", MessageCollectionItemType.UUID);
+                last = reader.readBoolean("last");
 
                 if (!reader.isLastRead())
                     return false;
@@ -389,7 +413,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 28:
-                miniId = reader.readIgniteUuid("miniId");
+                lastBackups = reader.readCollection("lastBackups", MessageCollectionItemType.UUID);
 
                 if (!reader.isLastRead())
                     return false;
@@ -397,7 +421,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 29:
-                near = reader.readBoolean("near");
+                miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -405,7 +429,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 30:
-                retVal = reader.readBoolean("retVal");
+                near = reader.readBoolean("near");
 
                 if (!reader.isLastRead())
                     return false;
@@ -413,7 +437,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 31:
-                subjId = reader.readUuid("subjId");
+                retVal = reader.readBoolean("retVal");
 
                 if (!reader.isLastRead())
                     return false;
@@ -421,7 +445,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 32:
-                taskNameHash = reader.readInt("taskNameHash");
+                subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -429,6 +453,14 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
                 reader.incrementState();
 
             case 33:
+                taskNameHash = reader.readInt("taskNameHash");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 34:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -448,7 +480,7 @@ public class GridNearTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 34;
+        return 35;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
index f8c07f7..0f0b2c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareResponse.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
@@ -27,6 +28,7 @@ import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.nio.*;
@@ -83,6 +85,9 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
     @GridDirectCollection(IgniteTxKey.class)
     private Collection<IgniteTxKey> filterFailedKeys;
 
+    /** Not {@code null} if client node should remap transaction. */
+    private AffinityTopologyVersion clientRemapVer;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -95,9 +100,11 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
      * @param futId Future ID.
      * @param miniId Mini future ID.
      * @param dhtVer DHT version.
+     * @param writeVer Write version.
      * @param invalidParts Invalid partitions.
      * @param retVal Return value.
      * @param err Error.
+     * @param clientRemapVer Not {@code null} if client node should remap transaction.
      */
     public GridNearTxPrepareResponse(
         GridCacheVersion xid,
@@ -107,7 +114,8 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         GridCacheVersion writeVer,
         Collection<Integer> invalidParts,
         GridCacheReturn retVal,
-        Throwable err
+        Throwable err,
+        AffinityTopologyVersion clientRemapVer
     ) {
         super(xid, err);
 
@@ -121,6 +129,14 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
         this.writeVer = writeVer;
         this.invalidParts = invalidParts;
         this.retVal = retVal;
+        this.clientRemapVer = clientRemapVer;
+    }
+
+    /**
+     * @return {@code True} if client node should remap transaction.
+     */
+    @Nullable public AffinityTopologyVersion clientRemapVersion() {
+        return clientRemapVer;
     }
 
     /**
@@ -330,60 +346,66 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
         switch (writer.state()) {
             case 10:
-                if (!writer.writeMessage("dhtVer", dhtVer))
+                if (!writer.writeMessage("clientRemapVer", clientRemapVer))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeCollection("filterFailedKeys", filterFailedKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("dhtVer", dhtVer))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeIgniteUuid("futId", futId))
+                if (!writer.writeCollection("filterFailedKeys", filterFailedKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeCollection("invalidParts", invalidParts, MessageCollectionItemType.INT))
+                if (!writer.writeIgniteUuid("futId", futId))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeCollection("invalidParts", invalidParts, MessageCollectionItemType.INT))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG))
+                if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG))
+                if (!writer.writeCollection("ownedValKeys", ownedValKeys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
+                if (!writer.writeCollection("ownedValVals", ownedValVals, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeMessage("retVal", retVal))
+                if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 19:
+                if (!writer.writeMessage("retVal", retVal))
+                    return false;
+
+                writer.incrementState();
+
+            case 20:
                 if (!writer.writeMessage("writeVer", writeVer))
                     return false;
 
@@ -406,7 +428,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
         switch (reader.state()) {
             case 10:
-                dhtVer = reader.readMessage("dhtVer");
+                clientRemapVer = reader.readMessage("clientRemapVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -414,7 +436,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 11:
-                filterFailedKeys = reader.readCollection("filterFailedKeys", MessageCollectionItemType.MSG);
+                dhtVer = reader.readMessage("dhtVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -422,7 +444,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 12:
-                futId = reader.readIgniteUuid("futId");
+                filterFailedKeys = reader.readCollection("filterFailedKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -430,7 +452,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 13:
-                invalidParts = reader.readCollection("invalidParts", MessageCollectionItemType.INT);
+                futId = reader.readIgniteUuid("futId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -438,7 +460,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 14:
-                miniId = reader.readIgniteUuid("miniId");
+                invalidParts = reader.readCollection("invalidParts", MessageCollectionItemType.INT);
 
                 if (!reader.isLastRead())
                     return false;
@@ -446,7 +468,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 15:
-                ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG);
+                miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -454,7 +476,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 16:
-                ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG);
+                ownedValKeys = reader.readCollection("ownedValKeys", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -462,7 +484,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 17:
-                pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
+                ownedValVals = reader.readCollection("ownedValVals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -470,7 +492,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 18:
-                retVal = reader.readMessage("retVal");
+                pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -478,6 +500,14 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
                 reader.incrementState();
 
             case 19:
+                retVal = reader.readMessage("retVal");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 20:
                 writeVer = reader.readMessage("writeVer");
 
                 if (!reader.isLastRead())
@@ -497,7 +527,7 @@ public class GridNearTxPrepareResponse extends GridDistributedTxPrepareResponse
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 20;
+        return 21;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 6120e25..4adcff5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -41,7 +41,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     private static final long serialVersionUID = 0L;
 
     /** */
-    private GridCachePreloader<K,V> preldr;
+    private GridCachePreloader preldr;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -56,7 +56,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     public GridLocalCache(GridCacheContext<K, V> ctx) {
         super(ctx, ctx.config().getStartSize());
 
-        preldr = new GridCachePreloaderAdapter<>(ctx);
+        preldr = new GridCachePreloaderAdapter(ctx);
     }
 
     /** {@inheritDoc} */
@@ -65,7 +65,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public GridCachePreloader<K, V> preloader() {
+    @Override public GridCachePreloader preloader() {
         return preldr;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 819b0f0..bcbdec4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -53,7 +53,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     private static final sun.misc.Unsafe UNSAFE = GridUnsafe.unsafe();
 
     /** */
-    private GridCachePreloader<K,V> preldr;
+    private GridCachePreloader preldr;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -68,7 +68,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     public GridLocalAtomicCache(GridCacheContext<K, V> ctx) {
         super(ctx, ctx.config().getStartSize());
 
-        preldr = new GridCachePreloaderAdapter<>(ctx);
+        preldr = new GridCachePreloaderAdapter(ctx);
     }
 
     /** {@inheritDoc} */
@@ -94,7 +94,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public GridCachePreloader<K, V> preloader() {
+    @Override public GridCachePreloader preloader() {
         return preldr;
     }
 
@@ -119,7 +119,11 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @Override public boolean put(K key, V val, CacheEntryPredicate[] filter) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
-        return (Boolean)updateAllInternal(UPDATE,
+        boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        long start = statsEnabled ? System.nanoTime() : 0L;
+
+        boolean res = (Boolean)updateAllInternal(UPDATE,
             Collections.singleton(key),
             Collections.singleton(val),
             null,
@@ -129,6 +133,11 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             filter,
             ctx.writeThrough(),
             ctx.readThrough());
+
+        if (statsEnabled)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
+
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -268,6 +277,10 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        boolean statsEnabled = ctx.config().isStatisticsEnabled();
+
+        long start = statsEnabled ? System.nanoTime() : 0L;
+
         updateAllInternal(UPDATE,
             m.keySet(),
             m.values(),
@@ -278,6 +291,9 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             CU.empty0(),
             ctx.writeThrough(),
             ctx.readThrough());
+
+        if (statsEnabled)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
@@ -727,7 +743,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
         final ExpiryPolicy expiry = expiryPerCall();
 
-        return asyncOp(new Callable<Object>() {
+        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(op,
                     keys,
@@ -741,6 +757,11 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                     readThrough);
             }
         });
+
+        if (ctx.config().isStatisticsEnabled())
+            fut.listen(new UpdatePutTimeStatClosure(metrics0(), System.nanoTime()));
+
+        return fut;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 4b1fc87..7f0a5ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -348,17 +348,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
      * @param duration Duration.
      */
     public void onExecuted(Object res, Throwable err, long startTime, long duration) {
-        boolean fail = err != null;
-
-        // Update own metrics.
-        metrics.onQueryExecute(duration, fail);
-
-        // Update metrics in query manager.
-        cctx.queries().onMetricsUpdate(duration, fail);
-
-        if (log.isDebugEnabled())
-            log.debug("Query execution finished [qry=" + this + ", startTime=" + startTime +
-                ", duration=" + duration + ", fail=" + fail + ", res=" + res + ']');
+        GridQueryProcessor.onExecuted(cctx, metrics, res, err, startTime, duration, log);
     }
 
     /** {@inheritDoc} */
@@ -376,10 +366,12 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         return execute(null, rmtTransform, args);
     }
 
+    /** {@inheritDoc} */
     @Override public QueryMetrics metrics() {
         return metrics.copy();
     }
 
+    /** {@inheritDoc} */
     @Override public void resetMetrics() {
         metrics = new GridCacheQueryMetricsAdapter();
     }
@@ -470,10 +462,14 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
     private static Collection<ClusterNode> nodes(final GridCacheContext<?, ?> cctx, @Nullable final ClusterGroup prj) {
         assert cctx != null;
 
-        return F.view(CU.allNodes(cctx), new P1<ClusterNode>() {
+        Collection<ClusterNode> affNodes = CU.affinityNodes(cctx);
+
+        if (prj == null)
+            return affNodes;
+
+        return F.view(affNodes, new P1<ClusterNode>() {
             @Override public boolean apply(ClusterNode n) {
-                return cctx.discovery().cacheAffinityNode(n, cctx.name()) &&
-                    (prj == null || prj.node(n.id()) != null);
+                return prj.node(n.id()) != null;
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
index 2999e7b..15eb368 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
@@ -43,6 +43,8 @@ public class GridCacheQueryErrorFuture<T> extends GridFinishedFuture<Collection<
 
     /** {@inheritDoc} */
     @Nullable @Override public T next() throws IgniteCheckedException {
+        get();
+
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 16a8028..32e9d63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -773,7 +773,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
 
-            private Iterator<K> iter = backups ? prj.keySet().iterator() : prj.primaryKeySet().iterator();
+            private Iterator<K> iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 
             {
                 advance();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 759a949..6277c5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -250,8 +250,13 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @return Continuous routine ID.
      * @throws IgniteCheckedException In case of error.
      */
-    public UUID executeQuery(CacheEntryUpdatedListener locLsnr, CacheEntryEventSerializableFilter rmtFilter,
-        int bufSize, long timeInterval, boolean autoUnsubscribe, ClusterGroup grp) throws IgniteCheckedException {
+    public UUID executeQuery(CacheEntryUpdatedListener locLsnr,
+        CacheEntryEventSerializableFilter rmtFilter,
+        int bufSize,
+        long timeInterval,
+        boolean autoUnsubscribe,
+        ClusterGroup grp) throws IgniteCheckedException
+    {
         return executeQuery0(
             locLsnr,
             rmtFilter,
@@ -301,7 +306,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         try {
             cctx.kernalContext().continuous().stopRoutine(routineId).get();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteCheckedException | IgniteException e) {
             if (log.isDebugEnabled())
                 log.debug("Failed to stop internal continuous query: " + e.getMessage());
         }
@@ -357,9 +362,18 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @return Continuous routine ID.
      * @throws IgniteCheckedException In case of error.
      */
-    private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, final CacheEntryEventSerializableFilter rmtFilter,
-        int bufSize, long timeInterval, boolean autoUnsubscribe, boolean internal, boolean notifyExisting,
-        boolean oldValRequired, boolean sync, boolean ignoreExpired, ClusterGroup grp) throws IgniteCheckedException {
+    private UUID executeQuery0(CacheEntryUpdatedListener locLsnr,
+        final CacheEntryEventSerializableFilter rmtFilter,
+        int bufSize,
+        long timeInterval,
+        boolean autoUnsubscribe,
+        boolean internal,
+        boolean notifyExisting,
+        boolean oldValRequired,
+        boolean sync,
+        boolean ignoreExpired,
+        ClusterGroup grp) throws IgniteCheckedException
+    {
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (grp == null)
@@ -745,7 +759,7 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            impl = (CacheEntryEventSerializableFilter)in.readObject();
+            impl = (CacheEntryEventFilter)in.readObject();
             types = in.readByte();
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
index 5fde622..02fe679 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheOsStoreManager.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.store;
 
-import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index d9f50ac..a14df6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -68,6 +68,11 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
     public boolean isWriteThrough();
 
     /**
+     * @return {@code True} is write-behind is enabled.
+     */
+    public boolean isWriteBehind();
+
+    /**
      * @return Whether DHT transaction can write to store from DHT.
      */
     public boolean isWriteToStoreFromDht();
@@ -160,7 +165,7 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
      * @param commit Commit.
      * @throws IgniteCheckedException If failed.
      */
-    public void sessionEnd(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException;
+    public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException;
 
     /**
      * End session initiated by write-behind store.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index f9a966c..b4a146a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -59,11 +59,20 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     private ThreadLocal<SessionData> sesHolder;
 
     /** */
+    private ThreadLocalSession locSes;
+
+    /** */
     private boolean locStore;
 
     /** */
     private boolean writeThrough;
 
+    /** */
+    private Collection<CacheStoreSessionListener> sesLsnrs;
+
+    /** */
+    private boolean globalSesLsnrs;
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void initialize(@Nullable CacheStore cfgStore, Map sesHolders) throws IgniteCheckedException {
@@ -84,14 +93,15 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             sesHolder0 = ((Map<CacheStore, ThreadLocal>)sesHolders).get(cfgStore);
 
             if (sesHolder0 == null) {
-                ThreadLocalSession locSes = new ThreadLocalSession();
+                sesHolder0 = new ThreadLocal<>();
 
-                if (ctx.resource().injectStoreSession(cfgStore, locSes)) {
-                    sesHolder0 = locSes.sesHolder;
+                locSes = new ThreadLocalSession(sesHolder0);
 
+                if (ctx.resource().injectStoreSession(cfgStore, locSes))
                     sesHolders.put(cfgStore, sesHolder0);
-                }
             }
+            else
+                locSes = new ThreadLocalSession(sesHolder0);
         }
 
         sesHolder = sesHolder0;
@@ -148,6 +158,24 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException("Failed to start cache store: " + e, e);
             }
         }
+
+        CacheConfiguration cfg = cctx.config();
+
+        if (cfgStore != null && !cfg.isWriteThrough() && !cfg.isReadThrough()) {
+            U.quietAndWarn(log,
+                "Persistence store is configured, but both read-through and write-through are disabled. This " +
+                "configuration makes sense if the store implements loadCache method only. If this is the " +
+                "case, ignore this warning. Otherwise, fix the configuration for cache: " + cfg.getName(),
+                "Persistence store is configured, but both read-through and write-through are disabled.");
+        }
+
+        sesLsnrs = CU.startStoreSessionListeners(cctx.kernalContext(), cfg.getCacheStoreSessionListenerFactories());
+
+        if (sesLsnrs == null) {
+            sesLsnrs = cctx.shared().storeSessionListeners();
+
+            globalSesLsnrs = true;
+        }
     }
 
     /** {@inheritDoc} */
@@ -164,6 +192,15 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 U.error(log(), "Failed to stop cache store.", e);
             }
         }
+
+        if (!globalSesLsnrs) {
+            try {
+                CU.stopStoreSessionListeners(cctx.kernalContext(), sesLsnrs);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to stop store session listeners for cache: " + cctx.name(), e);
+            }
+        }
     }
 
     /** {@inheritDoc} */
@@ -215,14 +252,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             Object val = null;
 
             try {
                 val = singleThreadGate.load(storeKey);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -234,7 +271,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -264,8 +301,13 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
+    @Override public boolean isWriteBehind() {
+        return cctx.config().isWriteBehindEnabled();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isWriteToStoreFromDht() {
-        return cctx.config().isWriteBehindEnabled() || locStore;
+        return isWriteBehind() || locStore;
     }
 
     /** {@inheritDoc} */
@@ -349,7 +391,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 IgniteBiInClosure<Object, Object> c = new CI2<Object, Object>() {
@@ -380,7 +422,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 else
                     singleThreadGate.loadAll(keys0, c);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -392,7 +434,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -408,7 +450,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(null);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.loadCache(new IgniteBiInClosure<Object, Object>() {
@@ -431,7 +473,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     }
                 }, args);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (CacheLoaderException e) {
                 throw new IgniteCheckedException(e);
@@ -440,7 +482,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheLoaderException(e));
             }
             finally {
-                sessionEnd0(null, thewEx);
+                sessionEnd0(null, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -473,12 +515,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.write(new CacheEntryImpl<>(key, locStore ? F.t(val, ver) : val));
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -490,7 +532,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheWriterException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -522,12 +564,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
                 sessionInit0(tx);
 
-                boolean thewEx = true;
+                boolean threwEx = true;
 
                 try {
                     store.writeAll(entries);
 
-                    thewEx = false;
+                    threwEx = false;
                 }
                 catch (ClassCastException e) {
                     handleClassCastException(e);
@@ -548,7 +590,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                     throw new IgniteCheckedException(e);
                 }
                 finally {
-                    sessionEnd0(tx, thewEx);
+                    sessionEnd0(tx, threwEx);
                 }
 
                 if (log.isDebugEnabled())
@@ -576,12 +618,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.delete(key);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -593,7 +635,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(new CacheWriterException(e));
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -606,8 +648,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys)
-        throws IgniteCheckedException {
+    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -625,12 +666,12 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
             sessionInit0(tx);
 
-            boolean thewEx = true;
+            boolean threwEx = true;
 
             try {
                 store.deleteAll(keys0);
 
-                thewEx = false;
+                threwEx = false;
             }
             catch (ClassCastException e) {
                 handleClassCastException(e);
@@ -645,7 +686,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
                 throw new IgniteCheckedException(e);
             }
             finally {
-                sessionEnd0(tx, thewEx);
+                sessionEnd0(tx, threwEx);
             }
 
             if (log.isDebugEnabled())
@@ -669,16 +710,27 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit) throws IgniteCheckedException {
+    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
         assert store != null;
 
         sessionInit0(tx);
 
         try {
-            store.sessionEnd(commit);
+            if (sesLsnrs != null) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionEnd(locSes, commit);
+            }
+
+            if (!sesHolder.get().ended(store))
+                store.sessionEnd(commit);
+        }
+        catch (Throwable e) {
+            last = true;
+
+            throw e;
         }
         finally {
-            if (sesHolder != null) {
+            if (last && sesHolder != null) {
                 sesHolder.set(null);
 
                 tx.removeMeta(SES_ATTR);
@@ -715,10 +767,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      * @param tx Current transaction.
      */
     private void sessionInit0(@Nullable IgniteInternalTx tx) {
-        if (sesHolder == null)
-            return;
-
-        assert sesHolder.get() == null;
+        assert sesHolder != null;
 
         SessionData ses;
 
@@ -738,6 +787,11 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             ses = new SessionData(null, cctx.name());
 
         sesHolder.set(ses);
+
+        if (sesLsnrs != null && !ses.started(this)) {
+            for (CacheStoreSessionListener lsnr : sesLsnrs)
+                lsnr.onSessionStart(locSes);
+        }
     }
 
     /**
@@ -745,8 +799,16 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private void sessionEnd0(@Nullable IgniteInternalTx tx, boolean threwEx) throws IgniteCheckedException {
         try {
-            if (tx == null)
-                store.sessionEnd(threwEx);
+            if (tx == null) {
+                if (sesLsnrs != null) {
+                    for (CacheStoreSessionListener lsnr : sesLsnrs)
+                        lsnr.onSessionEnd(locSes, !threwEx);
+                }
+
+                assert !sesHolder.get().ended(store);
+
+                store.sessionEnd(!threwEx);
+            }
         }
         catch (Exception e) {
             if (!threwEx)
@@ -788,6 +850,16 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         @GridToStringInclude
         private Map<Object, Object> props;
 
+        /** */
+        private Object attachment;
+
+        /** */
+        private final Set<CacheStoreManager> started =
+            new GridSetWrapper<>(new IdentityHashMap<CacheStoreManager, Object>());
+
+        /** */
+        private final Set<CacheStore> ended = new GridSetWrapper<>(new IdentityHashMap<CacheStore, Object>());
+
         /**
          * @param tx Current transaction.
          * @param cacheName Cache name.
@@ -815,6 +887,24 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /**
+         * @param attachment Attachment.
+         */
+        private Object attach(Object attachment) {
+            Object prev = this.attachment;
+
+            this.attachment = attachment;
+
+            return prev;
+        }
+
+        /**
+         * @return Attachment.
+         */
+        private Object attachment() {
+            return attachment;
+        }
+
+        /**
          * @return Cache name.
          */
         private String cacheName() {
@@ -828,6 +918,21 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             this.cacheName = cacheName;
         }
 
+        /**
+         * @return If session is started.
+         */
+        private boolean started(CacheStoreManager mgr) {
+            return !started.add(mgr);
+        }
+
+        /**
+         * @param store Cache store.
+         * @return Whether session already ended on this store instance.
+         */
+        private boolean ended(CacheStore store) {
+            return !ended.add(store);
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(SessionData.class, this, "tx", CU.txString(tx));
@@ -839,7 +944,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
      */
     private static class ThreadLocalSession implements CacheStoreSession {
         /** */
-        private final ThreadLocal<SessionData> sesHolder = new ThreadLocal<>();
+        private final ThreadLocal<SessionData> sesHolder;
+
+        /**
+         * @param sesHolder Session holder.
+         */
+        private ThreadLocalSession(ThreadLocal<SessionData> sesHolder) {
+            this.sesHolder = sesHolder;
+        }
 
         /** {@inheritDoc} */
         @Nullable @Override public Transaction transaction() {
@@ -854,6 +966,20 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
         }
 
         /** {@inheritDoc} */
+        @Override public Object attach(@Nullable Object attachment) {
+            SessionData ses0 = sesHolder.get();
+
+            return ses0 != null ? ses0.attach(attachment) : null;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public <T> T attachment() {
+            SessionData ses0 = sesHolder.get();
+
+            return ses0 != null ? (T)ses0.attachment() : null;
+        }
+
+        /** {@inheritDoc} */
         @Override public <K1, V1> Map<K1, V1> properties() {
             SessionData ses0 = sesHolder.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
index 5f877ec..cb86e0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteInternalTx.java
@@ -706,4 +706,9 @@ public interface IgniteInternalTx extends AutoCloseable, GridTimeoutObject {
      * @return Public API proxy.
      */
     public TransactionProxy proxy();
+
+    /**
+     * @param topVer New topology version.
+     */
+    public void onRemap(AffinityTopologyVersion topVer);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index eb8825e..9e8950f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -184,7 +184,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     private AtomicReference<GridFutureAdapter<IgniteInternalTx>> finFut = new AtomicReference<>();
 
     /** Topology version. */
-    private AtomicReference<AffinityTopologyVersion> topVer = new AtomicReference<>(AffinityTopologyVersion.NONE);
+    protected AtomicReference<AffinityTopologyVersion> topVer = new AtomicReference<>(AffinityTopologyVersion.NONE);
 
     /** Mutex. */
     private final Lock lock = new ReentrantLock();
@@ -405,7 +405,21 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @Override public boolean storeUsed() {
-        return storeEnabled() && store() != null;
+        if (!storeEnabled())
+            return false;
+
+        Collection<Integer> cacheIds = activeCacheIds();
+
+        if (!cacheIds.isEmpty()) {
+            for (int cacheId : cacheIds) {
+                CacheStoreManager store = cctx.cacheContext(cacheId).store();
+
+                if (store.configured())
+                    return true;
+            }
+        }
+
+        return false;
     }
 
     /**
@@ -413,13 +427,20 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
      *
      * @return Store manager.
      */
-    protected CacheStoreManager store() {
-        if (!activeCacheIds().isEmpty()) {
-            int cacheId = F.first(activeCacheIds());
+    protected Collection<CacheStoreManager> stores() {
+        Collection<Integer> cacheIds = activeCacheIds();
+
+        if (!cacheIds.isEmpty()) {
+            Collection<CacheStoreManager> stores = new ArrayList<>(cacheIds.size());
 
-            CacheStoreManager store = cctx.cacheContext(cacheId).store();
+            for (int cacheId : cacheIds) {
+                CacheStoreManager store = cctx.cacheContext(cacheId).store();
 
-            return store.configured() ? store : null;
+                if (store.configured())
+                    stores.add(store);
+            }
+
+            return stores;
         }
 
         return null;
@@ -493,13 +514,17 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
     }
 
     /** {@inheritDoc} */
+    @Override public void onRemap(AffinityTopologyVersion topVer) {
+        assert false : this;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean hasTransforms() {
         return transform;
     }
 
     /** {@inheritDoc} */
-    @Override
-    public boolean markPreparing() {
+    @Override public boolean markPreparing() {
         return preparing.compareAndSet(false, true);
     }
 
@@ -1716,6 +1741,11 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
+        @Override public void onRemap(AffinityTopologyVersion topVer) {
+            throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
+        }
+
+        /** {@inheritDoc} */
         @Override public boolean empty() {
             throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index f466bf2..e6d71aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
@@ -229,14 +230,22 @@ public class IgniteTxHandler {
             return null;
         }
 
+        IgniteTxEntry firstEntry = null;
+
         try {
-            for (IgniteTxEntry e : F.concat(false, req.reads(), req.writes()))
+            for (IgniteTxEntry e : F.concat(false, req.reads(), req.writes())) {
                 e.unmarshal(ctx, false, ctx.deploy().globalLoader());
+
+                if (firstEntry == null)
+                    firstEntry = e;
+            }
         }
         catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(e);
         }
 
+        assert firstEntry != null : req;
+
         GridDhtTxLocal tx;
 
         GridCacheVersion mappedVer = ctx.tm().mappedVersion(req.version());
@@ -253,36 +262,88 @@ public class IgniteTxHandler {
             }
         }
         else {
-            tx = new GridDhtTxLocal(
-                ctx,
-                nearNode.id(),
-                req.version(),
-                req.futureId(),
-                req.miniId(),
-                req.threadId(),
-                req.implicitSingle(),
-                req.implicitSingle(),
-                req.system(),
-                req.explicitLock(),
-                req.policy(),
-                req.concurrency(),
-                req.isolation(),
-                req.timeout(),
-                req.isInvalidate(),
-                false,
-                req.txSize(),
-                req.transactionNodes(),
-                req.subjectId(),
-                req.taskNameHash()
-            );
+            GridDhtPartitionTopology top = null;
 
-            tx = ctx.tm().onCreated(null, tx);
+            if (req.firstClientRequest()) {
+                assert req.concurrency().equals(OPTIMISTIC) : req;
+                assert CU.clientNode(nearNode) : nearNode;
 
-            if (tx != null)
-                tx.topologyVersion(req.topologyVersion());
-            else
-                U.warn(log, "Failed to create local transaction (was transaction rolled back?) [xid=" +
-                    req.version() + ", req=" + req + ']');
+                top = firstEntry.context().topology();
+
+                top.readLock();
+            }
+
+            try {
+                if (top != null && needRemap(req.topologyVersion(), top.topologyVersion(), req)) {
+                    if (log.isDebugEnabled()) {
+                        log.debug("Client topology version mismatch, need remap transaction [" +
+                            "reqTopVer=" + req.topologyVersion() +
+                            ", locTopVer=" + top.topologyVersion() +
+                            ", req=" + req + ']');
+                    }
+
+                    GridNearTxPrepareResponse res = new GridNearTxPrepareResponse(
+                        req.version(),
+                        req.futureId(),
+                        req.miniId(),
+                        req.version(),
+                        req.version(),
+                        null,
+                        null,
+                        null,
+                        top.topologyVersion());
+
+                    try {
+                        ctx.io().send(nearNode, res, req.policy());
+                    }
+                    catch (ClusterTopologyCheckedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send client tx remap response, client node failed " +
+                                "[node=" + nearNode + ", req=" + req + ']');
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send client tx remap response " +
+                            "[node=" + nearNode + ", req=" + req + ']', e);
+                    }
+
+                    return new GridFinishedFuture<>(res);
+                }
+
+                tx = new GridDhtTxLocal(
+                    ctx,
+                    nearNode.id(),
+                    req.version(),
+                    req.futureId(),
+                    req.miniId(),
+                    req.threadId(),
+                    req.implicitSingle(),
+                    req.implicitSingle(),
+                    req.system(),
+                    req.explicitLock(),
+                    req.policy(),
+                    req.concurrency(),
+                    req.isolation(),
+                    req.timeout(),
+                    req.isInvalidate(),
+                    false,
+                    req.txSize(),
+                    req.transactionNodes(),
+                    req.subjectId(),
+                    req.taskNameHash()
+                );
+
+                tx = ctx.tm().onCreated(null, tx);
+
+                if (tx != null)
+                    tx.topologyVersion(req.topologyVersion());
+                else
+                    U.warn(log, "Failed to create local transaction (was transaction rolled back?) [xid=" +
+                        req.version() + ", req=" + req + ']');
+            }
+            finally {
+                if (top != null)
+                    top.readUnlock();
+            }
         }
 
         if (tx != null) {
@@ -343,6 +404,31 @@ public class IgniteTxHandler {
     }
 
     /**
+     * @param expVer Expected topology version.
+     * @param curVer Current topology version.
+     * @param req Request.
+     * @return {@code True} if cache affinity changed and request should be remapped.
+     */
+    private boolean needRemap(AffinityTopologyVersion expVer,
+        AffinityTopologyVersion curVer,
+        GridNearTxPrepareRequest req) {
+        if (expVer.equals(curVer))
+            return false;
+
+        for (IgniteTxEntry e : F.concat(false, req.reads(), req.writes())) {
+            GridCacheContext ctx = e.context();
+
+            Collection<ClusterNode> cacheNodes0 = ctx.discovery().cacheAffinityNodes(ctx.name(), expVer);
+            Collection<ClusterNode> cacheNodes1 = ctx.discovery().cacheAffinityNodes(ctx.name(), curVer);
+
+            if (!cacheNodes0.equals(cacheNodes1))
+                return true;
+        }
+
+        return false;
+    }
+
+    /**
      * @param nodeId Node ID.
      * @param res Response.
      */
@@ -1024,8 +1110,10 @@ public class IgniteTxHandler {
                     return null;
                 }
             }
-            else
+            else {
+                tx.writeVersion(req.writeVersion());
                 tx.transactionNodes(req.transactionNodes());
+            }
 
             if (!tx.isSystemInvalidate() && !F.isEmpty(req.writes())) {
                 int idx = 0;


[19/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index db3d350..ed8e573 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -198,19 +198,14 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 // Remap regular mappings.
                 final Buffer buf = bufMappings.remove(id);
 
+                // Only async notification is possible since
+                // discovery thread may be trapped otherwise.
                 if (buf != null) {
-                    // Only async notification is possible since
-                    // discovery thread may be trapped otherwise.
-                    ctx.closure().callLocalSafe(
-                        new Callable<Object>() {
-                            @Override public Object call() throws Exception {
-                                buf.onNodeLeft();
-
-                                return null;
-                            }
-                        },
-                        true /* system pool */
-                    );
+                    waitAffinityAndRun(new Runnable() {
+                        @Override public void run() {
+                            buf.onNodeLeft();
+                        }
+                    }, discoEvt.topologyVersion(), true);
                 }
             }
         };
@@ -248,6 +243,31 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     * @param c Closure to run.
+     * @param topVer Topology version to wait for.
+     * @param async Async flag.
+     */
+    private void waitAffinityAndRun(final Runnable c, long topVer, boolean async) {
+        AffinityTopologyVersion topVer0 = new AffinityTopologyVersion(topVer, 0);
+
+        IgniteInternalFuture<?> fut = ctx.cache().context().exchange().affinityReadyFuture(topVer0);
+
+        if (fut != null && !fut.isDone()) {
+            fut.listen(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
+                    ctx.closure().runLocalSafe(c, true);
+                }
+            });
+        }
+        else {
+            if (async)
+                ctx.closure().runLocalSafe(c, true);
+            else
+                c.run();
+        }
+    }
+
+    /**
      * @return Cache object context.
      */
     public CacheObjectContext cacheObjectContext() {
@@ -527,6 +547,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         boolean initPda = ctx.deploy().enabled() && jobPda == null;
 
+        AffinityTopologyVersion topVer = ctx.cache().context().exchange().readyAffinityVersion();
+
         for (DataStreamerEntry entry : entries) {
             List<ClusterNode> nodes;
 
@@ -543,7 +565,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     initPda = false;
                 }
 
-                nodes = nodes(key);
+                nodes = nodes(key, topVer);
             }
             catch (IgniteCheckedException e) {
                 resFut.onDone(e);
@@ -621,10 +643,10 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 }
             };
 
-            GridFutureAdapter<?> f;
+            final GridFutureAdapter<?> f;
 
             try {
-                f = buf.update(entriesForNode, lsnr);
+                f = buf.update(entriesForNode, topVer, lsnr);
             }
             catch (IgniteInterruptedCheckedException e1) {
                 resFut.onDone(e1);
@@ -633,30 +655,38 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (ctx.discovery().node(nodeId) == null) {
-                if (bufMappings.remove(nodeId, buf))
-                    buf.onNodeLeft();
+                if (bufMappings.remove(nodeId, buf)) {
+                    final Buffer buf0 = buf;
+
+                    waitAffinityAndRun(new Runnable() {
+                        @Override public void run() {
+                            buf0.onNodeLeft();
 
-                if (f != null)
-                    f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
-                        "(node has left): " + nodeId));
+                            if (f != null)
+                                f.onDone(new ClusterTopologyCheckedException("Failed to wait for request completion " +
+                                    "(node has left): " + nodeId));
+                        }
+                    }, ctx.discovery().topologyVersion(), false);
+                }
             }
         }
     }
 
     /**
      * @param key Key to map.
+     * @param topVer Topology version.
      * @return Nodes to send requests to.
      * @throws IgniteCheckedException If failed.
      */
-    private List<ClusterNode> nodes(KeyCacheObject key) throws IgniteCheckedException {
+    private List<ClusterNode> nodes(KeyCacheObject key, AffinityTopologyVersion topVer) throws IgniteCheckedException {
         GridAffinityProcessor aff = ctx.affinity();
 
         List<ClusterNode> res = null;
 
         if (!allowOverwrite())
-            res = aff.mapKeyToPrimaryAndBackups(cacheName, key);
+            res = aff.mapKeyToPrimaryAndBackups(cacheName, key, topVer);
         else {
-            ClusterNode node = aff.mapKeyToNode(cacheName, key);
+            ClusterNode node = aff.mapKeyToNode(cacheName, key, topVer);
 
             if (node != null)
                 res = Collections.singletonList(node);
@@ -959,11 +989,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /**
          * @param newEntries Infos.
+         * @param topVer Topology version.
          * @param lsnr Listener for the operation future.
          * @throws IgniteInterruptedCheckedException If failed.
          * @return Future for operation.
          */
         @Nullable GridFutureAdapter<?> update(Iterable<DataStreamerEntry> newEntries,
+            AffinityTopologyVersion topVer,
             IgniteInClosure<IgniteInternalFuture<?>> lsnr) throws IgniteInterruptedCheckedException {
             List<DataStreamerEntry> entries0 = null;
             GridFutureAdapter<Object> curFut0;
@@ -986,7 +1018,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (entries0 != null) {
-                submit(entries0, curFut0);
+                submit(entries0, topVer, curFut0);
 
                 if (cancelled)
                     curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + DataStreamerImpl.this));
@@ -1023,7 +1055,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
 
             if (entries0 != null)
-                submit(entries0, curFut0);
+                submit(entries0, null, curFut0);
 
             // Create compound future for this flush.
             GridCompoundFuture<Object, Object> res = null;
@@ -1068,10 +1100,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         /**
          * @param entries Entries to submit.
+         * @param topVer Topology version.
          * @param curFut Current future.
          * @throws IgniteInterruptedCheckedException If interrupted.
          */
-        private void submit(final Collection<DataStreamerEntry> entries, final GridFutureAdapter<Object> curFut)
+        private void submit(final Collection<DataStreamerEntry> entries,
+            @Nullable AffinityTopologyVersion topVer,
+            final GridFutureAdapter<Object> curFut)
             throws IgniteInterruptedCheckedException {
             assert entries != null;
             assert !entries.isEmpty();
@@ -1160,6 +1195,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
                 reqs.put(reqId, (GridFutureAdapter<Object>)fut);
 
+                if (topVer == null)
+                    topVer = ctx.cache().context().exchange().readyAffinityVersion();
+
                 DataStreamerRequest req = new DataStreamerRequest(
                     reqId,
                     topicBytes,
@@ -1174,7 +1212,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     dep != null ? dep.participants() : null,
                     dep != null ? dep.classLoaderId() : null,
                     dep == null,
-                    ctx.cache().context().exchange().readyAffinityVersion());
+                    topVer);
 
                 try {
                     ctx.io().send(node, TOPIC_DATASTREAM, req, PUBLIC_POOL);
@@ -1422,6 +1460,8 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     cctx.evicts().touch(entry, topVer);
 
                     CU.unwindEvicts(cctx);
+
+                    entry.onUnlock();
                 }
                 catch (GridDhtInvalidPartitionException | GridCacheEntryRemovedException ignored) {
                     // No-op.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 72911af..aa3bfe2 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.datastructures;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
@@ -32,6 +33,7 @@ import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
+import javax.cache.event.*;
 import javax.cache.processor.*;
 import java.io.*;
 import java.util.*;
@@ -40,7 +42,6 @@ import java.util.concurrent.*;
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
 import static org.apache.ignite.cache.CacheRebalanceMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-import static org.apache.ignite.internal.processors.cache.GridCacheOperation.*;
 import static org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor.DataStructureType.*;
 import static org.apache.ignite.transactions.TransactionConcurrency.*;
 import static org.apache.ignite.transactions.TransactionIsolation.*;
@@ -99,6 +100,9 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     /** */
     private IgniteInternalCache<CacheDataStructuresCacheKey, List<CacheCollectionInfo>> utilityDataCache;
 
+    /** */
+    private volatile UUID qryId;
+
     /**
      * @param ctx Context.
      */
@@ -112,7 +116,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void onKernalStart() {
+    @Override public void onKernalStart() throws IgniteCheckedException {
         if (ctx.config().isDaemon())
             return;
 
@@ -139,11 +143,35 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
             seqView = atomicsCache;
 
-            dsCacheCtx = ctx.cache().internalCache(CU.ATOMICS_CACHE_NAME).context();
+            dsCacheCtx = atomicsCache.context();
         }
     }
 
     /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startQuery() throws IgniteCheckedException {
+        if (qryId == null) {
+            synchronized (this) {
+                if (qryId == null) {
+                    qryId = dsCacheCtx.continuousQueries().executeInternalQuery(new DataStructuresEntryListener(),
+                        new DataStructuresEntryFilter(),
+                        dsCacheCtx.isReplicated() && dsCacheCtx.affinityNode(),
+                        false);
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        super.onKernalStop(cancel);
+
+        if (qryId != null)
+            dsCacheCtx.continuousQueries().cancelInternalQuery(qryId);
+    }
+
+    /**
      * Gets a sequence from cache or creates one if it's not cached.
      *
      * @param name Sequence name.
@@ -161,6 +189,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicSequence>() {
             @Override public IgniteAtomicSequence applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -287,6 +317,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicLong>() {
             @Override public IgniteAtomicLong applyx() throws IgniteCheckedException {
                 final GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -490,6 +522,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicReference>() {
             @Override public IgniteAtomicReference<T> applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -591,6 +625,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteAtomicStamped>() {
             @Override public IgniteAtomicStamped<T, S> applyx() throws IgniteCheckedException {
                 GridCacheInternalKeyImpl key = new GridCacheInternalKeyImpl(name);
@@ -899,6 +935,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
 
         checkAtomicsConfiguration();
 
+        startQuery();
+
         return getAtomic(new IgniteOutClosureX<IgniteCountDownLatch>() {
             @Override public IgniteCountDownLatch applyx() throws IgniteCheckedException {
                 GridCacheInternalKey key = new GridCacheInternalKeyImpl(name);
@@ -906,8 +944,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                 dsCacheCtx.gate().enter();
 
                 try (IgniteInternalTx tx = CU.txStartInternal(dsCacheCtx, dsView, PESSIMISTIC, REPEATABLE_READ)) {
-                    GridCacheCountDownLatchValue val = cast(dsView.get(key),
-                        GridCacheCountDownLatchValue.class);
+                    GridCacheCountDownLatchValue val = cast(dsView.get(key), GridCacheCountDownLatchValue.class);
 
                     // Check that count down hasn't been created in other thread yet.
                     GridCacheCountDownLatchEx latch = cast(dsMap.get(key), GridCacheCountDownLatchEx.class);
@@ -1034,28 +1071,46 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Transaction committed callback for transaction manager.
      *
-     * @param tx Committed transaction.
      */
-    public <K, V> void onTxCommitted(IgniteInternalTx tx) {
-        if (dsCacheCtx == null)
-            return;
+    static class DataStructuresEntryFilter implements CacheEntryEventSerializableFilter<Object, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
 
-        if (!dsCacheCtx.isDht() && tx.internal() && (!dsCacheCtx.isColocated() || dsCacheCtx.isReplicated())) {
-            Collection<IgniteTxEntry> entries = tx.writeEntries();
+        /** {@inheritDoc} */
+        @Override public boolean evaluate(CacheEntryEvent<?, ?> evt) throws CacheEntryListenerException {
+            if (evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED)
+                return evt.getValue() instanceof GridCacheCountDownLatchValue;
+            else {
+                assert evt.getEventType() == EventType.REMOVED : evt;
 
-            if (log.isDebugEnabled())
-                log.debug("Committed entries: " + entries);
+                return true;
+            }
+        }
 
-            for (IgniteTxEntry entry : entries) {
-                // Check updated or created GridCacheInternalKey keys.
-                if ((entry.op() == CREATE || entry.op() == UPDATE) && entry.key().internal()) {
-                    GridCacheInternal key = entry.key().value(entry.context().cacheObjectContext(), false);
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DataStructuresEntryFilter.class, this);
+        }
+    }
 
-                    Object val0 = CU.value(entry.value(), entry.context(), false);
+    /**
+     *
+     */
+    private class DataStructuresEntryListener implements
+        CacheEntryUpdatedListener<GridCacheInternalKey, GridCacheInternal> {
+        /** {@inheritDoc} */
+        @Override public void onUpdated(
+            Iterable<CacheEntryEvent<? extends GridCacheInternalKey, ? extends GridCacheInternal>> evts)
+            throws CacheEntryListenerException
+        {
+            for (CacheEntryEvent<? extends GridCacheInternalKey, ? extends GridCacheInternal> evt : evts) {
+                if (evt.getEventType() == EventType.CREATED || evt.getEventType() == EventType.UPDATED) {
+                    GridCacheInternal val0 = evt.getValue();
 
                     if (val0 instanceof GridCacheCountDownLatchValue) {
+                        GridCacheInternalKey key = evt.getKey();
+
                         // Notify latch on changes.
                         GridCacheRemovable latch = dsMap.get(key);
 
@@ -1067,8 +1122,6 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                             latch0.onUpdate(val.get());
 
                             if (val.get() == 0 && val.autoDelete()) {
-                                entry.cached().markObsolete(dsCacheCtx.versions().next());
-
                                 dsMap.remove(key);
 
                                 latch.onRemoved();
@@ -1080,11 +1133,12 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                                 ", actual=" + latch.getClass() + ", value=" + latch + ']');
                         }
                     }
+
                 }
+                else {
+                    assert evt.getEventType() == EventType.REMOVED : evt;
 
-                // Check deleted GridCacheInternal keys.
-                if (entry.op() == DELETE && entry.key().internal()) {
-                    GridCacheInternal key = entry.key().value(entry.context().cacheObjectContext(), false);
+                    GridCacheInternal key = evt.getKey();
 
                     // Entry's val is null if entry deleted.
                     GridCacheRemovable obj = dsMap.remove(key);
@@ -1094,6 +1148,11 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                 }
             }
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DataStructuresEntryListener.class, this);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
index 65cb48d..5fd6c81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJob.java
@@ -98,5 +98,5 @@ public interface HadoopJob {
     /**
      * Cleans up the job staging directory.
      */
-    void cleanupStagingDirectory();
+    public void cleanupStagingDirectory();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
index 371fd81..3d2ee17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
@@ -21,13 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Task context.
  */
 public abstract class HadoopTaskContext {
     /** */
-    private final HadoopJob job;
+    protected final HadoopJob job;
 
     /** */
     private HadoopTaskInput input;
@@ -187,4 +188,15 @@ public abstract class HadoopTaskContext {
      * @throws IgniteCheckedException If failed.
      */
     public abstract void cleanupTaskEnvironment() throws IgniteCheckedException;
+
+    /**
+     * Executes a callable on behalf of the job owner.
+     * In case of embedded task execution the implementation of this method
+     * will use classes loaded by the ClassLoader this HadoopTaskContext loaded with.
+     * @param c The callable.
+     * @param <T> The return type of the Callable.
+     * @return The result of the callable.
+     * @throws IgniteCheckedException On any error in callable.
+     */
+    public abstract <T> T runAsJobOwner(Callable<T> c) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
index 7c1a837..361f75f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsEx.java
@@ -48,8 +48,12 @@ public interface IgfsEx extends IgniteFileSystem {
     /** Property name for URI of file system. */
     public static final String SECONDARY_FS_URI = "SECONDARY_FS_URI";
 
-    /** Property name for user name of file system. */
-    public static final String SECONDARY_FS_USER_NAME = "SECONDARY_FS_USER_NAME";
+    /** Property name for default user name of file system.
+     * NOTE: for secondary file system this is just a default user name, which is used
+     * when the 2ndary filesystem is used outside of any user context.
+     * If another user name is set in the context, 2ndary file system will work on behalf
+     * of that user, which is different from the default. */
+     public static final String SECONDARY_FS_USER_NAME = "SECONDARY_FS_USER_NAME";
 
     /**
      * Stops IGFS cleaning all used resources.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 34636d2..c3495e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -245,8 +245,12 @@ public final class IgfsImpl implements IgfsEx {
             for (IgfsFileWorkerBatch batch : workerMap.values())
                 batch.cancel();
 
-            if (secondaryFs instanceof AutoCloseable)
-                U.closeQuiet((AutoCloseable)secondaryFs);
+            try {
+                secondaryFs.close();
+            }
+            catch (Exception e) {
+                log.error("Failed to close secondary file system.", e);
+            }
         }
 
         igfsCtx.kernalContext().io().removeMessageListener(topic, delMsgLsnr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
index 8a8b858..cfe6ed4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsIpcHandler.java
@@ -51,10 +51,10 @@ class IgfsIpcHandler implements IgfsServerHandler {
     private final int bufSize; // Buffer size. Must not be less then file block size.
 
     /** IGFS instance for this handler. */
-    private IgfsEx igfs;
+    private final IgfsEx igfs;
 
     /** Resource ID generator. */
-    private AtomicLong rsrcIdGen = new AtomicLong();
+    private final AtomicLong rsrcIdGen = new AtomicLong();
 
     /** Stopping flag. */
     private volatile boolean stopping;
@@ -241,138 +241,148 @@ class IgfsIpcHandler implements IgfsServerHandler {
      * @return Response message.
      * @throws IgniteCheckedException If failed.
      */
-    private IgfsMessage processPathControlRequest(IgfsClientSession ses, IgfsIpcCommand cmd,
+    private IgfsMessage processPathControlRequest(final IgfsClientSession ses, final IgfsIpcCommand cmd,
         IgfsMessage msg) throws IgniteCheckedException {
-        IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
+        final IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
 
         if (log.isDebugEnabled())
             log.debug("Processing path control request [igfsName=" + igfs.name() + ", req=" + req + ']');
 
-        IgfsControlResponse res = new IgfsControlResponse();
+        final IgfsControlResponse res = new IgfsControlResponse();
+
+        final String userName = req.userName();
+
+        assert userName != null;
 
         try {
-            switch (cmd) {
-                case EXISTS:
-                    res.response(igfs.exists(req.path()));
+            IgfsUserContext.doAs(userName, new IgniteOutClosure<Object>() {
+                @Override public Void apply() {
+                    switch (cmd) {
+                        case EXISTS:
+                            res.response(igfs.exists(req.path()));
 
-                    break;
+                            break;
 
-                case INFO:
-                    res.response(igfs.info(req.path()));
+                        case INFO:
+                            res.response(igfs.info(req.path()));
 
-                    break;
+                            break;
 
-                case PATH_SUMMARY:
-                    res.response(igfs.summary(req.path()));
+                        case PATH_SUMMARY:
+                            res.response(igfs.summary(req.path()));
 
-                    break;
+                            break;
 
-                case UPDATE:
-                    res.response(igfs.update(req.path(), req.properties()));
+                        case UPDATE:
+                            res.response(igfs.update(req.path(), req.properties()));
 
-                    break;
+                            break;
 
-                case RENAME:
-                    igfs.rename(req.path(), req.destinationPath());
+                        case RENAME:
+                            igfs.rename(req.path(), req.destinationPath());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case DELETE:
-                    res.response(igfs.delete(req.path(), req.flag()));
+                        case DELETE:
+                            res.response(igfs.delete(req.path(), req.flag()));
 
-                    break;
+                            break;
 
-                case MAKE_DIRECTORIES:
-                    igfs.mkdirs(req.path(), req.properties());
+                        case MAKE_DIRECTORIES:
+                            igfs.mkdirs(req.path(), req.properties());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case LIST_PATHS:
-                    res.paths(igfs.listPaths(req.path()));
+                        case LIST_PATHS:
+                            res.paths(igfs.listPaths(req.path()));
 
-                    break;
+                            break;
 
-                case LIST_FILES:
-                    res.files(igfs.listFiles(req.path()));
+                        case LIST_FILES:
+                            res.files(igfs.listFiles(req.path()));
 
-                    break;
+                            break;
 
-                case SET_TIMES:
-                    igfs.setTimes(req.path(), req.accessTime(), req.modificationTime());
+                        case SET_TIMES:
+                            igfs.setTimes(req.path(), req.accessTime(), req.modificationTime());
 
-                    res.response(true);
+                            res.response(true);
 
-                    break;
+                            break;
 
-                case AFFINITY:
-                    res.locations(igfs.affinity(req.path(), req.start(), req.length()));
+                        case AFFINITY:
+                            res.locations(igfs.affinity(req.path(), req.start(), req.length()));
 
-                    break;
+                            break;
 
-                case OPEN_READ: {
-                    IgfsInputStreamAdapter igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) :
-                        igfs.open(req.path(), bufSize, req.sequentialReadsBeforePrefetch());
+                        case OPEN_READ: {
+                            IgfsInputStreamAdapter igfsIn = !req.flag() ? igfs.open(req.path(), bufSize) :
+                                igfs.open(req.path(), bufSize, req.sequentialReadsBeforePrefetch());
 
-                    long streamId = registerResource(ses, igfsIn);
+                            long streamId = registerResource(ses, igfsIn);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS input stream for file read [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    IgfsFileInfo info = new IgfsFileInfo(igfsIn.fileInfo(), null,
-                        igfsIn.fileInfo().modificationTime());
+                            IgfsFileInfo info = new IgfsFileInfo(igfsIn.fileInfo(), null,
+                                igfsIn.fileInfo().modificationTime());
 
-                    res.response(new IgfsInputStreamDescriptor(streamId, info.length()));
+                            res.response(new IgfsInputStreamDescriptor(streamId, info.length()));
 
-                    break;
-                }
+                            break;
+                        }
 
-                case OPEN_CREATE: {
-                    long streamId = registerResource(ses, igfs.create(
-                        req.path(),       // Path.
-                        bufSize,          // Buffer size.
-                        req.flag(),       // Overwrite if exists.
-                        affinityKey(req), // Affinity key based on replication factor.
-                        req.replication(),// Replication factor.
-                        req.blockSize(),  // Block size.
-                        req.properties()  // File properties.
-                    ));
+                        case OPEN_CREATE: {
+                            long streamId = registerResource(ses, igfs.create(
+                                req.path(),       // Path.
+                                bufSize,          // Buffer size.
+                                req.flag(),       // Overwrite if exists.
+                                affinityKey(req), // Affinity key based on replication factor.
+                                req.replication(),// Replication factor.
+                                req.blockSize(),  // Block size.
+                                req.properties()  // File properties.
+                            ));
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS output stream for file create [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS output stream for file create [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    res.response(streamId);
+                            res.response(streamId);
 
-                    break;
-                }
+                            break;
+                        }
 
-                case OPEN_APPEND: {
-                    long streamId = registerResource(ses, igfs.append(
-                        req.path(),        // Path.
-                        bufSize,           // Buffer size.
-                        req.flag(),        // Create if absent.
-                        req.properties()   // File properties.
-                    ));
+                        case OPEN_APPEND: {
+                            long streamId = registerResource(ses, igfs.append(
+                                req.path(),        // Path.
+                                bufSize,           // Buffer size.
+                                req.flag(),        // Create if absent.
+                                req.properties()   // File properties.
+                            ));
 
-                    if (log.isDebugEnabled())
-                        log.debug("Opened IGFS output stream for file append [igfsName=" + igfs.name() + ", path=" +
-                            req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Opened IGFS output stream for file append [igfsName=" + igfs.name() + ", path=" +
+                                    req.path() + ", streamId=" + streamId + ", ses=" + ses + ']');
 
-                    res.response(streamId);
+                            res.response(streamId);
 
-                    break;
-                }
+                            break;
+                        }
 
-                default:
-                    assert false : "Unhandled path control request command: " + cmd;
+                        default:
+                            assert false : "Unhandled path control request command: " + cmd;
 
-                    break;
-            }
+                            break;
+                    }
+
+                    return null;
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index e33e0d4..b98c5d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -669,7 +669,7 @@ public class IgfsMetaManager extends IgfsManager {
     private Map<String, IgfsListingEntry> directoryListing(IgniteUuid fileId, boolean skipTx) throws IgniteCheckedException {
         assert fileId != null;
 
-        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singletonList(fileId)).get(fileId) :
+        IgfsFileInfo info = skipTx ? id2InfoPrj.getAllOutTx(Collections.singleton(fileId)).get(fileId) :
             id2InfoPrj.get(fileId);
 
         return info == null ? Collections.<String, IgfsListingEntry>emptyMap() : info.listing();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
index 683b317..44ee90f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsSecondaryFileSystemImpl.java
@@ -30,14 +30,14 @@ import java.util.*;
  */
 class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
     /** Delegate. */
-    private final IgfsImpl igfs;
+    private final IgfsEx igfs;
 
     /**
      * Constructor.
      *
      * @param igfs Delegate.
      */
-    IgfsSecondaryFileSystemImpl(IgfsImpl igfs) {
+    IgfsSecondaryFileSystemImpl(IgfsEx igfs) {
         this.igfs = igfs;
     }
 
@@ -118,4 +118,9 @@ class IgfsSecondaryFileSystemImpl implements IgfsSecondaryFileSystem {
     @Override public Map<String, String> properties() {
         return Collections.emptyMap();
     }
+
+    /** {@inheritDoc} */
+    @Override public void close() throws IgniteException {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
index 253d5be..caa6866 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsServer.java
@@ -239,13 +239,13 @@ public class IgfsServer {
      */
     private class ClientWorker extends GridWorker {
         /** Connected client endpoint. */
-        private IpcEndpoint endpoint;
+        private final IpcEndpoint endpoint;
 
         /** Data output stream. */
         private final IgfsDataOutputStream out;
 
         /** Client session object. */
-        private IgfsClientSession ses;
+        private final IgfsClientSession ses;
 
         /** Queue node for fast unlink. */
         private ConcurrentLinkedDeque8.Node<ClientWorker> node;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
index 4b0234f..8026a44 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsUtils.java
@@ -18,9 +18,11 @@
 package org.apache.ignite.internal.processors.igfs;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.jetbrains.annotations.*;
 
 import java.lang.reflect.*;
 
@@ -88,4 +90,18 @@ public class IgfsUtils {
     private IgfsUtils() {
         // No-op.
     }
+
+    /**
+     * Provides non-null user name.
+     * If the user name is null or empty string, defaults to {@link FileSystemConfiguration#DFLT_USER_NAME},
+     * which is the current process owner user.
+     * @param user a user name to be fixed.
+     * @return non-null interned user name.
+     */
+    public static String fixUserName(@Nullable String user) {
+        if (F.isEmpty(user))
+           user = FileSystemConfiguration.DFLT_USER_NAME;
+
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 cd4d543..ed8e1e2 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
@@ -121,7 +121,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     if (F.isEmpty(meta.getValueType()))
                         throw new IgniteCheckedException("Value type is not set: " + meta);
 
-                    TypeDescriptor desc = new TypeDescriptor(ccfg);
+                    TypeDescriptor desc = new TypeDescriptor();
 
                     Class<?> valCls = U.classForName(meta.getValueType(), null);
 
@@ -160,7 +160,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     Class<?> keyCls = clss[i];
                     Class<?> valCls = clss[i + 1];
 
-                    TypeDescriptor desc = processKeyAndValueClasses(ccfg, keyCls, valCls);
+                    TypeDescriptor desc = processKeyAndValueClasses(keyCls, valCls);
 
                     addTypeByName(ccfg, desc);
                     types.put(new TypeId(ccfg.getName(), valCls), desc);
@@ -188,15 +188,17 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param ccfg Cache configuration.
      * @param keyCls Key class.
      * @param valCls Value class.
      * @return Type descriptor.
      * @throws IgniteCheckedException If failed.
      */
-    private TypeDescriptor processKeyAndValueClasses(CacheConfiguration<?,?> ccfg, Class<?> keyCls, Class<?> valCls)
+    private TypeDescriptor processKeyAndValueClasses(
+        Class<?> keyCls,
+        Class<?> valCls
+    )
         throws IgniteCheckedException {
-        TypeDescriptor d = new TypeDescriptor(ccfg);
+        TypeDescriptor d = new TypeDescriptor();
 
         d.keyClass(keyCls);
         d.valueClass(valCls);
@@ -318,7 +320,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
 
         try {
-            return rebuildIndexes(space, typesByName.get(new TypeName(space, valTypeName)));
+            return rebuildIndexes(
+                space,
+                typesByName.get(
+                    new TypeName(
+                        space,
+                        valTypeName)));
         }
         finally {
             busyLock.leaveBusy();
@@ -539,7 +546,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            return idx.queryTwoStep(ctx.cache().internalCache(space).context(), qry);
+            return idx.queryTwoStep(
+                ctx.cache().internalCache(space).context(),
+                qry);
         }
         finally {
             busyLock.leaveBusy();
@@ -589,59 +598,62 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Cursor.
      */
-    public <K,V> Iterator<Cache.Entry<K,V>> queryLocal(GridCacheContext<?,?> cctx, SqlQuery qry) {
+    public <K, V> Iterator<Cache.Entry<K, V>> queryLocal(final GridCacheContext<?, ?> cctx, final SqlQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            String space = cctx.name();
-            String type = qry.getType();
-            String sqlQry = qry.getSql();
-            Object[] params = qry.getArgs();
-
-            TypeDescriptor typeDesc = typesByName.get(new TypeName(space, type));
-
-            if (typeDesc == null || !typeDesc.registered())
-                throw new CacheException("Failed to find SQL table for type: " + type);
-
-            final GridCloseableIterator<IgniteBiTuple<K,V>> i = idx.query(space, sqlQry, F.asList(params), typeDesc,
-                idx.backupFilter());
-
-            if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
-                ctx.event().record(new CacheQueryExecutedEvent<>(
-                    ctx.discovery().localNode(),
-                    "SQL query executed.",
-                    EVT_CACHE_QUERY_EXECUTED,
-                    CacheQueryType.SQL.name(),
-                    null,
-                    null,
-                    sqlQry,
-                    null,
-                    null,
-                    params,
-                    null,
-                    null));
-            }
-
-            return new ClIter<Cache.Entry<K,V>>() {
-                @Override public void close() throws Exception {
-                    i.close();
-                }
-
-                @Override public boolean hasNext() {
-                    return i.hasNext();
-                }
-
-                @Override public Cache.Entry<K,V> next() {
-                    IgniteBiTuple<K,V> t = i.next();
-
-                    return new CacheEntryImpl<>(t.getKey(), t.getValue());
-                }
-
-                @Override public void remove() {
-                    throw new UnsupportedOperationException();
-                }
-            };
+            return executeQuery(
+                cctx,
+                new IgniteOutClosureX<Iterator<Cache.Entry<K, V>>>() {
+                    @Override public Iterator<Cache.Entry<K, V>> applyx() throws IgniteCheckedException {
+                        String space = cctx.name();
+                        String type = qry.getType();
+                        String sqlQry = qry.getSql();
+                        Object[] params = qry.getArgs();
+
+                        TypeDescriptor typeDesc = typesByName.get(
+                            new TypeName(
+                                space,
+                                type));
+
+                        if (typeDesc == null || !typeDesc.registered())
+                            throw new CacheException("Failed to find SQL table for type: " + type);
+
+                        final GridCloseableIterator<IgniteBiTuple<K, V>> i = idx.query(
+                            space,
+                            sqlQry,
+                            F.asList(params),
+                            typeDesc,
+                            idx.backupFilter());
+
+                        sendQueryExecutedEvent(
+                            sqlQry,
+                            params);
+
+                        return new ClIter<Cache.Entry<K, V>>() {
+                            @Override public void close() throws Exception {
+                                i.close();
+                            }
+
+                            @Override public boolean hasNext() {
+                                return i.hasNext();
+                            }
+
+                            @Override public Cache.Entry<K, V> next() {
+                                IgniteBiTuple<K, V> t = i.next();
+
+                                return new CacheEntryImpl<>(
+                                    t.getKey(),
+                                    t.getValue());
+                            }
+
+                            @Override public void remove() {
+                                throw new UnsupportedOperationException();
+                            }
+                        };
+                    }
+                });
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);
@@ -652,6 +664,28 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param sqlQry Sql query.
+     * @param params Params.
+     */
+    private void sendQueryExecutedEvent(String sqlQry, Object[] params) {
+        if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
+            ctx.event().record(new CacheQueryExecutedEvent<>(
+                ctx.discovery().localNode(),
+                "SQL query executed.",
+                EVT_CACHE_QUERY_EXECUTED,
+                CacheQueryType.SQL.name(),
+                null,
+                null,
+                sqlQry,
+                null,
+                null,
+                params,
+                null,
+                null));
+        }
+    }
+
+    /**
      * @return Message factory for {@link GridIoManager}.
      */
     public MessageFactory messageFactory() {
@@ -670,39 +704,29 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Iterator.
      */
-    public QueryCursor<List<?>> queryLocalFields(GridCacheContext<?,?> cctx, SqlFieldsQuery qry) {
+    public QueryCursor<List<?>> queryLocalFields(final GridCacheContext<?,?> cctx, final SqlFieldsQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
         try {
-            String space = cctx.name();
-            String sql = qry.getSql();
-            Object[] args = qry.getArgs();
-
-            GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
-
-            if (ctx.event().isRecordable(EVT_CACHE_QUERY_EXECUTED)) {
-                ctx.event().record(new CacheQueryExecutedEvent<>(
-                        ctx.discovery().localNode(),
-                        "SQL query executed.",
-                        EVT_CACHE_QUERY_EXECUTED,
-                        CacheQueryType.SQL.name(),
-                        null,
-                        null,
-                        sql,
-                        null,
-                        null,
-                        args,
-                        null,
-                        null));
-            }
+            return executeQuery(cctx, new IgniteOutClosureX<QueryCursor<List<?>>>() {
+                @Override public QueryCursor<List<?>> applyx() throws IgniteCheckedException {
+                    String space = cctx.name();
+                    String sql = qry.getSql();
+                    Object[] args = qry.getArgs();
 
-            QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
-                new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
+                    GridQueryFieldsResult res = idx.queryFields(space, sql, F.asList(args), idx.backupFilter());
 
-            cursor.fieldsMeta(res.metaData());
+                    sendQueryExecutedEvent(sql, args);
 
-            return cursor;
+                    QueryCursorImpl<List<?>> cursor = new QueryCursorImpl<>(
+                        new GridQueryCacheObjectsIterator(res.iterator(), cctx, cctx.keepPortable()));
+
+                    cursor.fieldsMeta(res.metaData());
+
+                    return cursor;
+                }
+            });
         }
         catch (IgniteCheckedException e) {
             throw new CacheException(e);
@@ -793,7 +817,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             if (type == null || !type.registered())
                 throw new CacheException("Failed to find SQL table for type: " + resType);
 
-            return idx.queryText(space, clause, type, filters);
+            return idx.queryText(
+                space,
+                clause,
+                type,
+                filters);
         }
         finally {
             busyLock.leaveBusy();
@@ -808,7 +836,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Field rows.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
+    public GridQueryFieldsResult queryFields(@Nullable String space, String clause, Collection<Object> params,
         IndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
@@ -837,7 +865,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (ctx.indexing().enabled()) {
             CacheObjectContext coctx = cacheObjectContext(spaceName);
 
-            ctx.indexing().onSwap(spaceName, key.value(coctx, false));
+            ctx.indexing().onSwap(
+                spaceName,
+                key.value(
+                    coctx,
+                    false));
         }
 
         if (idx == null)
@@ -847,7 +879,9 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             throw new IllegalStateException("Failed to process swap event (grid is stopping).");
 
         try {
-            idx.onSwap(spaceName, key);
+            idx.onSwap(
+                spaceName,
+                key);
         }
         finally {
             busyLock.leaveBusy();
@@ -1067,7 +1101,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         assert valCls != null;
 
         for (Map.Entry<String, Class<?>> entry : meta.getAscendingFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(keyCls, valCls, entry.getKey(), entry.getValue());
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                entry.getKey(),
+                entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1079,7 +1117,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getDescendingFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(keyCls, valCls, entry.getKey(), entry.getValue());
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                entry.getKey(),
+                entry.getValue());
 
             d.addProperty(prop, false);
 
@@ -1091,7 +1133,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (String txtIdx : meta.getTextFields()) {
-            ClassProperty prop = buildClassProperty(keyCls, valCls, txtIdx, String.class);
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                txtIdx,
+                String.class);
 
             d.addProperty(prop, false);
 
@@ -1109,7 +1155,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 int order = 0;
 
                 for (Map.Entry<String, IgniteBiTuple<Class<?>, Boolean>> idxField : idxFields.entrySet()) {
-                    ClassProperty prop = buildClassProperty(keyCls, valCls, idxField.getKey(), idxField.getValue().get1());
+                    ClassProperty prop = buildClassProperty(
+                        keyCls,
+                        valCls,
+                        idxField.getKey(),
+                        idxField.getValue().get1());
 
                     d.addProperty(prop, false);
 
@@ -1123,7 +1173,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         }
 
         for (Map.Entry<String, Class<?>> entry : meta.getQueryFields().entrySet()) {
-            ClassProperty prop = buildClassProperty(keyCls, valCls, entry.getKey(), entry.getValue());
+            ClassProperty prop = buildClassProperty(
+                keyCls,
+                valCls,
+                entry.getKey(),
+                entry.getValue());
 
             d.addProperty(prop, false);
         }
@@ -1231,7 +1285,11 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     private static ClassProperty buildClassProperty(Class<?> keyCls, Class<?> valCls, String pathStr, Class<?> resType)
         throws IgniteCheckedException {
-        ClassProperty res = buildClassProperty(true, keyCls, pathStr, resType);
+        ClassProperty res = buildClassProperty(
+            true,
+            keyCls,
+            pathStr,
+            resType);
 
         if (res == null) // We check key before value consistently with PortableProperty.
             res = buildClassProperty(false, valCls, pathStr, resType);
@@ -1330,6 +1388,59 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cctx Cache context.
+     * @param clo Closure.
+     */
+    private <R> R executeQuery(GridCacheContext<?,?> cctx, IgniteOutClosureX<R> clo)
+        throws IgniteCheckedException {
+        final long start = U.currentTimeMillis();
+
+        Throwable err = null;
+
+        R res = null;
+
+        try {
+            res = clo.apply();
+
+            return res;
+        }
+        catch (GridClosureException e) {
+            err = e.unwrap();
+
+            throw (IgniteCheckedException)err;
+        }
+        finally {
+            GridCacheQueryMetricsAdapter metrics = (GridCacheQueryMetricsAdapter)cctx.queries().metrics();
+
+            onExecuted(cctx, metrics, res, err, start, U.currentTimeMillis() - start, log);
+        }
+    }
+
+    /**
+     * @param cctx Cctx.
+     * @param metrics Metrics.
+     * @param res Result.
+     * @param err Err.
+     * @param startTime Start time.
+     * @param duration Duration.
+     * @param log Logger.
+     */
+    public static void onExecuted(GridCacheContext<?, ?> cctx, GridCacheQueryMetricsAdapter metrics,
+        Object res, Throwable err, long startTime, long duration, IgniteLogger log) {
+        boolean fail = err != null;
+
+        // Update own metrics.
+        metrics.onQueryExecute(duration, fail);
+
+        // Update metrics in query manager.
+        cctx.queries().onMetricsUpdate(duration, fail);
+
+        if (log.isTraceEnabled())
+            log.trace("Query execution finished [startTime=" + startTime +
+                    ", duration=" + duration + ", fail=" + (err != null) + ", res=" + res + ']');
+    }
+
+    /**
      *
      */
     private abstract static class Property {
@@ -1538,9 +1649,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      */
     private static class TypeDescriptor implements GridQueryTypeDescriptor {
         /** */
-        private CacheConfiguration<?,?> ccfg;
-
-        /** */
         private String name;
 
         /** Value field names and types with preserved order. */
@@ -1571,13 +1679,6 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         private boolean registered;
 
         /**
-         * @param ccfg Cache configuration.
-         */
-        private TypeDescriptor(CacheConfiguration<?,?> ccfg) {
-            this.ccfg = ccfg;
-        }
-
-        /**
          * @return {@code True} if type registration in SPI was finished and type was not rejected.
          */
         boolean registered() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 22d1ff0..64eb1c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
@@ -69,7 +70,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     private final ConcurrentMap<String, GridFutureAdapter<?>> undepFuts = new ConcurrentHashMap8<>();
 
     /** Deployment executor service. */
-    private final ExecutorService depExe = Executors.newSingleThreadExecutor();
+    private final ExecutorService depExe;
 
     /** Busy lock. */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
@@ -97,6 +98,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     public GridServiceProcessor(GridKernalContext ctx) {
         super(ctx);
+
+        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.gridName(), "srvc-deploy"));
     }
 
     /** {@inheritDoc} */
@@ -128,10 +131,10 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 ctx.cache().context().deploy().ignoreOwnership(true);
 
             cfgQryId = cache.context().continuousQueries().executeInternalQuery(
-                new DeploymentListener(), null, true, true);
+                new DeploymentListener(), null, cache.context().affinityNode(), true);
 
             assignQryId = cache.context().continuousQueries().executeInternalQuery(
-                new AssignmentListener(), null, true, true);
+                new AssignmentListener(), null, cache.context().affinityNode(), true);
         }
         finally {
             if (ctx.deploy().enabled())
@@ -345,7 +348,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'));
                         }
                         else {
-                            for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                ServiceAssignmentsPredicate.INSTANCE);
+
+                            while (it.hasNext()) {
+                                Cache.Entry<Object, Object> e = it.next();
+
                                 if (e.getKey() instanceof GridServiceAssignmentsKey) {
                                     GridServiceAssignments assigns = (GridServiceAssignments)e.getValue();
 
@@ -437,7 +445,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> cancelAll() {
         Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -456,7 +468,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public Collection<ServiceDescriptor> serviceDescriptors() {
         Collection<ServiceDescriptor> descs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -904,6 +920,43 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param p Entry predicate used to execute query from client node.
+     * @return Service deployment entries.
+     */
+    @SuppressWarnings("unchecked")
+    private Iterator<Cache.Entry<Object, Object>> serviceEntries(IgniteBiPredicate<Object, Object> p) {
+        if (!cache.context().affinityNode()) {
+            ClusterNode oldestSrvNode =
+                CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
+
+            if (oldestSrvNode == null)
+                return F.emptyIterator();
+
+            GridCacheQueryManager qryMgr = cache.context().queries();
+
+            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, false);
+
+            qry.keepAll(false);
+
+            qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
+
+            return cache.context().itHolder().iterator(qry.execute(),
+                new CacheIteratorConverter<Object, Map.Entry<Object,Object>>() {
+                    @Override protected Object convert(Map.Entry<Object, Object> e) {
+                        return new CacheEntryImpl<>(e.getKey(), e.getValue());
+                    }
+
+                    @Override protected void remove(Object item) {
+                        throw new UnsupportedOperationException();
+                    }
+                }
+            );
+        }
+        else
+            return cache.entrySetx().iterator();
+    }
+
+    /**
      * Service deployment listener.
      */
     private class DeploymentListener implements CacheEntryUpdatedListener<Object, Object> {
@@ -1045,18 +1098,24 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             try {
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
-                        long topVer = ((DiscoveryEvent)evt).topologyVersion();
+                        AffinityTopologyVersion topVer =
+                            new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion());
 
-                        ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
+                        ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 
-                        if (oldest.isLocal()) {
+                        if (oldest != null && oldest.isLocal()) {
                             final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();
 
                             if (ctx.deploy().enabled())
                                 ctx.cache().context().deploy().ignoreOwnership(true);
 
                             try {
-                                for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                    ServiceDeploymentPredicate.INSTANCE);
+
+                                while (it.hasNext()) {
+                                    Cache.Entry<Object, Object> e = it.next();
+
                                     if (!(e.getKey() instanceof GridServiceDeploymentKey))
                                         continue;
 
@@ -1068,7 +1127,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity().
                                             affinityReadyFuture(topVer).get();
 
-                                        reassign(dep, topVer);
+                                        reassign(dep, topVer.topologyVersion());
                                     }
                                     catch (IgniteCheckedException ex) {
                                         if (!(e instanceof ClusterTopologyCheckedException))
@@ -1085,7 +1144,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                             }
 
                             if (!retries.isEmpty())
-                                onReassignmentFailed(topVer, retries);
+                                onReassignmentFailed(topVer.topologyVersion(), retries);
                         }
 
                         // Clean up zombie assignments.
@@ -1265,4 +1324,46 @@ public class GridServiceProcessor extends GridProcessorAdapter {
          */
         public abstract void run0();
     }
+
+    /**
+     *
+     */
+    static class ServiceDeploymentPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        static final ServiceDeploymentPredicate INSTANCE = new ServiceDeploymentPredicate();
+
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceDeploymentKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceDeploymentPredicate.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ServiceAssignmentsPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        static final ServiceAssignmentsPredicate INSTANCE = new ServiceAssignmentsPredicate();
+
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceAssignmentsKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceAssignmentsPredicate.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
new file mode 100644
index 0000000..a0fd9b4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridSpiTimeoutObject.java
@@ -0,0 +1,73 @@
+/*
+ * 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.timeout;
+
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+
+/**
+ * Wrapper for {@link IgniteSpiTimeoutObject}.
+ */
+public class GridSpiTimeoutObject implements GridTimeoutObject {
+    /** */
+    @GridToStringInclude
+    private final IgniteSpiTimeoutObject obj;
+
+    /**
+     * @param obj SPI object.
+     */
+    public GridSpiTimeoutObject(IgniteSpiTimeoutObject obj) {
+        this.obj = obj;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onTimeout() {
+        obj.onTimeout();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid timeoutId() {
+        return obj.id();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long endTime() {
+        return obj.endTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        assert false;
+
+        return super.hashCode();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object obj) {
+        assert false;
+
+        return super.equals(obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public final String toString() {
+        return S.toString(GridSpiTimeoutObject.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
index 81ff72b..e4f370c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
@@ -21,11 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
 
+import java.io.*;
 import java.util.*;
 
 /**
@@ -40,10 +43,12 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
         new GridConcurrentSkipListSet<>(new Comparator<GridTimeoutObject>() {
             /** {@inheritDoc} */
             @Override public int compare(GridTimeoutObject o1, GridTimeoutObject o2) {
-                long time1 = o1.endTime();
-                long time2 = o2.endTime();
+                int res = Long.compare(o1.endTime(), o2.endTime());
 
-                return time1 < time2 ? -1 : time1 > time2 ? 1 : o1.timeoutId().compareTo(o2.timeoutId());
+                if (res != 0)
+                    return res;
+
+                return o1.timeoutId().compareTo(o2.timeoutId());
             }
         });
 
@@ -98,6 +103,26 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Schedule the specified timer task for execution at the specified
+     * time with the specified period, in milliseconds.
+     *
+     * @param task Task to execute.
+     * @param delay Delay to first execution in milliseconds.
+     * @param period Period for execution in milliseconds or -1.
+     * @return Cancelable to cancel task.
+     */
+    public CancelableTask schedule(Runnable task, long delay, long period) {
+        assert delay >= 0 : delay;
+        assert period > 0 || period == -1 : period;
+
+        CancelableTask obj = new CancelableTask(task, U.currentTimeMillis() + delay, period);
+
+        addTimeoutObject(obj);
+
+        return obj;
+    }
+
+    /**
      * @param timeoutObj Timeout object.
      */
     public void removeTimeoutObject(GridTimeoutObject timeoutObj) {
@@ -173,4 +198,78 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
         X.println(">>> Timeout processor memory stats [grid=" + ctx.gridName() + ']');
         X.println(">>>   timeoutObjsSize: " + timeoutObjs.size());
     }
+
+    /**
+     *
+     */
+    public class CancelableTask implements GridTimeoutObject, Closeable {
+        /** */
+        private final IgniteUuid id = IgniteUuid.randomUuid();
+
+        /** */
+        private long endTime;
+
+        /** */
+        private final long period;
+
+        /** */
+        private volatile boolean cancel;
+
+        /** */
+        @GridToStringInclude
+        private final Runnable task;
+
+        /**
+         * @param task Task to execute.
+         * @param firstTime First time.
+         * @param period Period.
+         */
+        CancelableTask(Runnable task, long firstTime, long period) {
+            this.task = task;
+            endTime = firstTime;
+            this.period = period;
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteUuid timeoutId() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public long endTime() {
+            return endTime;
+        }
+
+        /** {@inheritDoc} */
+        @Override public synchronized void onTimeout() {
+            if (cancel)
+                return;
+
+            try {
+                task.run();
+            }
+            finally {
+                if (!cancel && period > 0) {
+                    endTime = U.currentTimeMillis() + period;
+
+                    addTimeoutObject(this);
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() {
+            cancel = true;
+
+            synchronized (this) {
+                // Just waiting for task execution end to make sure that task will not be executed anymore.
+                removeTimeoutObject(this);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CancelableTask.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
index 1d1e022..f8ee265 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/transactions/IgniteTxRollbackCheckedException.java
@@ -36,6 +36,15 @@ public class IgniteTxRollbackCheckedException extends IgniteCheckedException {
     }
 
     /**
+     * Creates new exception with given nested exception.
+     *
+     * @param cause Nested exception.
+     */
+    public IgniteTxRollbackCheckedException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
      * Creates new rollback exception with given error message and optional nested exception.
      *
      * @param msg Error message.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
index bff26ec..42fe089 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridJavaProcess.java
@@ -128,25 +128,31 @@ public final class GridJavaProcess {
         gjProc.log = log;
         gjProc.procKilledC = procKilledC;
 
-        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
-        String classpath = System.getProperty("java.class.path");
-        String sfcp = System.getProperty("surefire.test.class.path");
-
-        if (sfcp != null)
-            classpath += System.getProperty("path.separator") + sfcp;
-
-        if (cp != null)
-            classpath += System.getProperty("path.separator") + cp;
-
         List<String> procParams = params == null || params.isEmpty() ?
             Collections.<String>emptyList() : Arrays.asList(params.split(" "));
 
         List<String> procCommands = new ArrayList<>();
 
+        String javaBin = System.getProperty("java.home") + File.separator + "bin" + File.separator + "java";
+
         procCommands.add(javaBin);
         procCommands.addAll(jvmArgs == null ? U.jvmArgs() : jvmArgs);
-        procCommands.add("-cp");
-        procCommands.add(classpath);
+
+        if (!jvmArgs.contains("-cp") && !jvmArgs.contains("-classpath")) {
+            String classpath = System.getProperty("java.class.path");
+
+            String sfcp = System.getProperty("surefire.test.class.path");
+
+            if (sfcp != null)
+                classpath += System.getProperty("path.separator") + sfcp;
+
+            if (cp != null)
+                classpath += System.getProperty("path.separator") + cp;
+
+            procCommands.add("-cp");
+            procCommands.add(classpath);
+        }
+
         procCommands.add(clsName);
         procCommands.addAll(procParams);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index fb9ad29..f8caf22 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -241,8 +241,8 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
             lsnr.apply(this);
         }
         catch (IllegalStateException e) {
-            U.warn(null, "Failed to notify listener (is grid stopped?) [fut=" + this +
-                ", lsnr=" + lsnr + ", err=" + e.getMessage() + ']');
+            U.error(null, "Failed to notify listener (is grid stopped?) [fut=" + this +
+                ", lsnr=" + lsnr + ", err=" + e.getMessage() + ']', e);
         }
         catch (RuntimeException | Error e) {
             U.error(null, "Failed to notify listener: " + lsnr, e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
index 31396fb..693a5a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
@@ -38,58 +38,58 @@ public interface GridCommunicationClient {
      * @param handshakeC Handshake.
      * @throws IgniteCheckedException If handshake failed.
      */
-    void doHandshake(IgniteInClosure2X<InputStream, OutputStream> handshakeC) throws IgniteCheckedException;
+    public void doHandshake(IgniteInClosure2X<InputStream, OutputStream> handshakeC) throws IgniteCheckedException;
 
     /**
      * @return {@code True} if client has been closed by this call,
      *      {@code false} if failed to close client (due to concurrent reservation or concurrent close).
      */
-    boolean close();
+    public boolean close();
 
     /**
      * Forces client close.
      */
-    void forceClose();
+    public void forceClose();
 
     /**
      * @return {@code True} if client is closed;
      */
-    boolean closed();
+    public boolean closed();
 
     /**
      * @return {@code True} if client was reserved, {@code false} otherwise.
      */
-    boolean reserve();
+    public boolean reserve();
 
     /**
      * Releases this client by decreasing reservations.
      */
-    void release();
+    public void release();
 
     /**
      * @return {@code True} if client was reserved.
      */
-    boolean reserved();
+    public boolean reserved();
 
     /**
      * Gets idle time of this client.
      *
      * @return Idle time of this client.
      */
-    long getIdleTime();
+    public long getIdleTime();
 
     /**
      * @param data Data to send.
      * @throws IgniteCheckedException If failed.
      */
-    void sendMessage(ByteBuffer data) throws IgniteCheckedException;
+    public void sendMessage(ByteBuffer data) throws IgniteCheckedException;
 
     /**
      * @param data Data to send.
      * @param len Length.
      * @throws IgniteCheckedException If failed.
      */
-    void sendMessage(byte[] data, int len) throws IgniteCheckedException;
+    public void sendMessage(byte[] data, int len) throws IgniteCheckedException;
 
     /**
      * @param nodeId Node ID (provided only if versions of local and remote nodes are different).
@@ -97,16 +97,10 @@ public interface GridCommunicationClient {
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if should try to resend message.
      */
-    boolean sendMessage(@Nullable UUID nodeId, Message msg) throws IgniteCheckedException;
-
-    /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    void flushIfNeeded(long timeout) throws IOException;
+    public boolean sendMessage(@Nullable UUID nodeId, Message msg) throws IgniteCheckedException;
 
     /**
      * @return {@code True} if send is asynchronous.
      */
-    boolean async();
+    public boolean async();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
index 2b764ec..44ab4a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBuffer.java
@@ -85,7 +85,7 @@ public class GridNioDelimitedBuffer {
                         idx++;
                     }
                     else {
-                        pos = cnt - idx;
+                        pos = cnt - (i - pos) - 1;
 
                         idx = 0;
                     }


[23/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 26eef50..703daf9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -81,7 +81,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
     @Override public void start() throws IgniteCheckedException {
         super.start();
 
-        preldr = new GridDhtPreloader<>(ctx);
+        preldr = new GridDhtPreloader(ctx);
 
         preldr.start();
 
@@ -518,7 +518,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             return;
         }
 
-        // Group lock can be only started from local node, so we never start group lock transaction on remote node.
         IgniteInternalFuture<?> f = lockAllAsync(ctx, nearNode, req, null);
 
         // Register listener just so we print out errors.
@@ -534,8 +533,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
     private void processDhtLockResponse(UUID nodeId, GridDhtLockResponse res) {
         assert nodeId != null;
         assert res != null;
-        GridDhtLockFuture<K, V> fut = (GridDhtLockFuture<K, V>)ctx.mvcc().<Boolean>future(res.version(),
-            res.futureId());
+        GridDhtLockFuture fut = (GridDhtLockFuture)ctx.mvcc().<Boolean>future(res.version(), res.futureId());
 
         if (fut == null) {
             if (log.isDebugEnabled())
@@ -604,7 +602,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
         assert tx != null;
 
-        GridDhtLockFuture<K, V> fut = new GridDhtLockFuture<>(
+        GridDhtLockFuture fut = new GridDhtLockFuture(
             ctx,
             tx.nearNodeId(),
             tx.nearXidVersion(),
@@ -669,7 +667,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
      * @return Future.
      */
     public IgniteInternalFuture<GridNearLockResponse> lockAllAsync(
-        final GridCacheContext<K, V> cacheCtx,
+        final GridCacheContext<?, ?> cacheCtx,
         final ClusterNode nearNode,
         final GridNearLockRequest req,
         @Nullable final CacheEntryPredicate[] filter0) {
@@ -719,26 +717,57 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                         if (filter == null)
                             filter = req.filter();
 
-                        GridDhtLockFuture<K, V> fut = null;
+                        GridDhtLockFuture fut = null;
 
                         if (!req.inTx()) {
-                            fut = new GridDhtLockFuture<>(ctx,
-                                nearNode.id(),
-                                req.version(),
-                                req.topologyVersion(),
-                                cnt,
-                                req.txRead(),
-                                req.needReturnValue(),
-                                req.timeout(),
-                                tx,
-                                req.threadId(),
-                                req.accessTtl(),
-                                filter,
-                                req.skipStore());
+                            GridDhtPartitionTopology top = null;
+
+                            if (req.firstClientRequest()) {
+                                assert CU.clientNode(nearNode);
+
+                                top = topology();
+
+                                topology().readLock();
+                            }
+
+                            try {
+                                if (top != null && needRemap(req.topologyVersion(), top.topologyVersion())) {
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("Client topology version mismatch, need remap lock request [" +
+                                            "reqTopVer=" + req.topologyVersion() +
+                                            ", locTopVer=" + top.topologyVersion() +
+                                            ", req=" + req + ']');
+                                    }
+
+                                    GridNearLockResponse res = sendClientLockRemapResponse(nearNode,
+                                        req,
+                                        top.topologyVersion());
+
+                                    return new GridFinishedFuture<>(res);
+                                }
+
+                                fut = new GridDhtLockFuture(ctx,
+                                    nearNode.id(),
+                                    req.version(),
+                                    req.topologyVersion(),
+                                    cnt,
+                                    req.txRead(),
+                                    req.needReturnValue(),
+                                    req.timeout(),
+                                    tx,
+                                    req.threadId(),
+                                    req.accessTtl(),
+                                    filter,
+                                    req.skipStore());
 
-                            // Add before mapping.
-                            if (!ctx.mvcc().addFuture(fut))
-                                throw new IllegalStateException("Duplicate future ID: " + fut);
+                                // Add before mapping.
+                                if (!ctx.mvcc().addFuture(fut))
+                                    throw new IllegalStateException("Duplicate future ID: " + fut);
+                            }
+                            finally {
+                                if (top != null)
+                                    top.readUnlock();
+                            }
                         }
 
                         boolean timedout = false;
@@ -788,45 +817,76 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                         // Handle implicit locks for pessimistic transactions.
                         if (req.inTx()) {
                             if (tx == null) {
-                                tx = new GridDhtTxLocal(
-                                    ctx.shared(),
-                                    nearNode.id(),
-                                    req.version(),
-                                    req.futureId(),
-                                    req.miniId(),
-                                    req.threadId(),
-                                    req.implicitTx(),
-                                    req.implicitSingleTx(),
-                                    ctx.systemTx(),
-                                    false,
-                                    ctx.ioPolicy(),
-                                    PESSIMISTIC,
-                                    req.isolation(),
-                                    req.timeout(),
-                                    req.isInvalidate(),
-                                    false,
-                                    req.txSize(),
-                                    null,
-                                    req.subjectId(),
-                                    req.taskNameHash());
+                                GridDhtPartitionTopology top = null;
 
-                                tx.syncCommit(req.syncCommit());
+                                if (req.firstClientRequest()) {
+                                    assert CU.clientNode(nearNode);
 
-                                tx = ctx.tm().onCreated(null, tx);
+                                    top = topology();
 
-                                if (tx == null || !tx.init()) {
-                                    String msg = "Failed to acquire lock (transaction has been completed): " +
-                                        req.version();
+                                    topology().readLock();
+                                }
 
-                                    U.warn(log, msg);
+                                try {
+                                    if (top != null && needRemap(req.topologyVersion(), top.topologyVersion())) {
+                                        if (log.isDebugEnabled()) {
+                                            log.debug("Client topology version mismatch, need remap lock request [" +
+                                                "reqTopVer=" + req.topologyVersion() +
+                                                ", locTopVer=" + top.topologyVersion() +
+                                                ", req=" + req + ']');
+                                        }
 
-                                    if (tx != null)
-                                        tx.rollback();
+                                        GridNearLockResponse res = sendClientLockRemapResponse(nearNode,
+                                            req,
+                                            top.topologyVersion());
 
-                                    return new GridDhtFinishedFuture<>(new IgniteCheckedException(msg));
-                                }
+                                        return new GridFinishedFuture<>(res);
+                                    }
 
-                                tx.topologyVersion(req.topologyVersion());
+                                    tx = new GridDhtTxLocal(
+                                        ctx.shared(),
+                                        nearNode.id(),
+                                        req.version(),
+                                        req.futureId(),
+                                        req.miniId(),
+                                        req.threadId(),
+                                        req.implicitTx(),
+                                        req.implicitSingleTx(),
+                                        ctx.systemTx(),
+                                        false,
+                                        ctx.ioPolicy(),
+                                        PESSIMISTIC,
+                                        req.isolation(),
+                                        req.timeout(),
+                                        req.isInvalidate(),
+                                        false,
+                                        req.txSize(),
+                                        null,
+                                        req.subjectId(),
+                                        req.taskNameHash());
+
+                                    tx.syncCommit(req.syncCommit());
+
+                                    tx = ctx.tm().onCreated(null, tx);
+
+                                    if (tx == null || !tx.init()) {
+                                        String msg = "Failed to acquire lock (transaction has been completed): " +
+                                            req.version();
+
+                                        U.warn(log, msg);
+
+                                        if (tx != null)
+                                            tx.rollback();
+
+                                        return new GridDhtFinishedFuture<>(new IgniteCheckedException(msg));
+                                    }
+
+                                    tx.topologyVersion(req.topologyVersion());
+                                }
+                                finally {
+                                    if (top != null)
+                                        top.readUnlock();
+                                }
                             }
 
                             ctx.tm().txContext(tx);
@@ -947,6 +1007,42 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
     }
 
     /**
+     * @param nearNode Client node.
+     * @param req Request.
+     * @param topVer Remap version.
+     * @return Response.
+     */
+    private GridNearLockResponse sendClientLockRemapResponse(ClusterNode nearNode,
+        GridNearLockRequest req,
+        AffinityTopologyVersion topVer) {
+        assert topVer != null;
+
+        GridNearLockResponse res = new GridNearLockResponse(
+            ctx.cacheId(),
+            req.version(),
+            req.futureId(),
+            req.miniId(),
+            false,
+            0,
+            null,
+            topVer);
+
+        try {
+            ctx.io().send(nearNode, res, ctx.ioPolicy());
+        }
+        catch (ClusterTopologyCheckedException e) {
+            if (log.isDebugEnabled())
+                log.debug("Failed to send client lock remap response, client node failed " +
+                    "[node=" + nearNode + ", req=" + req + ']');
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send client lock remap response [node=" + nearNode + ", req=" + req + ']', e);
+        }
+
+        return res;
+    }
+
+    /**
      * @param nearNode Near node.
      * @param entries Entries.
      * @param req Lock request.
@@ -968,7 +1064,13 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
         try {
             // Send reply back to originating near node.
             GridNearLockResponse res = new GridNearLockResponse(ctx.cacheId(),
-                req.version(), req.futureId(), req.miniId(), tx != null && tx.onePhaseCommit(), entries.size(), err);
+                req.version(),
+                req.futureId(),
+                req.miniId(),
+                tx != null && tx.onePhaseCommit(),
+                entries.size(),
+                err,
+                null);
 
             if (err == null) {
                 res.pending(localDhtPendingVersions(entries, mappedVer));
@@ -1077,8 +1179,14 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             U.error(log, "Failed to get value for lock reply message for node [node=" +
                 U.toShortString(nearNode) + ", req=" + req + ']', e);
 
-            return new GridNearLockResponse(ctx.cacheId(), req.version(), req.futureId(), req.miniId(), false,
-                entries.size(), e);
+            return new GridNearLockResponse(ctx.cacheId(),
+                req.version(),
+                req.futureId(),
+                req.miniId(),
+                false,
+                entries.size(),
+                e,
+                null);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 54b59b8..90edb0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -52,15 +52,13 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     private static final long serialVersionUID = 0L;
 
     /** Near mappings. */
-    protected Map<UUID, GridDistributedTxMapping> nearMap =
-        new ConcurrentHashMap8<>();
+    protected Map<UUID, GridDistributedTxMapping> nearMap = new ConcurrentHashMap8<>();
 
     /** DHT mappings. */
-    protected Map<UUID, GridDistributedTxMapping> dhtMap =
-        new ConcurrentHashMap8<>();
+    protected Map<UUID, GridDistributedTxMapping> dhtMap = new ConcurrentHashMap8<>();
 
     /** Mapped flag. */
-    private AtomicBoolean mapped = new AtomicBoolean();
+    protected AtomicBoolean mapped = new AtomicBoolean();
 
     /** */
     private long dhtThreadId;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index 293cf95..af0fbdf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -582,7 +582,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
             tx.writeVersion(),
             tx.invalidPartitions(),
             ret,
-            prepErr);
+            prepErr,
+            null);
 
         if (prepErr == null) {
             addDhtValues(res);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 8bbfe96..8630421 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
@@ -171,7 +171,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         metrics = m;
 
-        preldr = new GridDhtPreloader<>(ctx);
+        preldr = new GridDhtPreloader(ctx);
 
         preldr.start();
 
@@ -737,6 +737,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable final CacheEntryPredicate[] filter,
         final boolean waitTopFut
     ) {
+        assert ctx.updatesAllowed();
+
         if (map != null && keyCheck)
             validateCacheKeys(map.keySet());
 
@@ -793,6 +795,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         boolean rawRetval,
         @Nullable final CacheEntryPredicate[] filter
     ) {
+        assert ctx.updatesAllowed();
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -1024,9 +1028,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         IgniteCacheExpiryPolicy expiry = null;
 
         try {
-            // 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());
+            List<GridDhtCacheEntry> locked = null;
             Collection<IgniteBiTuple<GridDhtCacheEntry, GridCacheVersion>> deleted = null;
 
             try {
@@ -1043,11 +1045,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                     }
 
                     // Do not check topology version for CLOCK versioning since
-                    // partition exchange will wait for near update future.
+                    // partition exchange will wait for near update future (if future is on server node).
                     // Also do not check topology version if topology was locked on near node by
                     // external transaction or explicit lock.
-                    if (topology().topologyVersion().equals(req.topologyVersion()) || req.topologyLocked() ||
-                        ctx.config().getAtomicWriteOrderMode() == CLOCK) {
+                    if ((req.fastMap() && !req.clientRequest()) || req.topologyLocked() ||
+                        !needRemap(req.topologyVersion(), topology().topologyVersion())) {
                         ClusterNode node = ctx.discovery().node(nodeId);
 
                         if (node == null) {
@@ -1056,13 +1058,17 @@ 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();
 
                         if (ver == null) {
                             // Assign next version for update inside entries lock.
-                            ver = ctx.versions().next(req.topologyVersion());
+                            ver = ctx.versions().next(topology().topologyVersion());
 
                             if (hasNear)
                                 res.nearVersion(ver);
@@ -1105,7 +1111,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 retVal = updRes.invokeResults();
                         }
                         else {
-                            UpdateSingleResult<K, V> updRes = updateSingle(node,
+                            UpdateSingleResult updRes = updateSingle(node,
                                 hasNear,
                                 req,
                                 res,
@@ -1144,7 +1150,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 e.printStackTrace();
             }
             finally {
-                unlockEntries(locked, req.topologyVersion());
+                if (locked != null)
+                    unlockEntries(locked, req.topologyVersion());
 
                 // Enqueue if necessary after locks release.
                 if (deleted != null) {
@@ -1157,7 +1164,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             }
         }
         catch (GridDhtInvalidPartitionException ignore) {
-            assert ctx.config().getAtomicWriteOrderMode() == PRIMARY;
+            assert !req.fastMap() || req.clientRequest() : req;
 
             if (log.isDebugEnabled())
                 log.debug("Caught invalid partition exception for cache entry (will remap update request): " + req);
@@ -1605,7 +1612,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @return Return value.
      * @throws GridCacheEntryRemovedException Should be never thrown.
      */
-    private UpdateSingleResult<K, V> updateSingle(
+    private UpdateSingleResult updateSingle(
         ClusterNode node,
         boolean hasNear,
         GridNearAtomicUpdateRequest req,
@@ -1799,7 +1806,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             }
         }
 
-        return new UpdateSingleResult<>(retVal, deleted, dhtFut);
+        return new UpdateSingleResult(retVal, deleted, dhtFut);
     }
 
     /**
@@ -2572,7 +2579,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /**
      * Result of {@link GridDhtAtomicCache#updateSingle} execution.
      */
-    private static class UpdateSingleResult<K, V> {
+    private static class UpdateSingleResult {
         /** */
         private final GridCacheReturn retVal;
 
@@ -2772,14 +2779,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         /** {@inheritDoc} */
         @Override public void onTimeout() {
             if (guard.compareAndSet(false, true)) {
-                writeLock().lock();
+                ctx.closures().runLocalSafe(new Runnable() {
+                    @Override public void run() {
+                        writeLock().lock();
 
-                try {
-                    finish();
-                }
-                finally {
-                    writeLock().unlock();
-                }
+                        try {
+                            finish();
+                        }
+                        finally {
+                            writeLock().unlock();
+                        }
+                    }
+                });
             }
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 40ab104..ff8454e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -86,6 +86,9 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
     /** Future keys. */
     private Collection<KeyCacheObject> keys;
 
+    /** */
+    private boolean waitForExchange;
+
     /**
      * @param cctx Cache context.
      * @param completionCb Callback to invoke when future is completed.
@@ -113,6 +116,10 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
             log = U.logger(cctx.kernalContext(), logRef, GridDhtAtomicUpdateFuture.class);
 
         keys = new ArrayList<>(updateReq.keys().size());
+
+        boolean topLocked = updateReq.topologyLocked() || (updateReq.fastMap() && !updateReq.clientRequest());
+
+        waitForExchange = !topLocked;
     }
 
     /** {@inheritDoc} */
@@ -164,8 +171,7 @@ public class GridDhtAtomicUpdateFuture extends GridFutureAdapter<Void>
 
     /** {@inheritDoc} */
     @Override public boolean waitForPartitionExchange() {
-        // Wait dht update futures in PRIMARY mode.
-        return cctx.config().getAtomicWriteOrderMode() == CacheAtomicWriteOrderMode.PRIMARY;
+        return waitForExchange;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 76e05e5..07f5ecf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -128,6 +128,12 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /** Fast map flag. */
     private final boolean fastMap;
 
+    /** */
+    private boolean fastMapRemap;
+
+    /** */
+    private GridCacheVersion updVer;
+
     /** Near cache flag. */
     private final boolean nearEnabled;
 
@@ -304,11 +310,11 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             topVer = cctx.mvcc().lastExplicitLockTopologyVersion(Thread.currentThread().getId());
 
         if (topVer == null)
-            mapOnTopology(keys, false, null, waitTopFut);
+            mapOnTopology(null, false, null, waitTopFut);
         else {
             topLocked = true;
 
-            map0(topVer, keys, false, null);
+            map0(topVer, null, false, null);
         }
     }
 
@@ -343,9 +349,11 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
      */
     public void onResult(UUID nodeId, GridNearAtomicUpdateResponse res) {
         if (res.remapKeys() != null) {
-            assert cctx.config().getAtomicWriteOrderMode() == PRIMARY;
+            assert !fastMap || cctx.kernalContext().clientNode();
+
+            Collection<KeyCacheObject> remapKeys = fastMap ? null : res.remapKeys();
 
-            mapOnTopology(res.remapKeys(), true, nodeId, true);
+            mapOnTopology(remapKeys, true, nodeId, true);
 
             return;
         }
@@ -454,9 +462,12 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
             else {
                 if (waitTopFut) {
                     fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override
-                        public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                            mapOnTopology(keys, remap, oldNodeId, waitTopFut);
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
+                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                                @Override public void run() {
+                                    mapOnTopology(keys, remap, oldNodeId, waitTopFut);
+                                }
+                            });
                         }
                     });
                 }
@@ -476,29 +487,43 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /**
      * Checks if future is ready to be completed.
      */
-    private synchronized void checkComplete() {
-        if ((syncMode == FULL_ASYNC && cctx.config().getAtomicWriteOrderMode() == PRIMARY) || mappings.isEmpty()) {
-            CachePartialUpdateCheckedException err0 = err;
+    private void checkComplete() {
+        boolean remap = false;
 
-            if (err0 != null)
-                onDone(err0);
-            else
-                onDone(opRes);
+        synchronized (this) {
+            if ((syncMode == FULL_ASYNC && cctx.config().getAtomicWriteOrderMode() == PRIMARY) || mappings.isEmpty()) {
+                CachePartialUpdateCheckedException err0 = err;
+
+                if (err0 != null)
+                    onDone(err0);
+                else {
+                    if (fastMapRemap) {
+                        assert cctx.kernalContext().clientNode();
+
+                        remap = true;
+                    }
+                    else
+                        onDone(opRes);
+                }
+            }
         }
+
+        if (remap)
+            mapOnTopology(null, true, null, true);
     }
 
     /**
      * @param topVer Topology version.
-     * @param keys Keys to map.
+     * @param remapKeys Keys to remap or {@code null} to map all keys.
      * @param remap Flag indicating if this is partial remap for this future.
      * @param oldNodeId Old node ID if was remap.
      */
     private void map0(
         AffinityTopologyVersion topVer,
-        Collection<?> keys,
+        @Nullable Collection<?> remapKeys,
         boolean remap,
         @Nullable UUID oldNodeId) {
-        assert oldNodeId == null || remap;
+        assert oldNodeId == null || remap || fastMapRemap;
 
         Collection<ClusterNode> topNodes = CU.affinityNodes(cctx, topVer);
 
@@ -519,12 +544,15 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         CacheConfiguration ccfg = cctx.config();
 
         // Assign version on near node in CLOCK ordering mode even if fastMap is false.
-        GridCacheVersion updVer = ccfg.getAtomicWriteOrderMode() == CLOCK ? cctx.versions().next(topVer) : null;
+        if (updVer == null)
+            updVer = ccfg.getAtomicWriteOrderMode() == CLOCK ? cctx.versions().next(topVer) : null;
 
         if (updVer != null && log.isDebugEnabled())
             log.debug("Assigned fast-map version for update on near node: " + updVer);
 
         if (keys.size() == 1 && !fastMap && (single == null || single)) {
+            assert remapKeys == null || remapKeys.size() == 1 : remapKeys;
+
             Object key = F.first(keys);
 
             Object val;
@@ -610,7 +638,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 filter,
                 subjId,
                 taskNameHash,
-                skipStore);
+                skipStore,
+                cctx.kernalContext().clientNode());
 
             req.addUpdateEntry(cacheKey,
                 val,
@@ -647,9 +676,16 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
         // Must do this in synchronized block because we need to atomically remove and add mapping.
         // Otherwise checkComplete() may see empty intermediate state.
         synchronized (this) {
-            if (remap)
+            if (oldNodeId != null)
                 removeMapping(oldNodeId);
 
+            // For fastMap mode wait for all responses before remapping.
+            if (remap && fastMap && !mappings.isEmpty()) {
+                fastMapRemap = true;
+
+                return;
+            }
+
             // Create mappings first, then send messages.
             for (Object key : keys) {
                 if (key == null) {
@@ -705,6 +741,9 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
 
                 KeyCacheObject cacheKey = cctx.toCacheKeyObject(key);
 
+                if (remapKeys != null && !remapKeys.contains(cacheKey))
+                    continue;
+
                 if (op != TRANSFORM)
                     val = cctx.toCacheObject(val);
 
@@ -748,7 +787,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                             filter,
                             subjId,
                             taskNameHash,
-                            skipStore);
+                            skipStore,
+                            cctx.kernalContext().clientNode());
 
                         pendingMappings.put(nodeId, mapped);
 
@@ -763,6 +803,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                     i++;
                 }
             }
+
+            fastMapRemap = false;
         }
 
         if ((single == null || single) && pendingMappings.size() == 1) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
index a96a666..86c5ab8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateRequest.java
@@ -132,6 +132,9 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
     /** Skip write-through to a persistent storage. */
     private boolean skipStore;
 
+    /** */
+    private boolean clientReq;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -148,6 +151,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
      * @param fastMap Fast map scheme flag.
      * @param updateVer Update version set if fast map is performed.
      * @param topVer Topology version.
+     * @param topLocked Topology locked flag.
      * @param syncMode Synchronization mode.
      * @param op Cache update operation.
      * @param retval Return value required flag.
@@ -157,6 +161,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
      * @param subjId Subject ID.
      * @param taskNameHash Task name hash code.
      * @param skipStore Skip write-through to a persistent storage.
+     * @param clientReq Client node request flag.
      */
     public GridNearAtomicUpdateRequest(
         int cacheId,
@@ -174,7 +179,8 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
         @Nullable CacheEntryPredicate[] filter,
         @Nullable UUID subjId,
         int taskNameHash,
-        boolean skipStore
+        boolean skipStore,
+        boolean clientReq
     ) {
         this.cacheId = cacheId;
         this.nodeId = nodeId;
@@ -193,6 +199,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
         this.skipStore = skipStore;
+        this.clientReq = clientReq;
 
         keys = new ArrayList<>();
     }
@@ -266,6 +273,13 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
     }
 
     /**
+     * @return {@code True} if request sent from client node.
+     */
+    public boolean clientRequest() {
+        return clientReq;
+    }
+
+    /**
      * @return Cache write synchronization mode.
      */
     public CacheWriteSynchronizationMode writeSynchronizationMode() {
@@ -574,126 +588,132 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
         switch (writer.state()) {
             case 3:
-                if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
+                if (!writer.writeBoolean("clientReq", clientReq))
                     return false;
 
                 writer.incrementState();
 
             case 4:
-                if (!writer.writeMessage("conflictTtls", conflictTtls))
+                if (!writer.writeMessage("conflictExpireTimes", conflictExpireTimes))
                     return false;
 
                 writer.incrementState();
 
             case 5:
-                if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("conflictTtls", conflictTtls))
                     return false;
 
                 writer.incrementState();
 
             case 6:
-                if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
+                if (!writer.writeCollection("conflictVers", conflictVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 7:
-                if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes))
+                if (!writer.writeCollection("entryProcessorsBytes", entryProcessorsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 8:
-                if (!writer.writeBoolean("fastMap", fastMap))
+                if (!writer.writeByteArray("expiryPlcBytes", expiryPlcBytes))
                     return false;
 
                 writer.incrementState();
 
             case 9:
-                if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
+                if (!writer.writeBoolean("fastMap", fastMap))
                     return false;
 
                 writer.incrementState();
 
             case 10:
-                if (!writer.writeMessage("futVer", futVer))
+                if (!writer.writeObjectArray("filter", filter, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 11:
-                if (!writer.writeBoolean("hasPrimary", hasPrimary))
+                if (!writer.writeMessage("futVer", futVer))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
+                if (!writer.writeBoolean("hasPrimary", hasPrimary))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
+                if (!writer.writeObjectArray("invokeArgsBytes", invokeArgsBytes, MessageCollectionItemType.BYTE_ARR))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
+                if (!writer.writeCollection("keys", keys, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 15:
-                if (!writer.writeBoolean("retval", retval))
+                if (!writer.writeByte("op", op != null ? (byte)op.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
             case 16:
-                if (!writer.writeBoolean("skipStore", skipStore))
+                if (!writer.writeBoolean("retval", retval))
                     return false;
 
                 writer.incrementState();
 
             case 17:
-                if (!writer.writeUuid("subjId", subjId))
+                if (!writer.writeBoolean("skipStore", skipStore))
                     return false;
 
                 writer.incrementState();
 
             case 18:
-                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
+                if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
             case 19:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
+                if (!writer.writeByte("syncMode", syncMode != null ? (byte)syncMode.ordinal() : -1))
                     return false;
 
                 writer.incrementState();
 
             case 20:
-                if (!writer.writeBoolean("topLocked", topLocked))
+                if (!writer.writeInt("taskNameHash", taskNameHash))
                     return false;
 
                 writer.incrementState();
 
             case 21:
-                if (!writer.writeMessage("topVer", topVer))
+                if (!writer.writeBoolean("topLocked", topLocked))
                     return false;
 
                 writer.incrementState();
 
             case 22:
-                if (!writer.writeMessage("updateVer", updateVer))
+                if (!writer.writeMessage("topVer", topVer))
                     return false;
 
                 writer.incrementState();
 
             case 23:
+                if (!writer.writeMessage("updateVer", updateVer))
+                    return false;
+
+                writer.incrementState();
+
+            case 24:
                 if (!writer.writeCollection("vals", vals, MessageCollectionItemType.MSG))
                     return false;
 
@@ -716,7 +736,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
         switch (reader.state()) {
             case 3:
-                conflictExpireTimes = reader.readMessage("conflictExpireTimes");
+                clientReq = reader.readBoolean("clientReq");
 
                 if (!reader.isLastRead())
                     return false;
@@ -724,7 +744,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 4:
-                conflictTtls = reader.readMessage("conflictTtls");
+                conflictExpireTimes = reader.readMessage("conflictExpireTimes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -732,7 +752,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 5:
-                conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
+                conflictTtls = reader.readMessage("conflictTtls");
 
                 if (!reader.isLastRead())
                     return false;
@@ -740,7 +760,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 6:
-                entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
+                conflictVers = reader.readCollection("conflictVers", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
                     return false;
@@ -748,7 +768,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 7:
-                expiryPlcBytes = reader.readByteArray("expiryPlcBytes");
+                entryProcessorsBytes = reader.readCollection("entryProcessorsBytes", MessageCollectionItemType.BYTE_ARR);
 
                 if (!reader.isLastRead())
                     return false;
@@ -756,7 +776,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 8:
-                fastMap = reader.readBoolean("fastMap");
+                expiryPlcBytes = reader.readByteArray("expiryPlcBytes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -764,7 +784,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 9:
-                filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
+                fastMap = reader.readBoolean("fastMap");
 
                 if (!reader.isLastRead())
                     return false;
@@ -772,7 +792,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 10:
-                futVer = reader.readMessage("futVer");
+                filter = reader.readObjectArray("filter", MessageCollectionItemType.MSG, CacheEntryPredicate.class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -780,7 +800,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 11:
-                hasPrimary = reader.readBoolean("hasPrimary");
+                futVer = reader.readMessage("futVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -788,7 +808,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 12:
-                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
+                hasPrimary = reader.readBoolean("hasPrimary");
 
                 if (!reader.isLastRead())
                     return false;
@@ -796,7 +816,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 13:
-                keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
+                invokeArgsBytes = reader.readObjectArray("invokeArgsBytes", MessageCollectionItemType.BYTE_ARR, byte[].class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -804,6 +824,14 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
                 reader.incrementState();
 
             case 14:
+                keys = reader.readCollection("keys", MessageCollectionItemType.MSG);
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 15:
                 byte opOrd;
 
                 opOrd = reader.readByte("op");
@@ -815,7 +843,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 15:
+            case 16:
                 retval = reader.readBoolean("retval");
 
                 if (!reader.isLastRead())
@@ -823,7 +851,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 16:
+            case 17:
                 skipStore = reader.readBoolean("skipStore");
 
                 if (!reader.isLastRead())
@@ -831,7 +859,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 17:
+            case 18:
                 subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
@@ -839,7 +867,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 18:
+            case 19:
                 byte syncModeOrd;
 
                 syncModeOrd = reader.readByte("syncMode");
@@ -851,7 +879,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 19:
+            case 20:
                 taskNameHash = reader.readInt("taskNameHash");
 
                 if (!reader.isLastRead())
@@ -859,7 +887,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 20:
+            case 21:
                 topLocked = reader.readBoolean("topLocked");
 
                 if (!reader.isLastRead())
@@ -867,7 +895,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 21:
+            case 22:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -875,7 +903,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 22:
+            case 23:
                 updateVer = reader.readMessage("updateVer");
 
                 if (!reader.isLastRead())
@@ -883,7 +911,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
                 reader.incrementState();
 
-            case 23:
+            case 24:
                 vals = reader.readCollection("vals", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -903,7 +931,7 @@ public class GridNearAtomicUpdateRequest extends GridCacheMessage implements Gri
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 24;
+        return 25;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 05b3c7b..221b230 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -362,13 +362,13 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         @Nullable TransactionIsolation isolation,
         long accessTtl
     ) {
-        assert tx == null || tx instanceof GridNearTxLocal;
+        assert tx == null || tx instanceof GridNearTxLocal : tx;
 
         GridNearTxLocal txx = (GridNearTxLocal)tx;
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        GridDhtColocatedLockFuture<K, V> fut = new GridDhtColocatedLockFuture<>(ctx,
+        GridDhtColocatedLockFuture fut = new GridDhtColocatedLockFuture(ctx,
             keys,
             txx,
             isRead,
@@ -619,7 +619,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @return Lock future.
      */
     IgniteInternalFuture<Exception> lockAllAsync(
-        final GridCacheContext<K, V> cacheCtx,
+        final GridCacheContext<?, ?> cacheCtx,
         @Nullable final GridNearTxLocal tx,
         final long threadId,
         final GridCacheVersion ver,
@@ -700,7 +700,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @return Lock future.
      */
     private IgniteInternalFuture<Exception> lockAllAsync0(
-        GridCacheContext<K, V> cacheCtx,
+        GridCacheContext<?, ?> cacheCtx,
         @Nullable final GridNearTxLocal tx,
         long threadId,
         final GridCacheVersion ver,
@@ -715,7 +715,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         int cnt = keys.size();
 
         if (tx == null) {
-            GridDhtLockFuture<K, V> fut = new GridDhtLockFuture<>(ctx,
+            GridDhtLockFuture fut = new GridDhtLockFuture(ctx,
                 ctx.localNodeId(),
                 ver,
                 topVer,
@@ -838,7 +838,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         assert nodeId != null;
         assert res != null;
 
-        GridDhtColocatedLockFuture<K, V> fut = (GridDhtColocatedLockFuture<K, V>)ctx.mvcc().
+        GridDhtColocatedLockFuture fut = (GridDhtColocatedLockFuture)ctx.mvcc().
             <Boolean>future(res.version(), res.futureId());
 
         if (fut != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 372c517..c784948 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -46,7 +46,7 @@ import static org.apache.ignite.events.EventType.*;
 /**
  * Colocated cache lock future.
  */
-public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentityFuture<Boolean>
+public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture<Boolean>
     implements GridCacheFuture<Boolean> {
     /** */
     private static final long serialVersionUID = 0L;
@@ -59,7 +59,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
     /** Cache registry. */
     @GridToStringExclude
-    private GridCacheContext<K, V> cctx;
+    private GridCacheContext<?, ?> cctx;
 
     /** Lock owner thread. */
     @GridToStringInclude
@@ -121,10 +121,10 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
      * @param timeout Lock acquisition timeout.
      * @param accessTtl TTL for read operation.
      * @param filter Filter.
-     * @param skipStore
+     * @param skipStore Skip store flag.
      */
     public GridDhtColocatedLockFuture(
-        GridCacheContext<K, V> cctx,
+        GridCacheContext<?, ?> cctx,
         Collection<KeyCacheObject> keys,
         @Nullable GridNearTxLocal tx,
         boolean read,
@@ -326,13 +326,14 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
      * Undoes all locks.
      *
      * @param dist If {@code true}, then remove locks from remote nodes as well.
+     * @param rollback {@code True} if should rollback tx.
      */
-    private void undoLocks(boolean dist) {
+    private void undoLocks(boolean dist, boolean rollback) {
         // Transactions will undo during rollback.
         if (dist && tx == null)
             cctx.colocated().removeLocks(threadId, lockVer, keys);
         else {
-            if (tx != null) {
+            if (rollback && tx != null) {
                 if (tx.setRollbackOnly()) {
                     if (log.isDebugEnabled())
                         log.debug("Marked transaction as rollback only because locks could not be acquired: " + tx);
@@ -346,16 +347,6 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
     }
 
     /**
-     *
-     * @param dist {@code True} if need to distribute lock release.
-     */
-    private void onFailed(boolean dist) {
-        undoLocks(dist);
-
-        complete(false);
-    }
-
-    /**
      * @param success Success flag.
      */
     public void complete(boolean success) {
@@ -475,7 +466,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                 ", fut=" + this + ']');
 
         if (!success)
-            undoLocks(distribute);
+            undoLocks(distribute, true);
 
         if (tx != null)
             cctx.tm().txContext(tx);
@@ -550,7 +541,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
             // Continue mapping on the same topology version as it was before.
             this.topVer.compareAndSet(null, topVer);
 
-            map(keys);
+            map(keys, false);
 
             markInitialized();
 
@@ -558,14 +549,17 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
         }
 
         // Must get topology snapshot and map on that version.
-        mapOnTopology();
+        mapOnTopology(false, null);
     }
 
     /**
      * Acquires topology future and checks it completeness under the read lock. If it is not complete,
      * will asynchronously wait for it's completeness and then try again.
+     *
+     * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void mapOnTopology() {
+    private void mapOnTopology(final boolean remap, @Nullable final Runnable c) {
         // We must acquire topology snapshot from the topology version future.
         cctx.topology().readLock();
 
@@ -589,19 +583,30 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
                 AffinityTopologyVersion topVer = fut.topologyVersion();
 
-                if (tx != null)
-                    tx.topologyVersion(topVer);
+                if (remap) {
+                    if (tx != null)
+                        tx.onRemap(topVer);
+
+                    this.topVer.set(topVer);
+                }
+                else {
+                    if (tx != null)
+                        tx.topologyVersion(topVer);
+
+                    this.topVer.compareAndSet(null, topVer);
+                }
 
-                this.topVer.compareAndSet(null, topVer);
+                map(keys, remap);
 
-                map(keys);
+                if (c != null)
+                    c.run();
 
                 markInitialized();
             }
             else {
                 fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                        mapOnTopology();
+                        mapOnTopology(remap, c);
                     }
                 });
             }
@@ -617,8 +622,9 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
      * groups belonging to one primary node and locks for these groups are acquired sequentially.
      *
      * @param keys Keys.
+     * @param remap Remap flag.
      */
-    private void map(Collection<KeyCacheObject> keys) {
+    private void map(Collection<KeyCacheObject> keys, boolean remap) {
         try {
             AffinityTopologyVersion topVer = this.topVer.get();
 
@@ -633,8 +639,12 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                 return;
             }
 
+            boolean clientNode = cctx.kernalContext().clientNode();
+
+            assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
+
             // First assume this node is primary for all keys passed in.
-            if (mapAsPrimary(keys, topVer))
+            if (!clientNode && mapAsPrimary(keys, topVer))
                 return;
 
             Deque<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
@@ -668,6 +678,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
             boolean hasRmtNodes = false;
 
+            boolean first = true;
+
             // Create mini futures.
             for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
                 GridNearLockMapping mapping = iter.next();
@@ -736,6 +748,14 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
                             if (cand != null && !cand.reentry()) {
                                 if (req == null) {
+                                    boolean clientFirst = false;
+
+                                    if (first) {
+                                        clientFirst = clientNode && (tx == null || !tx.hasRemoteLocks());
+
+                                        first = false;
+                                    }
+
                                     req = new GridNearLockRequest(
                                         cctx.cacheId(),
                                         topVer,
@@ -757,7 +777,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                                         inTx() ? tx.subjectId() : null,
                                         inTx() ? tx.taskNameHash() : 0,
                                         read ? accessTtl : -1L,
-                                        skipStore);
+                                        skipStore,
+                                        clientFirst);
 
                                     mapping.request(req);
                                 }
@@ -815,7 +836,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
             if (hasRmtNodes) {
                 trackable = true;
 
-                if (!cctx.mvcc().addFuture(this))
+                if (!remap && !cctx.mvcc().addFuture(this))
                     throw new IllegalStateException("Duplicate future ID: " + this);
             }
             else
@@ -1249,75 +1270,111 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                     return;
                 }
 
-                int i = 0;
+                if (res.clientRemapVersion() != null) {
+                    assert cctx.kernalContext().clientNode();
+
+                    IgniteInternalFuture<?> affFut =
+                        cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
+
+                    if (affFut != null && !affFut.isDone()) {
+                        affFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                            @Override public void apply(IgniteInternalFuture<?> fut) {
+                                remap();
+                            }
+                        });
+                    }
+                    else
+                        remap();
+                }
+                else  {
+                    int i = 0;
 
-                for (KeyCacheObject k : keys) {
-                    IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(k);
+                    for (KeyCacheObject k : keys) {
+                        IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(k);
 
-                    CacheObject newVal = res.value(i);
+                        CacheObject newVal = res.value(i);
 
-                    GridCacheVersion dhtVer = res.dhtVersion(i);
+                        GridCacheVersion dhtVer = res.dhtVersion(i);
 
-                    if (newVal == null) {
-                        if (oldValTup != null) {
-                            if (oldValTup.get1().equals(dhtVer))
-                                newVal = oldValTup.get2();
+                        if (newVal == null) {
+                            if (oldValTup != null) {
+                                if (oldValTup.get1().equals(dhtVer))
+                                    newVal = oldValTup.get2();
+                            }
                         }
-                    }
 
-                    if (inTx()) {
-                        IgniteTxEntry txEntry = tx.entry(cctx.txKey(k));
+                        if (inTx()) {
+                            IgniteTxEntry txEntry = tx.entry(cctx.txKey(k));
+
+                            // In colocated cache we must receive responses only for detached entries.
+                            assert txEntry.cached().detached() : txEntry;
 
-                        // In colocated cache we must receive responses only for detached entries.
-                        assert txEntry.cached().detached();
+                            txEntry.markLocked();
 
-                        txEntry.markLocked();
+                            GridDhtDetachedCacheEntry entry = (GridDhtDetachedCacheEntry)txEntry.cached();
 
-                        GridDhtDetachedCacheEntry entry = (GridDhtDetachedCacheEntry)txEntry.cached();
+                            if (res.dhtVersion(i) == null) {
+                                onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
+                                    "(will fail the lock): " + res));
+
+                                return;
+                            }
 
-                        if (res.dhtVersion(i) == null) {
-                            onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
-                                "(will fail the lock): " + res));
+                            // Set value to detached entry.
+                            entry.resetFromPrimary(newVal, dhtVer);
 
-                            return;
+                            tx.hasRemoteLocks(true);
+
+                            if (log.isDebugEnabled())
+                                log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
+                        }
+                        else
+                            cctx.mvcc().markExplicitOwner(k, threadId);
+
+                        if (retval && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
+                            cctx.events().addEvent(cctx.affinity().partition(k),
+                                k,
+                                tx,
+                                null,
+                                EVT_CACHE_OBJECT_READ,
+                                newVal,
+                                newVal != null,
+                                null,
+                                false,
+                                CU.subjectId(tx, cctx.shared()),
+                                null,
+                                tx == null ? null : tx.resolveTaskName());
                         }
 
-                        // Set value to detached entry.
-                        entry.resetFromPrimary(newVal, dhtVer);
+                        i++;
+                    }
 
-                        if (log.isDebugEnabled())
-                            log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
+                    try {
+                        proceedMapping(mappings);
                     }
-                    else
-                        cctx.mvcc().markExplicitOwner(k, threadId);
-
-                    if (retval && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
-                        cctx.events().addEvent(cctx.affinity().partition(k),
-                            k,
-                            tx,
-                            null,
-                            EVT_CACHE_OBJECT_READ,
-                            newVal,
-                            newVal != null,
-                            null,
-                            false,
-                            CU.subjectId(tx, cctx.shared()),
-                            null,
-                            tx == null ? null : tx.resolveTaskName());
+                    catch (IgniteCheckedException e) {
+                        onDone(e);
                     }
 
-                    i++;
+                    onDone(true);
                 }
+            }
+        }
 
-                try {
-                    proceedMapping(mappings);
-                }
-                catch (IgniteCheckedException e) {
-                    onDone(e);
-                }
+        /**
+         *
+         */
+        private void remap() {
+            undoLocks(false, false);
 
-                onDone(true);
-            }
+            for (KeyCacheObject key : GridDhtColocatedLockFuture.this.keys)
+                cctx.mvcc().removeExplicitLock(threadId, key, lockVer);
+
+            mapOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(true);
+                }
+            });
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 78966d0..1d57ef7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -80,7 +80,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
     private IgniteUuid futId = IgniteUuid.randomUuid();
 
     /** Preloader. */
-    private GridDhtPreloader<K, V> preloader;
+    private GridDhtPreloader preloader;
 
     /** Trackable flag. */
     private boolean trackable;
@@ -95,7 +95,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
         GridCacheContext<K, V> cctx,
         AffinityTopologyVersion topVer,
         Collection<KeyCacheObject> keys,
-        GridDhtPreloader<K, V> preloader
+        GridDhtPreloader preloader
     ) {
         assert topVer.topologyVersion() != 0 : topVer;
         assert !F.isEmpty(keys) : keys;
@@ -208,21 +208,21 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @return {@code True} if some mapping was added.
      */
     private boolean map(Iterable<KeyCacheObject> keys, Collection<ClusterNode> exc) {
-        Map<ClusterNode, Set<KeyCacheObject>> mappings = new HashMap<>();
-
-        ClusterNode loc = cctx.localNode();
-
-        int curTopVer = topCntr.get();
+        Map<ClusterNode, Set<KeyCacheObject>> mappings = null;
 
         for (KeyCacheObject key : keys)
-            map(key, mappings, exc);
+            mappings = map(key, mappings, exc);
 
         if (isDone())
             return false;
 
         boolean ret = false;
 
-        if (!mappings.isEmpty()) {
+        if (mappings != null) {
+            ClusterNode loc = cctx.localNode();
+
+            int curTopVer = topCntr.get();
+
             preloader.addFuture(this);
 
             trackable = true;
@@ -275,22 +275,27 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @param key Key.
      * @param exc Exclude nodes.
      * @param mappings Mappings.
+     * @return Mappings.
      */
-    private void map(KeyCacheObject key, Map<ClusterNode, Set<KeyCacheObject>> mappings, Collection<ClusterNode> exc) {
+    private Map<ClusterNode, Set<KeyCacheObject>> map(KeyCacheObject key,
+        @Nullable Map<ClusterNode, Set<KeyCacheObject>> mappings,
+        Collection<ClusterNode> exc)
+    {
         ClusterNode loc = cctx.localNode();
 
-        int part = cctx.affinity().partition(key);
-
         GridCacheEntryEx e = cctx.dht().peekEx(key);
 
         try {
             if (e != null && !e.isNewLocked()) {
-                if (log.isDebugEnabled())
+                if (log.isDebugEnabled()) {
+                    int part = cctx.affinity().partition(key);
+
                     log.debug("Will not rebalance key (entry is not new) [cacheName=" + cctx.name() +
                         ", key=" + key + ", part=" + part + ", locId=" + cctx.nodeId() + ']');
+                }
 
                 // Key has been rebalanced or retrieved already.
-                return;
+                return mappings;
             }
         }
         catch (GridCacheEntryRemovedException ignore) {
@@ -299,6 +304,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     ", locId=" + cctx.nodeId() + ']');
         }
 
+        int part = cctx.affinity().partition(key);
+
         List<ClusterNode> owners = F.isEmpty(exc) ? top.owners(part, topVer) :
             new ArrayList<>(F.view(top.owners(part, topVer), F.notIn(exc)));
 
@@ -308,7 +315,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     "topVer=" + topVer + ", locId=" + cctx.nodeId() + ']');
 
             // Key is already rebalanced.
-            return;
+            return mappings;
         }
 
         // Create partition.
@@ -337,9 +344,12 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                     log.debug("Will not rebalance key (no nodes to request from with rebalancing disabled) [key=" +
                         key + ", part=" + part + ", locId=" + cctx.nodeId() + ']');
 
-                return;
+                return mappings;
             }
 
+            if (mappings == null)
+                mappings = U.newHashMap(keys.size());
+
             Collection<KeyCacheObject> mappedKeys = F.addIfAbsent(mappings, pick, F.<KeyCacheObject>newSet());
 
             assert mappedKeys != null;
@@ -357,6 +367,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
                 log.debug("Will not rebalance key (local partition is not MOVING) [cacheName=" + cctx.name() +
                     ", key=" + key + ", part=" + locPart + ", locId=" + cctx.nodeId() + ']');
         }
+
+        return mappings;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 633f237..a6e6c4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -53,12 +53,12 @@ import static org.apache.ignite.internal.processors.dr.GridDrType.*;
  * and populating local cache.
  */
 @SuppressWarnings("NonConstantFieldWithUpperCaseName")
-public class GridDhtPartitionDemandPool<K, V> {
+public class GridDhtPartitionDemandPool {
     /** Dummy message to wake up a blocking queue if a node leaves. */
     private final SupplyMessage DUMMY_TOP = new SupplyMessage();
 
     /** */
-    private final GridCacheContext<K, V> cctx;
+    private final GridCacheContext<?, ?> cctx;
 
     /** */
     private final IgniteLogger log;
@@ -99,7 +99,7 @@ public class GridDhtPartitionDemandPool<K, V> {
      * @param cctx Cache context.
      * @param busyLock Shutdown lock.
      */
-    public GridDhtPartitionDemandPool(GridCacheContext<K, V> cctx, ReadWriteLock busyLock) {
+    public GridDhtPartitionDemandPool(GridCacheContext<?, ?> cctx, ReadWriteLock busyLock) {
         assert cctx != null;
         assert busyLock != null;
 
@@ -108,9 +108,11 @@ public class GridDhtPartitionDemandPool<K, V> {
 
         log = cctx.logger(getClass());
 
-        poolSize = cctx.rebalanceEnabled() ? cctx.config().getRebalanceThreadPoolSize() : 0;
+        boolean enabled = cctx.rebalanceEnabled() && !cctx.kernalContext().clientNode();
 
-        if (poolSize > 0) {
+        poolSize = enabled ? cctx.config().getRebalanceThreadPoolSize() : 0;
+
+        if (enabled) {
             barrier = new CyclicBarrier(poolSize);
 
             dmdWorkers = new ArrayList<>(poolSize);
@@ -327,7 +329,7 @@ public class GridDhtPartitionDemandPool<K, V> {
      * @param assigns Assignments.
      * @param force {@code True} if dummy reassign.
      */
-    void addAssignments(final GridDhtPreloaderAssignments<K, V> assigns, boolean force) {
+    void addAssignments(final GridDhtPreloaderAssignments assigns, boolean force) {
         if (log.isDebugEnabled())
             log.debug("Adding partition assignments: " + assigns);
 
@@ -399,7 +401,7 @@ public class GridDhtPartitionDemandPool<K, V> {
         private int id;
 
         /** Partition-to-node assignments. */
-        private final LinkedBlockingDeque<GridDhtPreloaderAssignments<K, V>> assignQ = new LinkedBlockingDeque<>();
+        private final LinkedBlockingDeque<GridDhtPreloaderAssignments> assignQ = new LinkedBlockingDeque<>();
 
         /** Message queue. */
         private final LinkedBlockingDeque<SupplyMessage> msgQ =
@@ -425,7 +427,7 @@ public class GridDhtPartitionDemandPool<K, V> {
         /**
          * @param assigns Assignments.
          */
-        void addAssignments(GridDhtPreloaderAssignments<K, V> assigns) {
+        void addAssignments(GridDhtPreloaderAssignments assigns) {
             assert assigns != null;
 
             assignQ.offer(assigns);
@@ -885,7 +887,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                     }
 
                     // Sync up all demand threads at this step.
-                    GridDhtPreloaderAssignments<K, V> assigns = null;
+                    GridDhtPreloaderAssignments assigns = null;
 
                     while (assigns == null)
                         assigns = poll(assignQ, cctx.gridConfig().getNetworkTimeout(), this);
@@ -995,12 +997,12 @@ public class GridDhtPartitionDemandPool<K, V> {
      * @param exchFut Exchange future.
      * @return Assignments of partitions to nodes.
      */
-    GridDhtPreloaderAssignments<K, V> assign(GridDhtPartitionsExchangeFuture exchFut) {
+    GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
         // No assignments for disabled preloader.
         GridDhtPartitionTopology top = cctx.dht().topology();
 
         if (!cctx.rebalanceEnabled())
-            return new GridDhtPreloaderAssignments<>(exchFut, top.topologyVersion());
+            return new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
 
         int partCnt = cctx.affinity().partitions();
 
@@ -1009,7 +1011,7 @@ public class GridDhtPartitionDemandPool<K, V> {
             "Topology version mismatch [exchId=" + exchFut.exchangeId() +
                 ", topVer=" + top.topologyVersion() + ']';
 
-        GridDhtPreloaderAssignments<K, V> assigns = new GridDhtPreloaderAssignments<>(exchFut, top.topologyVersion());
+        GridDhtPreloaderAssignments assigns = new GridDhtPreloaderAssignments(exchFut, top.topologyVersion());
 
         AffinityTopologyVersion topVer = assigns.topologyVersion();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
index facf7e3..faa6cf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
@@ -237,7 +237,7 @@ public class GridDhtPartitionMap implements Comparable<GridDhtPartitionMap>, Ext
      * @return Full string representation.
      */
     public String toFullString() {
-        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", super.toString());
+        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", map.toString());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
index 5d9677d..13cfef3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
@@ -43,9 +43,9 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDh
 /**
  * Thread pool for supplying partitions to demanding nodes.
  */
-class GridDhtPartitionSupplyPool<K, V> {
+class GridDhtPartitionSupplyPool {
     /** */
-    private final GridCacheContext<K, V> cctx;
+    private final GridCacheContext<?, ?> cctx;
 
     /** */
     private final IgniteLogger log;
@@ -72,7 +72,7 @@ class GridDhtPartitionSupplyPool<K, V> {
      * @param cctx Cache context.
      * @param busyLock Shutdown lock.
      */
-    GridDhtPartitionSupplyPool(GridCacheContext<K, V> cctx, ReadWriteLock busyLock) {
+    GridDhtPartitionSupplyPool(GridCacheContext<?, ?> cctx, ReadWriteLock busyLock) {
         assert cctx != null;
         assert busyLock != null;
 
@@ -83,16 +83,18 @@ class GridDhtPartitionSupplyPool<K, V> {
 
         top = cctx.dht().topology();
 
-        int poolSize = cctx.rebalanceEnabled() ? cctx.config().getRebalanceThreadPoolSize() : 0;
+        if (!cctx.kernalContext().clientNode()) {
+            int poolSize = cctx.rebalanceEnabled() ? cctx.config().getRebalanceThreadPoolSize() : 0;
 
-        for (int i = 0; i < poolSize; i++)
-            workers.add(new SupplyWorker());
+            for (int i = 0; i < poolSize; i++)
+                workers.add(new SupplyWorker());
 
-        cctx.io().addHandler(cctx.cacheId(), GridDhtPartitionDemandMessage.class, new CI2<UUID, GridDhtPartitionDemandMessage>() {
-            @Override public void apply(UUID id, GridDhtPartitionDemandMessage m) {
-                processDemandMessage(id, m);
-            }
-        });
+            cctx.io().addHandler(cctx.cacheId(), GridDhtPartitionDemandMessage.class, new CI2<UUID, GridDhtPartitionDemandMessage>() {
+                @Override public void apply(UUID id, GridDhtPartitionDemandMessage m) {
+                    processDemandMessage(id, m);
+                }
+            });
+        }
 
         depEnabled = cctx.gridDeploy().enabled();
     }
@@ -248,11 +250,6 @@ class GridDhtPartitionSupplyPool<K, V> {
             boolean ack = false;
 
             try {
-                // Partition map exchange is finished which means that all near transactions with given
-                // topology version are committed. We can wait for local locks here as it will not take
-                // much time.
-                cctx.mvcc().finishLocks(d.topologyVersion()).get();
-
                 for (int part : d.partitions()) {
                     GridDhtLocalPartition loc = top.localPartition(part, d.topologyVersion(), false);
 


[11/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
index 44171a8..445a5e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConfigurationConsistencySelfTest.java
@@ -442,7 +442,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictSynchronized(true);
-                    cfg.setEvictionPolicy(new FifoEvictionPolicy(100));
+
+                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
+
+                    plc.setMaxSize(100);
+
+                    cfg.setEvictionPolicy(plc);
                     return null;
                 }
             },
@@ -450,7 +455,12 @@ public class GridCacheConfigurationConsistencySelfTest extends GridCommonAbstrac
                 /** {@inheritDoc} */
                 @Override public Void apply(CacheConfiguration cfg) {
                     cfg.setEvictSynchronized(false);
-                    cfg.setEvictionPolicy(new FifoEvictionPolicy(100));
+
+                    FifoEvictionPolicy plc = new FifoEvictionPolicy();
+
+                    plc.setMaxSize(100);
+
+                    cfg.setEvictionPolicy(plc);
                     return null;
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
deleted file mode 100644
index 7a1b7cc..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheExAbstractFullApiSelfTest.java
+++ /dev/null
@@ -1,103 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.transactions.*;
-
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.transactions.TransactionConcurrency.*;
-import static org.apache.ignite.transactions.TransactionIsolation.*;
-
-/**
- * Abstract test for private cache interface.
- */
-public abstract class GridCacheExAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected int gridCount() {
-        return 4;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return null;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testGetOutTx() throws Exception {
-        final AtomicInteger lockEvtCnt = new AtomicInteger();
-
-        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
-            @Override public boolean apply(Event evt) {
-                lockEvtCnt.incrementAndGet();
-
-                return true;
-            }
-        };
-
-        try {
-            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-
-            GridCacheAdapter<String, Integer> cache = ((IgniteKernal)grid(0)).internalCache();
-
-            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                int key = 0;
-
-                for (int i = 0; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.get("key" + key);
-
-                for (int i = key + 1; i < 1000; i++) {
-                    if (grid(0).affinity(null).mapKeyToNode("key" + i).id().equals(grid(0).localNode().id())) {
-                        key = i;
-
-                        break;
-                    }
-                }
-
-                cache.getAllOutTx(F.asList("key" + key));
-            }
-
-            assertTrue(GridTestUtils.waitForCondition(new PA() {
-                @Override public boolean apply() {
-                    info("Lock event count: " + lockEvtCnt.get());
-
-                    return lockEvtCnt.get() == (nearEnabled() ? 4 : 2);
-                }
-            }, 15000));
-        }
-        finally {
-            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
index 08e35ce..b4e523e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMemoryModeSelfTest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.swapspace.file.*;
 import org.apache.ignite.testframework.junits.common.*;
+
 import org.junit.*;
 
 import java.util.*;
@@ -81,6 +82,11 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         return cfg;
     }
 
+    /**
+     * Returns cache configuration.
+     *
+     * @return cache configuration.
+     */
     protected CacheConfiguration cacheConfiguration() {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
 
@@ -89,7 +95,16 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
         cacheCfg.setSwapEnabled(swapEnabled);
         cacheCfg.setCacheMode(mode);
         cacheCfg.setMemoryMode(memoryMode);
-        cacheCfg.setEvictionPolicy(maxOnheapSize == Integer.MAX_VALUE ? null : new LruEvictionPolicy(maxOnheapSize));
+
+        LruEvictionPolicy plc = null;
+
+        if (maxOnheapSize != Integer.MAX_VALUE) {
+            plc = new LruEvictionPolicy();
+            plc.setMaxSize(maxOnheapSize);
+        }
+
+        cacheCfg.setEvictionPolicy(plc);
+
         cacheCfg.setAtomicityMode(atomicity);
         cacheCfg.setOffHeapMaxMemory(offheapSize);
 
@@ -199,7 +214,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
      * @param swapEmpty Swap is empty.
      * @throws Exception If failed.
      */
-    private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty) throws Exception {
+    private void doTestPutAndPutAll(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty)
+        throws Exception {
         final int all = cache + offheapSwap;
 
         // put
@@ -231,7 +247,8 @@ public class GridCacheMemoryModeSelfTest extends GridCommonAbstractTest {
      * @param x Cache modifier.
      * @throws IgniteCheckedException If failed.
      */
-    void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty, CIX1<IgniteCache<String, Integer>> x) throws Exception {
+    void doTest(int cache, int offheapSwap, boolean offheapEmpty, boolean swapEmpty,
+        CIX1<IgniteCache<String, Integer>> x) throws Exception {
         ipFinder = new TcpDiscoveryVmIpFinder(true);
 
         startGrid();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
index 266937f..2fad523 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
@@ -56,7 +56,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         if (cache)
             cfg.setCacheConfiguration(cacheConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
index d69f7fa..b0f07f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapTest.java
@@ -76,7 +76,10 @@ public class GridCacheOffHeapTest extends GridCommonAbstractTest {
         cacheCfg.setStartSize(startSize);
 
         if (onheap > 0) {
-            cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(onheap));
+            FifoEvictionPolicy plc = new FifoEvictionPolicy();
+            plc.setMaxSize(onheap);
+
+            cacheCfg.setEvictionPolicy(plc);
 
             cacheCfg.setOffHeapMaxMemory(80 * 1024L * 1024L * 1024L); // 10GB
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
index ae36c4de..fc9e17b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -617,6 +617,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
         discoverySpi.setAckTimeout(60000);
         discoverySpi.setIpFinder(ipFinder);
+        discoverySpi.setForceServerMode(true);
 
         cfg.setDiscoverySpi(discoverySpi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
index e2cdd08..b6bfbc7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReloadSelfTest.java
@@ -74,7 +74,11 @@ public class GridCacheReloadSelfTest extends GridCommonAbstractTest {
         CacheConfiguration cacheCfg = defaultCacheConfiguration();
         cacheCfg.setName(CACHE_NAME);
         cacheCfg.setCacheMode(cacheMode);
-        cacheCfg.setEvictionPolicy(new LruEvictionPolicy(MAX_CACHE_ENTRIES));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(MAX_CACHE_ENTRIES);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setNearConfiguration(nearEnabled ? new NearCacheConfiguration() : null);
 
         final CacheStore store = new CacheStoreAdapter<Integer, Integer>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
index 3e5987a..7723319 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
@@ -20,6 +20,7 @@ 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.testframework.junits.common.*;
 
 import javax.cache.processor.*;
@@ -62,6 +63,8 @@ public class GridCacheReturnValueTransferSelfTest extends GridCommonAbstractTest
 
         cfg.setCacheConfiguration(ccfg);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (!cache)
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
new file mode 100644
index 0000000..b817f4c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
@@ -0,0 +1,190 @@
+/*
+ * 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.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link TcpDiscoverySpi} in client mode with multiple client nodes that interact with a cache concurrently.
+ */
+public class GridCacheTcpClientDiscoveryMultiThreadedTest extends GridCacheAbstractSelfTest {
+    /** Server nodes count. */
+    private static int srvNodesCnt;
+
+    /** Client nodes count. */
+    private static int clientNodesCnt;
+
+    /** Client node or not. */
+    private static boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return srvNodesCnt + clientNodesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        // Filling configuration for client nodes
+        if (client) {
+            TcpDiscoveryVmIpFinder clientFinder = new TcpDiscoveryVmIpFinder();
+            Collection<String> addrs = new ArrayList<>(ipFinder.getRegisteredAddresses().size());
+
+            for (InetSocketAddress sockAddr : ipFinder.getRegisteredAddresses())
+                addrs.add(sockAddr.getHostString() + ":" + sockAddr.getPort());
+
+            clientFinder.setAddresses(addrs);
+
+            cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(clientFinder));
+
+            cfg.setClientMode(true);
+        }
+
+        cfg.setLocalHost("127.0.0.1");
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 2 * 60 * 1000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheConcurrentlyWithMultipleClientNodes() throws Exception {
+        srvNodesCnt = 2;
+        clientNodesCnt = 3;
+
+        startServerNodes();
+
+        client = true;
+
+        for (int n = 0; n < 2; n++) {
+            startGridsMultiThreaded(srvNodesCnt, clientNodesCnt);
+
+            checkTopology(gridCount());
+
+            awaitPartitionMapExchange();
+
+            // Explicitly create near cache for even client nodes
+            for (int i = srvNodesCnt; i < gridCount(); i++)
+                grid(i).createNearCache(null, new NearCacheConfiguration<>());
+
+            final AtomicInteger threadsCnt = new AtomicInteger();
+
+            IgniteInternalFuture<?> f = multithreadedAsync(
+                    new Callable<Object>() {
+                        @Override public Object call() throws Exception {
+                            int clientIdx = srvNodesCnt + threadsCnt.getAndIncrement();
+
+                            Ignite node = grid(clientIdx);
+
+                            assert node.configuration().isClientMode();
+
+                            IgniteCache<Integer, Integer> cache = node.cache(null);
+
+                            boolean isNearCacheNode = clientIdx % 2 == 0;
+
+                            for (int i = 100 * clientIdx; i < 100 * (clientIdx + 1); i++)
+                                cache.put(i, i);
+
+                            for (int i = 100 * clientIdx; i < 100 * (clientIdx + 1); i++) {
+                                assertEquals(i, (int) cache.get(i));
+
+                                if (isNearCacheNode)
+                                    assertEquals(i, (int) cache.localPeek(i, CachePeekMode.ONHEAP));
+                            }
+
+                            stopGrid(clientIdx);
+
+                            return null;
+                        }
+                    },
+                    clientNodesCnt
+            );
+
+            f.get();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void startServerNodes() throws Exception {
+        client = false;
+
+        for (int i = 0; i < srvNodesCnt; i++)
+            startGrid(i);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void stopServerNodes() throws Exception {
+        for (int i = 0; i < srvNodesCnt; i++)
+            stopGrid(i);
+    }
+
+    /**
+     * Executes simple operation on the cache.
+     *
+     * @param cache Cache instance to use.
+     */
+    private void performSimpleOperationsOnCache(IgniteCache<Integer, Integer> cache) {
+        for (int i = 100; i < 200; i++)
+            cache.put(i, i);
+
+        for (int i = 100; i < 200; i++)
+            assertEquals(i, (int) cache.get(i));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 9a24109..eaa6e13 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -843,4 +843,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         return null;
     }
 
+    /** {@inheritDoc} */
+    @Override public void onUnlock() {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
index ce0a55c..dc50ee6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
@@ -19,8 +19,10 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -137,8 +139,14 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest {
 
                         tx.commit();
                     }
-                    catch (TransactionOptimisticException e) {
-                        info("Caught cache optimistic exception: " + e);
+                    catch (ClusterTopologyException e) {
+                        info("Caught topology exception: " + e);
+                    }
+                    catch (IgniteException e) {
+                        if (X.hasCause(e, ClusterTopologyCheckedException.class))
+                            info("Caught cache exception: " + e);
+                        else
+                            throw e;
                     }
 
                     try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index aa8e2f7..10b14cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -101,11 +101,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
 
         cfg.setCommunicationSpi(commSpi);
 
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(finder);
-
-        cfg.setDiscoverySpi(spi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(finder).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index 371e547..1a62d4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@ -82,7 +82,7 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+        TcpDiscoverySpi disco = new TcpDiscoverySpi().setForceServerMode(true);
 
         disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
index a030039..937a3b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
@@ -59,7 +59,7 @@ public class IgniteCacheConfigurationTemplateTest extends GridCommonAbstractTest
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         if (addTemplate) {
             CacheConfiguration dfltCfg = new CacheConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index 5cc9d04..cd04433 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.*;
 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.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
 
@@ -54,6 +55,8 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true));
+
         if (getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index 5aa0ac8..ce68b17 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -65,27 +65,33 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (gridName.endsWith("0"))
+        if (getTestGridName(0).equals(gridName)) {
             cfg.setClientMode(true);
 
+            cfg.setCacheConfiguration();
+        }
+
         return cfg;
     }
 
     /** Test key 1. */
     public static class TestKey implements Externalizable {
-        /** Test key 1. */
+        /** Field. */
+        @QuerySqlField(index = true)
+        private String field;
+
+        /**
+         * @param field Test key 1.
+         */
         public TestKey(String field) {
             this.field = field;
         }
 
         /** Test key 1. */
         public TestKey() {
+            // No-op.
         }
 
-        /** Field. */
-        @QuerySqlField(index = true)
-        private String field;
-
         /** {@inheritDoc} */
         @Override public boolean equals(Object o) {
             if (this == o)
@@ -112,14 +118,15 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             field = (String)in.readObject();
 
-            if (readCnt.decrementAndGet() <= 0) {
-                throw new IOException("Class can not be unmarshalled");
-            }
+            if (readCnt.decrementAndGet() <= 0)
+                throw new IOException("Class can not be unmarshalled.");
         }
     }
 
     /**
      * Sends put atomically and handles fail.
+     *
+     * @param k Key.
      */
     protected void failAtomicPut(int k) {
         try {
@@ -136,6 +143,8 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
     /**
      * Sends get atomically and handles fail.
+     *
+     * @param k Key.
      */
     protected void failAtomicGet(int k) {
         try {
@@ -150,6 +159,8 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
 
     /**
      * Tests that correct response will be sent to client node in case of unmarshalling failed.
+     *
+     * @throws Exception If failed.
      */
     public void testResponseMessageOnUnmarshallingFailed() throws Exception {
         //GridNearAtomicUpdateRequest unmarshalling failed test

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
index 732d12d..3462d71 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingNearErrorTest.java
@@ -33,10 +33,12 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0);
-        cfg.getCacheConfiguration()[0].setEvictSynchronized(true);
-        cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1);
-        cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1));
+        if (cfg.isClientMode() == null || !cfg.isClientMode()) {
+            cfg.getCacheConfiguration()[0].setEvictMaxOverflowRatio(0);
+            cfg.getCacheConfiguration()[0].setEvictSynchronized(true);
+            cfg.getCacheConfiguration()[0].setEvictSynchronizedKeyBufferSize(1);
+            cfg.getCacheConfiguration()[0].setEvictionPolicy(new FifoEvictionPolicy(1));
+        }
 
         return cfg;
     }
@@ -51,6 +53,7 @@ public class IgniteCacheP2pUnmarshallingNearErrorTest extends IgniteCacheP2pUnma
 
         //Eviction request unmarshalling failed but ioManager does not hangs up.
 
-        Thread.sleep(1000); //todo: wait for eviction complete
+        // Wait for eviction complete.
+        Thread.sleep(1000);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePartitionMapUpdateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePartitionMapUpdateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePartitionMapUpdateTest.java
new file mode 100644
index 0000000..5b45701
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePartitionMapUpdateTest.java
@@ -0,0 +1,226 @@
+/*
+ * 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.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+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.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCachePartitionMapUpdateTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE1_ATTR = "cache1";
+
+    /** */
+    private static final String CACHE2_ATTR = "cache2";
+
+    /** */
+    private static final String CACHE1 = "cache1";
+
+    /** */
+    private static final String CACHE2 = "cache2";
+
+    /** */
+    private boolean startClientCache;
+
+    /** */
+    private boolean cache1;
+
+    /** */
+    private boolean cache2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setName(CACHE1);
+        ccfg1.setCacheMode(PARTITIONED);
+        ccfg1.setBackups(1);
+        ccfg1.setNodeFilter(new AttributeFilter(CACHE1_ATTR));
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setName(CACHE2);
+        ccfg2.setCacheMode(PARTITIONED);
+        ccfg2.setNodeFilter(new AttributeFilter(CACHE2_ATTR));
+
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        Map<String, String> attrs = new HashMap<>();
+
+        if (cache1)
+            attrs.put(CACHE1_ATTR, "true");
+
+        if (cache1 || startClientCache)
+            ccfgs.add(ccfg1);
+
+        if (cache2)
+            attrs.put(CACHE2_ATTR, "true");
+
+        if (cache2 || startClientCache)
+            ccfgs.add(ccfg2);
+
+        cfg.setUserAttributes(attrs);
+
+        cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMapUpdate1() throws Exception {
+        cache1 = false;
+        cache2 = false;
+
+        startGrid(0);
+
+        cache1 = true;
+        cache2 = false;
+
+        startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        cache1 = false;
+        cache2 = true;
+
+        startGrid(2);
+
+        cache1 = true;
+        cache2 = true;
+
+        startGrid(3);
+
+        awaitPartitionMapExchange();
+
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
+
+        stopGrid(1);
+
+        awaitPartitionMapExchange();
+
+        stopGrid(2);
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionMapUpdate2() throws Exception {
+        startClientCache = true;
+
+        testPartitionMapUpdate1();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandom() throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        final int NODE_CNT = 10;
+
+        for (int iter = 0; iter < 1; iter++) {
+            log.info("Iteration: " + iter);
+
+            for (int i = 0; i < NODE_CNT; i++) {
+                cache1 = rnd.nextBoolean();
+                cache2 = rnd.nextBoolean();
+
+                log.info("Start node [idx=" + i + ", cache1=" + cache1 + ", cache2=" + cache2 + ']');
+
+                startGrid(i);
+
+                awaitPartitionMapExchange();
+            }
+
+            LinkedHashSet<Integer> stopSeq = new LinkedHashSet<>();
+
+            while (stopSeq.size() != NODE_CNT)
+                stopSeq.add(rnd.nextInt(0, NODE_CNT));
+
+            for (Integer idx : stopSeq) {
+                log.info("Stop node: " + idx);
+
+                stopGrid(idx);
+
+                awaitPartitionMapExchange();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandom2() throws Exception {
+        startClientCache = true;
+
+        testRandom();
+    }
+
+    /**
+     *
+     */
+    static class AttributeFilter implements IgnitePredicate<ClusterNode> {
+        /** */
+        private String attrName;
+
+        /**
+         * @param attrName Attribute name.
+         */
+        public AttributeFilter(String attrName) {
+            this.attrName = attrName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return F.eq(node.attribute(attrName), "true");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
index 7cd8414..448f171 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCachePeekModesAbstractTest.java
@@ -93,7 +93,10 @@ public abstract class IgniteCachePeekModesAbstractTest extends IgniteCacheAbstra
         if (hasNearCache())
             ccfg.setNearConfiguration(new NearCacheConfiguration());
 
-        ccfg.setEvictionPolicy(new FifoEvictionPolicy(HEAP_ENTRIES));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(HEAP_ENTRIES);
+
+        ccfg.setEvictionPolicy(plc);
 
         return ccfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
index 095221e..db9e6a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartSelfTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.util.typedef.*;
+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.*;
@@ -68,6 +69,9 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     private boolean testAttribute = true;
 
     /** */
+    private boolean client;
+
+    /** */
     private boolean daemon;
 
     /**
@@ -85,6 +89,12 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
+        if (client) {
+            cfg.setClientMode(true);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        }
+
         cfg.setUserAttributes(F.asMap(TEST_ATTRIBUTE_NAME, testAttribute));
 
         CacheConfiguration cacheCfg = new CacheConfiguration();
@@ -1024,4 +1034,56 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             stopGrid(nodeCount());
         }
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopWithClientJoin() throws Exception {
+        Ignite ignite1 = ignite(1);
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                client = true;
+
+                int iter = 0;
+
+                while (!stop.get()) {
+                    if (iter % 10 == 0)
+                        log.info("Client start/stop iteration: " + iter);
+
+                    iter++;
+
+                    try (Ignite ignite = startGrid(nodeCount())) {
+                        assertTrue(ignite.configuration().isClientMode());
+                    }
+                }
+
+                return null;
+            }
+        }, 1, "client-start-stop");
+
+        try {
+            long stopTime = U.currentTimeMillis() + 30_000;
+
+            int iter = 0;
+
+            while (System.currentTimeMillis() < stopTime) {
+                if (iter % 10 == 0)
+                    log.info("Cache start/stop iteration: " + iter);
+
+                try (IgniteCache<Object, Object> cache = ignite1.getOrCreateCache("cache-" + iter)) {
+                    assertNotNull(cache);
+                }
+
+                iter++;
+            }
+        }
+        finally {
+            stop.set(true);
+        }
+
+        fut.get();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
new file mode 100644
index 0000000..6386f8c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheWithConfigStartSelfTest.java
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheWithConfigStartSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_NAME = "partitioned";
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoSpi);
+
+        if (client)
+            cfg.setCacheConfiguration(cacheConfiguration());
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>(CACHE_NAME);
+
+        ccfg.setIndexedTypes(String.class, String.class);
+
+        return ccfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartCacheOnClient() throws Exception {
+        int srvCnt = 3;
+
+        startGrids(srvCnt);
+
+        try {
+            client = true;
+
+            IgniteEx client = startGrid(srvCnt);
+
+            for (int i = 0; i < 100; i++)
+                client.cache(CACHE_NAME).put(i, i);
+
+            for (int i = 0; i < 100; i++)
+                assertEquals(i, grid(0).cache(CACHE_NAME).get(i));
+
+            client.cache(CACHE_NAME).removeAll();
+
+            for (int i = 0; i < 100; i++)
+                assertNull(grid(0).cache(CACHE_NAME).get(i));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
new file mode 100644
index 0000000..24935c7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicClientCacheStartSelfTest.java
@@ -0,0 +1,283 @@
+/*
+ * 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.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests that cache specified in configuration start on client nodes.
+ */
+public class IgniteDynamicClientCacheStartSelfTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private CacheConfiguration ccfg;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        if (ccfg != null)
+            cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredCacheOnClientNode() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        ccfg = new CacheConfiguration();
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, true);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+
+        assertNotNull(ignite3.cache(cacheName));
+
+        checkCache(ignite3, cacheName, false, false);
+
+        Ignite ignite4 = startGrid(4);
+
+        checkNoCache(ignite4, cacheName);
+
+        assertNotNull(ignite4.createNearCache(cacheName, new NearCacheConfiguration<>()));
+
+        checkCache(ignite4, cacheName, false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearCacheStartError() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite1.getOrCreateNearCache(cacheName, new NearCacheConfiguration<>());
+
+                return null;
+            }
+        }, IgniteException.class, null);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ignite1.createNearCache(cacheName, new NearCacheConfiguration<>());
+
+                return null;
+            }
+        }, IgniteException.class, null);
+
+        checkCache(ignite1, cacheName, false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedCacheClient() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, false);
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, true);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedWithNearCacheClient() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        ccfg.setCacheMode(REPLICATED);
+
+        final String cacheName = null;
+
+        Ignite ignite0 = startGrid(0);
+
+        checkCache(ignite0, cacheName, true, false);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        checkCache(ignite1, cacheName, false, true);
+
+        ccfg.setNearConfiguration(null);
+
+        Ignite ignite2 = startGrid(2);
+
+        checkCache(ignite2, cacheName, false, false);
+
+        ccfg = null;
+
+        Ignite ignite3 = startGrid(3);
+
+        checkNoCache(ignite3, cacheName);
+    }
+
+    /**
+     * @param ignite Node.
+     * @param cacheName Cache name
+     * @param srv {@code True} if server cache is expected.
+     * @param near {@code True} if near cache is expected.
+     */
+    private void checkCache(Ignite ignite, String cacheName, boolean srv, boolean near) {
+        GridCacheAdapter<Object, Object> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
+
+        assertNotNull("No cache on node " + ignite.name(), cache);
+
+        assertEquals(near, cache.context().isNear());
+
+        if (near)
+            cache = ((GridNearCacheAdapter)cache).dht();
+
+        if (srv)
+            assertSame(GridCacheConcurrentMap.class, cache.map().getClass());
+        else
+            assertSame(GridNoStorageCacheMap.class, cache.map().getClass());
+
+        ClusterNode node = ((IgniteKernal)ignite).localNode();
+
+        for (Ignite ignite0 : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite0).context().discovery();
+
+            assertTrue(disco.cacheNode(node, cacheName));
+            assertEquals(srv, disco.cacheAffinityNode(node, cacheName));
+            assertEquals(near, disco.cacheNearNode(node, cacheName));
+
+            if (srv)
+                assertTrue(ignite0.affinity(null).primaryPartitions(node).length > 0);
+            else
+                assertEquals(0, ignite0.affinity(null).primaryPartitions(node).length);
+        }
+
+        assertNotNull(ignite.cache(cacheName));
+    }
+
+    /**
+     * @param ignite Node.
+     * @param cacheName Cache name.
+     */
+    private void checkNoCache(Ignite ignite, String cacheName) {
+        GridCacheAdapter<Object, Object> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
+
+        assertNull("Unexpected cache on node " + ignite.name(), cache);
+
+        ClusterNode node = ((IgniteKernal)ignite).localNode();
+
+        for (Ignite ignite0 : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite0).context().discovery();
+
+            assertFalse(disco.cacheNode(node, cacheName));
+            assertFalse(disco.cacheAffinityNode(node, cacheName));
+            assertFalse(disco.cacheNearNode(node, cacheName));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
new file mode 100644
index 0000000..a7b2df6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteSystemCacheOnClientTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class IgniteSystemCacheOnClientTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSystemCacheOnClientNode() throws Exception {
+        startGrids(2);
+
+        final IgniteKernal ignite = (IgniteKernal)ignite(1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return ignite.internalCache(CU.MARSH_CACHE_NAME) != null;
+            }
+        }, 5000);
+
+        GridCacheAdapter marshCache = ignite.internalCache(CU.MARSH_CACHE_NAME);
+
+        assertNotNull(marshCache);
+
+        assertFalse(marshCache.context().isNear());
+
+        marshCache = ((IgniteKernal)ignite(0)).internalCache(CU.MARSH_CACHE_NAME);
+
+        assertFalse(marshCache.context().isNear());
+
+        Collection<ClusterNode> affNodes = marshCache.affinity().mapKeyToPrimaryAndBackups(1);
+
+        assertEquals(1, affNodes.size());
+        assertTrue(affNodes.contains(ignite(0).cluster().localNode()));
+
+        GridCacheAdapter utilityCache = ((IgniteKernal)ignite(0)).internalCache(CU.UTILITY_CACHE_NAME);
+
+        affNodes = utilityCache.affinity().mapKeyToPrimaryAndBackups(1);
+
+        assertEquals(1, affNodes.size());
+        assertTrue(affNodes.contains(ignite(0).cluster().localNode()));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
index 12397b2..7af0490 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueApiSelfAbstractTest.java
@@ -549,7 +549,7 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
 
         CollectionConfiguration colCfg2 = collectionConfiguration();
 
-        colCfg2.setNodeFilter(CacheConfiguration.SERVER_NODES);
+        colCfg2.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         IgniteQueue queue1 = grid(0).queue("Queue1", 0, colCfg1);
 
@@ -557,7 +557,7 @@ public abstract class GridCacheQueueApiSelfAbstractTest extends IgniteCollection
 
         assertNotSame(getQueueCache(queue1), getQueueCache(queue2));
 
-        colCfg1.setNodeFilter(CacheConfiguration.SERVER_NODES);
+        colCfg1.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         IgniteQueue queue3 = grid(0).queue("Queue3", 0, colCfg1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
new file mode 100644
index 0000000..5a6be8e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -0,0 +1,283 @@
+/*
+ * 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.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.*;
+
+/**
+ *
+ */
+public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1))) {
+            cfg.setClientMode(true);
+
+            if (!clientDiscovery())
+                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        }
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return {@code True} if use client discovery.
+     */
+    protected abstract boolean clientDiscovery();
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSequence() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+
+        try (IgniteAtomicSequence seq = clientNode.atomicSequence("seq1", 1L, true)) {
+            assertNotNull(seq);
+
+            assertEquals(1L, seq.get());
+
+            assertEquals(1L, seq.getAndAdd(1));
+
+            assertEquals(2L, seq.get());
+
+            IgniteAtomicSequence seq0 = srvNode.atomicSequence("seq1", 1L, false);
+
+            assertNotNull(seq0);
+        }
+
+        assertNull(clientNode.atomicSequence("seq1", 1L, false));
+        assertNull(srvNode.atomicSequence("seq1", 1L, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicLong() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.atomicLong("long1", 1L, false));
+
+        try (IgniteAtomicLong cntr = clientNode.atomicLong("long1", 1L, true)) {
+            assertNotNull(cntr);
+
+            assertEquals(1L, cntr.get());
+
+            assertEquals(1L, cntr.getAndAdd(1));
+
+            assertEquals(2L, cntr.get());
+
+            IgniteAtomicLong cntr0 = srvNode.atomicLong("long1", 1L, false);
+
+            assertNotNull(cntr0);
+
+            assertEquals(2L, cntr0.get());
+
+            assertEquals(3L, cntr0.incrementAndGet());
+
+            assertEquals(3L, cntr.get());
+        }
+
+        assertNull(clientNode.atomicLong("long1", 1L, false));
+        assertNull(srvNode.atomicLong("long1", 1L, false));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSet() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        Ignite srvNode = serverNode();
+
+        assertNull(clientNode.set("set1", null));
+
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        try (IgniteSet<Integer> set = clientNode.set("set1", colCfg)) {
+            assertNotNull(set);
+
+            assertEquals(0, set.size());
+
+            assertFalse(set.contains(1));
+
+            assertTrue(set.add(1));
+
+            assertTrue(set.contains(1));
+
+            IgniteSet<Integer> set0 = srvNode.set("set1", null);
+
+            assertTrue(set0.contains(1));
+
+            assertEquals(1, set0.size());
+
+            assertTrue(set0.remove(1));
+
+            assertFalse(set.contains(1));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLatch() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        final Ignite srvNode = serverNode();
+
+        assertNull(clientNode.countDownLatch("latch1", 1, true, false));
+
+        try (IgniteCountDownLatch latch = clientNode.countDownLatch("latch1", 1, true, true)) {
+            assertNotNull(latch);
+
+            assertEquals(1, latch.count());
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    U.sleep(1000);
+
+                    IgniteCountDownLatch latch0 = srvNode.countDownLatch("latch1", 1, true, false);
+
+                    assertEquals(1, latch0.count());
+
+                    log.info("Count down latch.");
+
+                    latch0.countDown();
+
+                    assertEquals(0, latch0.count());
+
+                    return null;
+                }
+            });
+
+            log.info("Await latch.");
+
+            assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
+
+            log.info("Finished wait.");
+
+            fut.get();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueue() throws Exception {
+        Ignite clientNode = clientIgnite();
+
+        final Ignite srvNode = serverNode();
+
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        assertNull(clientNode.queue("q1", 0, null));
+
+        try (IgniteQueue<Integer> queue = clientNode.queue("q1", 0, colCfg)) {
+            assertNotNull(queue);
+
+            queue.add(1);
+
+            assertEquals(1, queue.poll().intValue());
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    U.sleep(1000);
+
+                    IgniteQueue<Integer> queue0 = srvNode.queue("q1", 0, null);
+
+                    assertEquals(0, queue0.size());
+
+                    log.info("Add in queue.");
+
+                    queue0.add(2);
+
+                    return null;
+                }
+            });
+
+            log.info("Try take.");
+
+            assertEquals(2, queue.take().intValue());
+
+            log.info("Finished take.");
+
+            fut.get();
+        }
+    }
+
+    /**
+     * @return Client node.
+     */
+    private Ignite clientIgnite() {
+        Ignite ignite = ignite(NODE_CNT - 1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
+
+        return ignite;
+    }
+
+    /**
+     * @return Server node.
+     */
+    private Ignite serverNode() {
+        Ignite ignite = ignite(0);
+
+        assertFalse(ignite.configuration().isClientMode());
+
+        return ignite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
new file mode 100644
index 0000000..a228cc2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class IgniteClientDataStructuresTest extends IgniteClientDataStructuresAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean clientDiscovery() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
new file mode 100644
index 0000000..bd5cce8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDiscoveryDataStructuresTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ *
+ */
+public class IgniteClientDiscoveryDataStructuresTest extends IgniteClientDataStructuresAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected boolean clientDiscovery() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
index 0f2a898..80e6123 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteCountDownLatchAbstractSelfTest.java
@@ -28,6 +28,7 @@ import org.jetbrains.annotations.*;
 import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
 
 import static java.util.concurrent.TimeUnit.*;
 
@@ -258,6 +259,107 @@ public abstract class IgniteCountDownLatchAbstractSelfTest extends IgniteAtomics
         checkRemovedLatch(latch);
     }
 
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLatchMultinode1() throws Exception {
+        if (gridCount() == 1)
+            return;
+
+        IgniteCountDownLatch latch = grid(0).countDownLatch("l1", 10,
+            true,
+            true);
+
+        List<IgniteInternalFuture<?>> futs = new ArrayList<>();
+
+        final AtomicBoolean countedDown = new AtomicBoolean();
+
+        for (int i = 0; i < gridCount(); i++) {
+            final Ignite ignite = grid(i);
+
+            futs.add(GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgniteCountDownLatch latch = ignite.countDownLatch("l1", 10,
+                        true,
+                        false);
+
+                    assertNotNull(latch);
+
+                    boolean wait = latch.await(30_000);
+
+                    assertTrue(countedDown.get());
+
+                    assertEquals(0, latch.count());
+
+                    assertTrue(wait);
+
+                    return null;
+                }
+            }));
+        }
+
+        for (int i = 0; i < 10; i++) {
+            if (i == 9)
+                countedDown.set(true);
+
+            latch.countDown();
+        }
+
+        for (IgniteInternalFuture<?> fut : futs)
+            fut.get(30_000);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLatchMultinode2() throws Exception {
+        if (gridCount() == 1)
+            return;
+
+        IgniteCountDownLatch latch = grid(0).countDownLatch("l2", gridCount() * 3,
+            true,
+            true);
+
+        assertNotNull(latch);
+
+        List<IgniteInternalFuture<?>> futs = new ArrayList<>();
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        for (int i = 0; i < gridCount(); i++) {
+            final Ignite ignite = grid(i);
+
+            futs.add(GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    IgniteCountDownLatch latch = ignite.countDownLatch("l2", 10,
+                        true,
+                        false);
+
+                    assertNotNull(latch);
+
+                    for (int i = 0; i < 3; i++) {
+                        cnt.incrementAndGet();
+
+                        latch.countDown();
+                    }
+
+                    boolean wait = latch.await(30_000);
+
+                    assertEquals(gridCount() * 3, cnt.get());
+
+                    assertEquals(0, latch.count());
+
+                    assertTrue(wait);
+
+                    return null;
+                }
+            }));
+        }
+
+        for (IgniteInternalFuture<?> fut : futs)
+            fut.get(30_000);
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         // No-op.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
index 85256b4..f996568 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
@@ -21,10 +21,11 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 
+import java.io.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.cache.CacheMode.*;
@@ -50,20 +51,24 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
 
         super.beforeTestsStarted();
 
-        if (!clientOnly())
-            grid(nearOnlyGridName).createNearCache(null, new NearCacheConfiguration());
+        if (nearEnabled())
+            grid(nearOnlyGridName).createNearCache(null, nearConfiguration());
     }
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (gridCnt.getAndIncrement() == 0) {
+        int cnt = gridCnt.incrementAndGet();
+
+        if ((cnt == gridCount() && isClientStartedLast()) || (cnt == 1 && !isClientStartedLast())) {
             cfg.setClientMode(true);
 
             nearOnlyGridName = gridName;
         }
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         return cfg;
     }
 
@@ -75,17 +80,14 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
         cfg.setCacheStoreFactory(null);
         cfg.setReadThrough(false);
         cfg.setWriteThrough(false);
-        cfg.setAffinity(new RendezvousAffinityFunction(false, 32));
         cfg.setBackups(1);
 
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        super.afterTestsStopped();
+        if (cfg.getCacheMode() == REPLICATED)
+            cfg.setAffinity(null);
+        else
+            cfg.setAffinity(new RendezvousAffinityFunction(false, 32));
 
-        gridCnt.set(0);
+        return cfg;
     }
 
     /** {@inheritDoc} */
@@ -94,9 +96,11 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
     }
 
     /**
-     * @return If {@code true} then uses CLIENT_ONLY mode, otherwise NEAR_ONLY.
+     * @return boolean {@code True} if client's grid must be started last, {@code false} if it must be started first.
      */
-    protected abstract boolean clientOnly();
+    protected boolean isClientStartedLast() {
+        return false;
+    }
 
     /**
      * @throws Exception If failed.
@@ -110,7 +114,7 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
         nearOnly.putAll(F.asMap(5, 5, 6, 6, 7, 7, 8, 8, 9, 9));
 
         for (int key = 0; key < 10; key++) {
-            for (int i = 1; i < gridCount(); i++) {
+            for (int i = 0; i < gridCount(); i++) {
                 if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key))
                     assertEquals(key, grid(i).cache(null).localPeek(key, CachePeekMode.ONHEAP));
             }
@@ -120,6 +124,24 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
 
             assertNull(nearOnly.localPeek(key, CachePeekMode.PRIMARY, CachePeekMode.BACKUP));
         }
+
+        Integer key = 1000;
+
+        nearOnly.put(key, new TestClass1(key));
+
+        if (nearEnabled())
+            assertNotNull(nearOnly.localPeek(key, CachePeekMode.ALL));
+        else
+            assertNull(nearOnly.localPeek(key, CachePeekMode.ALL));
+
+        for (int i = 0; i < gridCount(); i++) {
+            if (grid(i).affinity(null).isPrimaryOrBackup(grid(i).localNode(), key)) {
+                TestClass1 val = (TestClass1)grid(i).cache(null).localPeek(key, CachePeekMode.ONHEAP);
+
+                assertNotNull(val);
+                assertEquals(key.intValue(), val.val);
+            }
+        }
     }
 
     /**
@@ -147,6 +169,18 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
             if (nearEnabled())
                 assertEquals(key, nearOnly.localPeek(key, CachePeekMode.ONHEAP));
         }
+
+        Integer key = 2000;
+
+        dht.put(key, new TestClass2(key));
+
+        TestClass2 val = (TestClass2)nearOnly.get(key);
+
+        assertNotNull(val);
+        assertEquals(key.intValue(), val.val);
+
+        if (nearEnabled())
+            assertNotNull(nearOnly.localPeek(key, CachePeekMode.ONHEAP));
     }
 
     /**
@@ -210,4 +244,34 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
 
         return null;
     }
+
+    /**
+     *
+     */
+    static class TestClass1 implements Serializable {
+        /** */
+        int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass1(int val) {
+            this.val = val;
+        }
+    }
+
+    /**
+     *
+     */
+    static class TestClass2 implements Serializable {
+        /** */
+        int val;
+
+        /**
+         * @param val Value.
+         */
+        public TestClass2(int val) {
+            this.val = val;
+        }
+    }
 }


[16/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
new file mode 100644
index 0000000..0270a7c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -0,0 +1,4733 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.events.*;
+import org.apache.ignite.internal.processors.security.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.net.*;
+import java.text.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
+import static org.apache.ignite.spi.IgnitePortProtocol.*;
+import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.*;
+import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage.*;
+
+/**
+ *
+ */
+@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
+class ServerImpl extends TcpDiscoveryImpl {
+    /** */
+    private final Executor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>());
+
+    /** Nodes ring. */
+    @GridToStringExclude
+    private final TcpDiscoveryNodesRing ring = new TcpDiscoveryNodesRing();
+
+    /** Topology snapshots history. */
+    private final SortedMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
+
+    /** Socket readers. */
+    private final Collection<SocketReader> readers = new LinkedList<>();
+
+    /** TCP server for discovery SPI. */
+    private TcpServer tcpSrvr;
+
+    /** Message worker. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private RingMessageWorker msgWorker;
+
+    /** Client message workers. */
+    protected ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
+
+    /** Metrics sender. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private HeartbeatsSender hbsSnd;
+
+    /** Status checker. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private CheckStatusSender chkStatusSnd;
+
+    /** IP finder cleaner. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private IpFinderCleaner ipFinderCleaner;
+
+    /** Statistics printer thread. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private StatisticsPrinter statsPrinter;
+
+    /** Failed nodes (but still in topology). */
+    private Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
+
+    /** Leaving nodes (but still in topology). */
+    private Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
+
+    /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */
+    private boolean ipFinderHasLocAddr;
+
+    /** Addresses that do not respond during join requests send (for resolving concurrent start). */
+    private final Collection<SocketAddress> noResAddrs = new GridConcurrentHashSet<>();
+
+    /** Addresses that incoming join requests send were send from (for resolving concurrent start). */
+    private final Collection<SocketAddress> fromAddrs = new GridConcurrentHashSet<>();
+
+    /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
+    private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
+
+    /** Mutex. */
+    private final Object mux = new Object();
+
+    /** Discovery state. */
+    protected TcpDiscoverySpiState spiState = DISCONNECTED;
+
+    /** Map with proceeding ping requests. */
+    private final ConcurrentMap<InetSocketAddress, IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
+        new ConcurrentHashMap8<>();
+
+    /** Debug mode. */
+    private boolean debugMode;
+
+    /** Debug messages history. */
+    private int debugMsgHist = 512;
+
+    /** Received messages. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private ConcurrentLinkedDeque<String> debugLog;
+
+    /**
+     * @param adapter Adapter.
+     */
+    ServerImpl(TcpDiscoverySpi adapter) {
+        super(adapter);
+    }
+
+    /**
+     * This method is intended for troubleshooting purposes only.
+     *
+     * @param debugMode {code True} to start SPI in debug mode.
+     */
+    public void setDebugMode(boolean debugMode) {
+        this.debugMode = debugMode;
+    }
+
+    /**
+     * This method is intended for troubleshooting purposes only.
+     *
+     * @param debugMsgHist Message history log size.
+     */
+    public void setDebugMessageHistory(int debugMsgHist) {
+        this.debugMsgHist = debugMsgHist;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+        synchronized (mux) {
+            return spiState.name();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return msgWorker.queueSize();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID getCoordinator() {
+        TcpDiscoveryNode crd = resolveCoordinator();
+
+        return crd != null ? crd.id() : null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        assert nodeId != null;
+
+        UUID locNodeId0 = getLocalNodeId();
+
+        if (locNodeId0 != null && locNodeId0.equals(nodeId))
+            // Return local node directly.
+            return locNode;
+
+        TcpDiscoveryNode node = ring.node(nodeId);
+
+        if (node != null && !node.visible())
+            return null;
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return F.upcast(ring.visibleRemoteNodes());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(String gridName) throws IgniteSpiException {
+        synchronized (mux) {
+            spiState = DISCONNECTED;
+        }
+
+        if (debugMode) {
+            if (!log.isInfoEnabled())
+                throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
+                    "in debug mode.");
+
+            debugLog = new ConcurrentLinkedDeque<>();
+
+            U.quietAndWarn(log, "TCP discovery SPI is configured in debug mode.");
+        }
+
+        // Clear addresses collections.
+        fromAddrs.clear();
+        noResAddrs.clear();
+
+        msgWorker = new RingMessageWorker();
+        msgWorker.start();
+
+        tcpSrvr = new TcpServer();
+
+        spi.initLocalNode(tcpSrvr.port, true);
+
+        locNode = spi.locNode;
+
+        // Start TCP server thread after local node is initialized.
+        tcpSrvr.start();
+
+        ring.localNode(locNode);
+
+        if (spi.ipFinder.isShared())
+            registerLocalNodeAddress();
+        else {
+            if (F.isEmpty(spi.ipFinder.getRegisteredAddresses()))
+                throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
+                    "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
+                    "(specify list of IP addresses in configuration).");
+
+            ipFinderHasLocAddr = spi.ipFinderHasLocalAddress();
+        }
+
+        if (spi.getStatisticsPrintFrequency() > 0 && log.isInfoEnabled()) {
+            statsPrinter = new StatisticsPrinter();
+            statsPrinter.start();
+        }
+
+        spi.stats.onJoinStarted();
+
+        joinTopology();
+
+        spi.stats.onJoinFinished();
+
+        hbsSnd = new HeartbeatsSender();
+        hbsSnd.start();
+
+        chkStatusSnd = new CheckStatusSender();
+        chkStatusSnd.start();
+
+        if (spi.ipFinder.isShared()) {
+            ipFinderCleaner = new IpFinderCleaner();
+            ipFinderCleaner.start();
+        }
+
+        spi.printStartInfo();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        spiCtx.registerPort(tcpSrvr.port, TCP);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        spiStop0(false);
+    }
+
+    /**
+     * Stops SPI finally or stops SPI for restart.
+     *
+     * @param disconnect {@code True} if SPI is being disconnected.
+     * @throws IgniteSpiException If failed.
+     */
+    private void spiStop0(boolean disconnect) throws IgniteSpiException {
+        if (log.isDebugEnabled()) {
+            if (disconnect)
+                log.debug("Disconnecting SPI.");
+            else
+                log.debug("Preparing to start local node stop procedure.");
+        }
+
+        if (disconnect) {
+            synchronized (mux) {
+                spiState = DISCONNECTING;
+            }
+        }
+
+        if (msgWorker != null && msgWorker.isAlive() && !disconnect) {
+            // Send node left message only if it is final stop.
+            msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(locNode.id()));
+
+            synchronized (mux) {
+                long threshold = U.currentTimeMillis() + spi.netTimeout;
+
+                long timeout = spi.netTimeout;
+
+                while (spiState != LEFT && timeout > 0) {
+                    try {
+                        mux.wait(timeout);
+
+                        timeout = threshold - U.currentTimeMillis();
+                    }
+                    catch (InterruptedException ignored) {
+                        Thread.currentThread().interrupt();
+
+                        break;
+                    }
+                }
+
+                if (spiState == LEFT) {
+                    if (log.isDebugEnabled())
+                        log.debug("Verification for local node leave has been received from coordinator" +
+                            " (continuing stop procedure).");
+                }
+                else if (log.isInfoEnabled()) {
+                    log.info("No verification for local node leave has been received from coordinator" +
+                        " (will stop node anyway).");
+                }
+            }
+        }
+
+        U.interrupt(tcpSrvr);
+        U.join(tcpSrvr, log);
+
+        Collection<SocketReader> tmp;
+
+        synchronized (mux) {
+            tmp = U.arrayList(readers);
+        }
+
+        U.interrupt(tmp);
+        U.joinThreads(tmp, log);
+
+        U.interrupt(hbsSnd);
+        U.join(hbsSnd, log);
+
+        U.interrupt(chkStatusSnd);
+        U.join(chkStatusSnd, log);
+
+        U.interrupt(ipFinderCleaner);
+        U.join(ipFinderCleaner, log);
+
+        U.interrupt(msgWorker);
+        U.join(msgWorker, log);
+
+        U.interrupt(statsPrinter);
+        U.join(statsPrinter, log);
+
+        Collection<TcpDiscoveryNode> rmts = null;
+
+        if (!disconnect)
+            spi.printStopInfo();
+        else {
+            spi.getSpiContext().deregisterPorts();
+
+            rmts = ring.visibleRemoteNodes();
+        }
+
+        long topVer = ring.topologyVersion();
+
+        ring.clear();
+
+        if (rmts != null && !rmts.isEmpty()) {
+            // This is restart/disconnection and remote nodes are not empty.
+            // We need to fire FAIL event for each.
+            DiscoverySpiListener lsnr = spi.lsnr;
+
+            if (lsnr != null) {
+                Collection<ClusterNode> processed = new HashSet<>();
+
+                for (TcpDiscoveryNode n : rmts) {
+                    assert n.visible();
+
+                    processed.add(n);
+
+                    List<ClusterNode> top = U.arrayList(rmts, F.notIn(processed));
+
+                    topVer++;
+
+                    Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer,
+                        Collections.unmodifiableList(top));
+
+                    lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, hist, null);
+                }
+            }
+        }
+
+        printStatistics();
+
+        spi.stats.clear();
+
+        synchronized (mux) {
+            // Clear stored data.
+            leavingNodes.clear();
+            failedNodes.clear();
+
+            spiState = DISCONNECTED;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        assert nodeId != null;
+
+        if (log.isDebugEnabled())
+            log.debug("Pinging node: " + nodeId + "].");
+
+        if (nodeId == getLocalNodeId())
+            return true;
+
+        TcpDiscoveryNode node = ring.node(nodeId);
+
+        if (node == null || !node.visible())
+            return false;
+
+        boolean res = pingNode(node);
+
+        if (!res && !node.isClient()) {
+            LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
+
+            msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
+        }
+
+        return res;
+    }
+
+    /**
+     * Pings the remote node to see if it's alive.
+     *
+     * @param node Node.
+     * @return {@code True} if ping succeeds.
+     */
+    private boolean pingNode(TcpDiscoveryNode node) {
+        assert node != null;
+
+        if (node.id().equals(getLocalNodeId()))
+            return true;
+
+        UUID clientNodeId = null;
+
+        if (node.isClient()) {
+            clientNodeId = node.id();
+
+            node = ring.node(node.clientRouterNodeId());
+
+            if (node == null || !node.visible())
+                return false;
+        }
+
+        for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
+            try {
+                // ID returned by the node should be the same as ID of the parameter for ping to succeed.
+                IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
+
+                return node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
+            }
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']');
+
+                onException("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']', e);
+                // continue;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Pings the node by its address to see if it's alive.
+     *
+     * @param addr Address of the node.
+     * @return ID of the remote node and "client exists" flag if node alive.
+     * @throws IgniteSpiException If an error occurs.
+     */
+    private IgniteBiTuple<UUID, Boolean> pingNode(InetSocketAddress addr, @Nullable UUID clientNodeId)
+        throws IgniteCheckedException {
+        assert addr != null;
+
+        UUID locNodeId = getLocalNodeId();
+
+        if (F.contains(spi.locNodeAddrs, addr)) {
+            if (clientNodeId == null)
+                return F.t(getLocalNodeId(), false);
+
+            ClientMessageWorker clientWorker = clientMsgWorkers.get(clientNodeId);
+
+            if (clientWorker == null)
+                return F.t(getLocalNodeId(), false);
+
+            boolean clientPingRes;
+
+            try {
+                clientPingRes = clientWorker.ping();
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new IgniteInterruptedCheckedException(e);
+            }
+
+            return F.t(getLocalNodeId(), clientPingRes);
+        }
+
+        GridFutureAdapter<IgniteBiTuple<UUID, Boolean>> fut = new GridFutureAdapter<>();
+
+        IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
+
+        if (oldFut != null)
+            return oldFut.get();
+        else {
+            Collection<Throwable> errs = null;
+
+            try {
+                Socket sock = null;
+
+                for (int i = 0; i < spi.reconCnt; i++) {
+                    try {
+                        if (addr.isUnresolved())
+                            addr = new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort());
+
+                        long tstamp = U.currentTimeMillis();
+
+                        sock = spi.openSocket(addr);
+
+                        spi.writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
+
+                        TcpDiscoveryPingResponse res = spi.readMessage(sock, null, spi.netTimeout);
+
+                        if (locNodeId.equals(res.creatorNodeId())) {
+                            if (log.isDebugEnabled())
+                                log.debug("Ping response from local node: " + res);
+
+                            break;
+                        }
+
+                        spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                        IgniteBiTuple<UUID, Boolean> t = F.t(res.creatorNodeId(), res.clientExists());
+
+                        fut.onDone(t);
+
+                        return t;
+                    }
+                    catch (IOException | IgniteCheckedException e) {
+                        if (errs == null)
+                            errs = new ArrayList<>();
+
+                        errs.add(e);
+                    }
+                    finally {
+                        U.closeQuiet(sock);
+                    }
+                }
+            }
+            catch (Throwable t) {
+                fut.onDone(t);
+
+                if (t instanceof Error)
+                    throw t;
+
+                throw U.cast(t);
+            }
+            finally {
+                if (!fut.isDone())
+                    fut.onDone(U.exceptionWithSuppressed("Failed to ping node by address: " + addr, errs));
+
+                boolean b = pingMap.remove(addr, fut);
+
+                assert b;
+            }
+
+            return fut.get();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        spiStop0(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
+        try {
+            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, spi.marsh.marshal(evt)));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        ClusterNode node = ring.node(nodeId);
+
+        if (node != null) {
+            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
+                node.id(), node.order());
+
+            msgWorker.addMessage(msg);
+        }
+    }
+
+    /**
+     * Tries to join this node to topology.
+     *
+     * @throws IgniteSpiException If any error occurs.
+     */
+    private void joinTopology() throws IgniteSpiException {
+        synchronized (mux) {
+            assert spiState == CONNECTING || spiState == DISCONNECTED;
+
+            spiState = CONNECTING;
+        }
+
+        SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes()
+            .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+        // Marshal credentials for backward compatibility and security.
+        marshalCredentials(locNode);
+
+        while (true) {
+            if (!sendJoinRequestMessage()) {
+                if (log.isDebugEnabled())
+                    log.debug("Join request message has not been sent (local node is the first in the topology).");
+
+                if (spi.nodeAuth != null) {
+                    // Authenticate local node.
+                    try {
+                        SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
+
+                        if (subj == null)
+                            throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
+
+                        Map<String, Object> attrs = new HashMap<>(locNode.attributes());
+
+                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
+                            spi.ignite().configuration().getMarshaller().marshal(subj));
+                        attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+                        locNode.setAttributes(attrs);
+                    }
+                    catch (IgniteException | IgniteCheckedException e) {
+                        throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
+                    }
+                }
+
+                locNode.order(1);
+                locNode.internalOrder(1);
+
+                spi.gridStartTime = U.currentTimeMillis();
+
+                locNode.visible(true);
+
+                ring.clear();
+
+                ring.topologyVersion(1);
+
+                synchronized (mux) {
+                    topHist.clear();
+
+                    spiState = CONNECTED;
+
+                    mux.notifyAll();
+                }
+
+                notifyDiscovery(EVT_NODE_JOINED, 1, locNode);
+
+                break;
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("Join request message has been sent (waiting for coordinator response).");
+
+            synchronized (mux) {
+                long threshold = U.currentTimeMillis() + spi.netTimeout;
+
+                long timeout = spi.netTimeout;
+
+                while (spiState == CONNECTING && timeout > 0) {
+                    try {
+                        mux.wait(timeout);
+
+                        timeout = threshold - U.currentTimeMillis();
+                    }
+                    catch (InterruptedException ignored) {
+                        Thread.currentThread().interrupt();
+
+                        throw new IgniteSpiException("Thread has been interrupted.");
+                    }
+                }
+
+                if (spiState == CONNECTED)
+                    break;
+                else if (spiState == DUPLICATE_ID)
+                    throw spi.duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
+                else if (spiState == AUTH_FAILED)
+                    throw spi.authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
+                else if (spiState == CHECK_FAILED)
+                    throw spi.checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
+                else if (spiState == LOOPBACK_PROBLEM) {
+                    TcpDiscoveryLoopbackProblemMessage msg = (TcpDiscoveryLoopbackProblemMessage)joinRes.get();
+
+                    boolean locHostLoopback = spi.locHost.isLoopbackAddress();
+
+                    String firstNode = locHostLoopback ? "local" : "remote";
+
+                    String secondNode = locHostLoopback ? "remote" : "local";
+
+                    throw new IgniteSpiException("Failed to add node to topology because " + firstNode +
+                        " node is configured to use loopback address, but " + secondNode + " node is not " +
+                        "(consider changing 'localAddress' configuration parameter) " +
+                        "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" +
+                        U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
+                }
+                else
+                    LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
+                        "Check remote nodes logs for possible error messages. " +
+                        "Note that large topology may require significant time to start. " +
+                        "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
+                        "if getting this message on the starting nodes [networkTimeout=" + spi.netTimeout + ']');
+            }
+        }
+
+        assert locNode.order() != 0;
+        assert locNode.internalOrder() != 0;
+
+        if (log.isDebugEnabled())
+            log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder());
+    }
+
+    /**
+     * Tries to send join request message to a random node presenting in topology.
+     * Address is provided by {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} and message is
+     * sent to first node connection succeeded to.
+     *
+     * @return {@code true} if send succeeded.
+     * @throws IgniteSpiException If any error occurs.
+     */
+    @SuppressWarnings({"BusyWait"})
+    private boolean sendJoinRequestMessage() throws IgniteSpiException {
+        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
+            spi.collectExchangeData(getLocalNodeId()));
+
+        // Time when it has been detected, that addresses from IP finder do not respond.
+        long noResStart = 0;
+
+        while (true) {
+            Collection<InetSocketAddress> addrs = spi.resolvedAddresses();
+
+            if (F.isEmpty(addrs))
+                return false;
+
+            boolean retry = false;
+            Collection<Exception> errs = new ArrayList<>();
+
+            for (InetSocketAddress addr : addrs) {
+                try {
+                    Integer res = sendMessageDirectly(joinReq, addr);
+
+                    assert res != null;
+
+                    noResAddrs.remove(addr);
+
+                    // Address is responsive, reset period start.
+                    noResStart = 0;
+
+                    switch (res) {
+                        case RES_WAIT:
+                            // Concurrent startup, try sending join request again or wait if no success.
+                            retry = true;
+
+                            break;
+                        case RES_OK:
+                            if (log.isDebugEnabled())
+                                log.debug("Join request message has been sent to address [addr=" + addr +
+                                    ", req=" + joinReq + ']');
+
+                            // Join request sending succeeded, wait for response from topology.
+                            return true;
+
+                        default:
+                            // Concurrent startup, try next node.
+                            if (res == RES_CONTINUE_JOIN) {
+                                if (!fromAddrs.contains(addr))
+                                    retry = true;
+                            }
+                            else {
+                                if (log.isDebugEnabled())
+                                    log.debug("Unexpected response to join request: " + res);
+
+                                retry = true;
+                            }
+
+                            break;
+                    }
+                }
+                catch (IgniteSpiException e) {
+                    errs.add(e);
+
+                    if (log.isDebugEnabled()) {
+                        IOException ioe = X.cause(e, IOException.class);
+
+                        log.debug("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']');
+
+                        onException("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']', ioe);
+                    }
+
+                    noResAddrs.add(addr);
+                }
+            }
+
+            if (retry) {
+                if (log.isDebugEnabled())
+                    log.debug("Concurrent discovery SPI start has been detected (local node should wait).");
+
+                try {
+                    U.sleep(2000);
+                }
+                catch (IgniteInterruptedCheckedException e) {
+                    throw new IgniteSpiException("Thread has been interrupted.", e);
+                }
+            }
+            else if (!spi.ipFinder.isShared() && !ipFinderHasLocAddr) {
+                IgniteCheckedException e = null;
+
+                if (!errs.isEmpty()) {
+                    e = new IgniteCheckedException("Multiple connection attempts failed.");
+
+                    for (Exception err : errs)
+                        e.addSuppressed(err);
+                }
+
+                if (e != null && X.hasCause(e, ConnectException.class))
+                    LT.warn(log, null, "Failed to connect to any address from IP finder " +
+                        "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
+                        addrs);
+
+                if (spi.joinTimeout > 0) {
+                    if (noResStart == 0)
+                        noResStart = U.currentTimeMillis();
+                    else if (U.currentTimeMillis() - noResStart > spi.joinTimeout)
+                        throw new IgniteSpiException(
+                            "Failed to connect to any address from IP finder within join timeout " +
+                                "(make sure IP finder addresses are correct, and operating system firewalls are disabled " +
+                                "on all host machines, or consider increasing 'joinTimeout' configuration property): " +
+                                addrs, e);
+                }
+
+                try {
+                    U.sleep(2000);
+                }
+                catch (IgniteInterruptedCheckedException ex) {
+                    throw new IgniteSpiException("Thread has been interrupted.", ex);
+                }
+            }
+            else
+                break;
+        }
+
+        return false;
+    }
+
+    /**
+     * Establishes connection to an address, sends message and returns the response (if any).
+     *
+     * @param msg Message to send.
+     * @param addr Address to send message to.
+     * @return Response read from the recipient or {@code null} if no response is supposed.
+     * @throws IgniteSpiException If an error occurs.
+     */
+    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr)
+        throws IgniteSpiException {
+        assert msg != null;
+        assert addr != null;
+
+        Collection<Throwable> errs = null;
+
+        long ackTimeout0 = spi.ackTimeout;
+
+        int connectAttempts = 1;
+
+        boolean joinReqSent = false;
+
+        UUID locNodeId = getLocalNodeId();
+
+        for (int i = 0; i < spi.reconCnt; i++) {
+            // Need to set to false on each new iteration,
+            // since remote node may leave in the middle of the first iteration.
+            joinReqSent = false;
+
+            boolean openSock = false;
+
+            Socket sock = null;
+
+            try {
+                long tstamp = U.currentTimeMillis();
+
+                sock = spi.openSocket(addr);
+
+                openSock = true;
+
+                // Handshake.
+                spi.writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
+
+                TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, ackTimeout0);
+
+                if (locNodeId.equals(res.creatorNodeId())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Handshake response from local node: " + res);
+
+                    break;
+                }
+
+                spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                // Send message.
+                tstamp = U.currentTimeMillis();
+
+                spi.writeToSocket(sock, msg);
+
+                spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+
+                if (debugMode)
+                    debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
+                        ", rmtNodeId=" + res.creatorNodeId() + ']');
+
+                if (log.isDebugEnabled())
+                    log.debug("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
+                        ", rmtNodeId=" + res.creatorNodeId() + ']');
+
+                // Connection has been established, but
+                // join request may not be unmarshalled on remote host.
+                // E.g. due to class not found issue.
+                joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
+
+                return spi.readReceipt(sock, ackTimeout0);
+            }
+            catch (ClassCastException e) {
+                // This issue is rarely reproducible on AmazonEC2, but never
+                // on dedicated machines.
+                if (log.isDebugEnabled())
+                    U.error(log, "Class cast exception on direct send: " + addr, e);
+
+                onException("Class cast exception on direct send: " + addr, e);
+
+                if (errs == null)
+                    errs = new ArrayList<>();
+
+                errs.add(e);
+            }
+            catch (IOException | IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.error("Exception on direct send: " + e.getMessage(), e);
+
+                onException("Exception on direct send: " + e.getMessage(), e);
+
+                if (errs == null)
+                    errs = new ArrayList<>();
+
+                errs.add(e);
+
+                if (!openSock) {
+                    // Reconnect for the second time, if connection is not established.
+                    if (connectAttempts < 2) {
+                        connectAttempts++;
+
+                        continue;
+                    }
+
+                    break; // Don't retry if we can not establish connection.
+                }
+
+                if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
+                    ackTimeout0 *= 2;
+
+                    if (!checkAckTimeout(ackTimeout0))
+                        break;
+                }
+            }
+            finally {
+                U.closeQuiet(sock);
+            }
+        }
+
+        if (joinReqSent) {
+            if (log.isDebugEnabled())
+                log.debug("Join request has been sent, but receipt has not been read (returning RES_WAIT).");
+
+            // Topology will not include this node,
+            // however, warning on timed out join will be output.
+            return RES_OK;
+        }
+
+        throw new IgniteSpiException(
+            "Failed to send message to address [addr=" + addr + ", msg=" + msg + ']',
+            U.exceptionWithSuppressed("Failed to send message to address " +
+                "[addr=" + addr + ", msg=" + msg + ']', errs));
+    }
+
+    /**
+     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+     *
+     * @param node Node to marshall credentials for.
+     * @throws IgniteSpiException If marshalling failed.
+     */
+    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            // Use security-unsafe getter.
+            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
+                spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+
+            node.setAttributes(attrs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+        }
+    }
+
+    /**
+     * Unmarshalls credentials with discovery SPI marshaller (will not replace attribute value).
+     *
+     * @param node Node to unmarshall credentials for.
+     * @return Security credentials.
+     * @throws IgniteSpiException If unmarshal fails.
+     */
+    private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            byte[] credBytes = (byte[])node.getAttributes().get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+            if (credBytes == null)
+                return null;
+
+            return spi.marsh.unmarshal(credBytes, null);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e);
+        }
+    }
+
+    /**
+     * @param ackTimeout Acknowledgement timeout.
+     * @return {@code True} if acknowledgement timeout is less or equal to
+     * maximum acknowledgement timeout, {@code false} otherwise.
+     */
+    private boolean checkAckTimeout(long ackTimeout) {
+        if (ackTimeout > spi.maxAckTimeout) {
+            LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
+                "(consider increasing 'maxAckTimeout' configuration property) " +
+                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + spi.maxAckTimeout + ']');
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Notify external listener on discovery event.
+     *
+     * @param type Discovery event type. See {@link org.apache.ignite.events.DiscoveryEvent} for more details.
+     * @param topVer Topology version.
+     * @param node Remote node this event is connected with.
+     */
+    private void notifyDiscovery(int type, long topVer, TcpDiscoveryNode node) {
+        assert type > 0;
+        assert node != null;
+
+        DiscoverySpiListener lsnr = spi.lsnr;
+
+        TcpDiscoverySpiState spiState = spiStateCopy();
+
+        if (lsnr != null && node.visible() && (spiState == CONNECTED || spiState == DISCONNECTING)) {
+            if (log.isDebugEnabled())
+                log.debug("Discovery notification [node=" + node + ", spiState=" + spiState +
+                    ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
+
+            Collection<ClusterNode> top = F.upcast(ring.visibleNodes());
+
+            Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer, top);
+
+            lsnr.onDiscovery(type, topVer, node, top, hist, null);
+        }
+        else if (log.isDebugEnabled())
+            log.debug("Skipped discovery notification [node=" + node + ", spiState=" + spiState +
+                ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
+    }
+
+    /**
+     * Update topology history with new topology snapshots.
+     *
+     * @param topVer Topology version.
+     * @param top Topology snapshot.
+     * @return Copy of updated topology history.
+     */
+    @Nullable private Map<Long, Collection<ClusterNode>> updateTopologyHistory(long topVer, Collection<ClusterNode> top) {
+        synchronized (mux) {
+            if (topHist.containsKey(topVer))
+                return null;
+
+            topHist.put(topVer, top);
+
+            while (topHist.size() > spi.topHistSize)
+                topHist.remove(topHist.firstKey());
+
+            if (log.isDebugEnabled())
+                log.debug("Added topology snapshot to history, topVer=" + topVer + ", historySize=" + topHist.size());
+
+            return new TreeMap<>(topHist);
+        }
+    }
+
+    /**
+     * Checks whether local node is coordinator. Nodes that are leaving or failed
+     * (but are still in topology) are removed from search.
+     *
+     * @return {@code true} if local node is coordinator.
+     */
+    private boolean isLocalNodeCoordinator() {
+        synchronized (mux) {
+            boolean crd = spiState == CONNECTED && locNode.equals(resolveCoordinator());
+
+            if (crd)
+                spi.stats.onBecomingCoordinator();
+
+            return crd;
+        }
+    }
+
+    /**
+     * @return Spi state copy.
+     */
+    private TcpDiscoverySpiState spiStateCopy() {
+        TcpDiscoverySpiState state;
+
+        synchronized (mux) {
+            state = spiState;
+        }
+
+        return state;
+    }
+
+    /**
+     * Resolves coordinator. Nodes that are leaving or failed (but are still in
+     * topology) are removed from search.
+     *
+     * @return Coordinator node or {@code null} if there are no coordinator
+     * (i.e. local node is the last one and is currently stopping).
+     */
+    @Nullable private TcpDiscoveryNode resolveCoordinator() {
+        return resolveCoordinator(null);
+    }
+
+    /**
+     * Resolves coordinator. Nodes that are leaving or failed (but are still in
+     * topology) are removed from search as well as provided filter.
+     *
+     * @param filter Nodes to exclude when resolving coordinator (optional).
+     * @return Coordinator node or {@code null} if there are no coordinator
+     * (i.e. local node is the last one and is currently stopping).
+     */
+    @Nullable private TcpDiscoveryNode resolveCoordinator(
+        @Nullable Collection<TcpDiscoveryNode> filter) {
+        synchronized (mux) {
+            Collection<TcpDiscoveryNode> excluded = F.concat(false, failedNodes, leavingNodes);
+
+            if (!F.isEmpty(filter))
+                excluded = F.concat(false, excluded, filter);
+
+            return ring.coordinator(excluded);
+        }
+    }
+
+    /**
+     * Prints SPI statistics.
+     */
+    private void printStatistics() {
+        if (log.isInfoEnabled() && spi.statsPrintFreq > 0) {
+            int failedNodesSize;
+            int leavingNodesSize;
+
+            synchronized (mux) {
+                failedNodesSize = failedNodes.size();
+                leavingNodesSize = leavingNodes.size();
+            }
+
+            Runtime runtime = Runtime.getRuntime();
+
+            TcpDiscoveryNode coord = resolveCoordinator();
+
+            log.info("Discovery SPI statistics [statistics=" + spi.stats + ", spiState=" + spiStateCopy() +
+                ", coord=" + coord +
+                ", topSize=" + ring.allNodes().size() +
+                ", leavingNodesSize=" + leavingNodesSize + ", failedNodesSize=" + failedNodesSize +
+                ", msgWorker.queue.size=" + (msgWorker != null ? msgWorker.queueSize() : "N/A") +
+                ", lastUpdate=" + (locNode != null ? U.format(locNode.lastUpdateTime()) : "N/A") +
+                ", heapFree=" + runtime.freeMemory() / (1024 * 1024) +
+                "M, heapTotal=" + runtime.maxMemory() / (1024 * 1024) + "M]");
+        }
+    }
+
+    /**
+     * @param msg Message to prepare.
+     * @param destNodeId Destination node ID.
+     * @param msgs Messages to include.
+     * @param discardMsgId Discarded message ID.
+     */
+    private void prepareNodeAddedMessage(TcpDiscoveryAbstractMessage msg, UUID destNodeId,
+        @Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardMsgId) {
+        assert destNodeId != null;
+
+        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
+
+            TcpDiscoveryNode node = nodeAddedMsg.node();
+
+            if (node.id().equals(destNodeId)) {
+                Collection<TcpDiscoveryNode> allNodes = ring.allNodes();
+                Collection<TcpDiscoveryNode> topToSend = new ArrayList<>(allNodes.size());
+
+                for (TcpDiscoveryNode n0 : allNodes) {
+                    assert n0.internalOrder() != 0 : n0;
+
+                    // Skip next node and nodes added after next
+                    // in case this message is resent due to failures/leaves.
+                    // There will be separate messages for nodes with greater
+                    // internal order.
+                    if (n0.internalOrder() < nodeAddedMsg.node().internalOrder())
+                        topToSend.add(n0);
+                }
+
+                nodeAddedMsg.topology(topToSend);
+                nodeAddedMsg.messages(msgs, discardMsgId);
+
+                Map<Long, Collection<ClusterNode>> hist;
+
+                synchronized (mux) {
+                    hist = new TreeMap<>(topHist);
+                }
+
+                nodeAddedMsg.topologyHistory(hist);
+            }
+        }
+    }
+
+    /**
+     * @param msg Message to clear.
+     */
+    private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) {
+        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+            // Nullify topology before registration.
+            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
+
+            nodeAddedMsg.topology(null);
+            nodeAddedMsg.topologyHistory(null);
+            nodeAddedMsg.messages(null, null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override void simulateNodeFailure() {
+        U.warn(log, "Simulating node failure: " + getLocalNodeId());
+
+        U.interrupt(tcpSrvr);
+        U.join(tcpSrvr, log);
+
+        U.interrupt(hbsSnd);
+        U.join(hbsSnd, log);
+
+        U.interrupt(chkStatusSnd);
+        U.join(chkStatusSnd, log);
+
+        U.interrupt(ipFinderCleaner);
+        U.join(ipFinderCleaner, log);
+
+        Collection<SocketReader> tmp;
+
+        synchronized (mux) {
+            tmp = U.arrayList(readers);
+        }
+
+        U.interrupt(tmp);
+        U.joinThreads(tmp, log);
+
+        U.interrupt(msgWorker);
+        U.join(msgWorker, log);
+
+        U.interrupt(statsPrinter);
+        U.join(statsPrinter, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void brakeConnection() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteSpiThread workerThread() {
+        return msgWorker;
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * Simulates situation when next node is still alive but is bypassed
+     * since it has been excluded from the ring, possibly, due to short time
+     * network problems.
+     * <p>
+     * This method is intended for test purposes only.
+     */
+    void forceNextNodeFailure() {
+        U.warn(log, "Next node will be forcibly failed (if any).");
+
+        TcpDiscoveryNode next;
+
+        synchronized (mux) {
+            next = ring.nextNode(failedNodes);
+        }
+
+        if (next != null)
+            msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(getLocalNodeId(), next.id(),
+                next.internalOrder()));
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * This method is intended for test purposes only.
+     *
+     * @return Nodes ring.
+     */
+    TcpDiscoveryNodesRing ring() {
+        return ring;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dumpDebugInfo(IgniteLogger log) {
+        if (!debugMode) {
+            U.quietAndWarn(log, "Failed to dump debug info (discovery SPI was not configured " +
+                "in debug mode, consider setting 'debugMode' configuration property to 'true').");
+
+            return;
+        }
+
+        assert log.isInfoEnabled();
+
+        synchronized (mux) {
+            StringBuilder b = new StringBuilder(U.nl());
+
+            b.append(">>>").append(U.nl());
+            b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl());
+            b.append(">>>").append(U.nl());
+
+            b.append("Local node ID: ").append(getLocalNodeId()).append(U.nl()).append(U.nl());
+            b.append("Local node: ").append(locNode).append(U.nl()).append(U.nl());
+            b.append("SPI state: ").append(spiState).append(U.nl()).append(U.nl());
+
+            b.append("Internal threads: ").append(U.nl());
+
+            b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
+            b.append("    Check status sender: ").append(threadStatus(chkStatusSnd)).append(U.nl());
+            b.append("    HB sender: ").append(threadStatus(hbsSnd)).append(U.nl());
+            b.append("    IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl());
+            b.append("    Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Socket readers: ").append(U.nl());
+
+            for (SocketReader rdr : readers)
+                b.append("    ").append(rdr).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("In-memory log messages: ").append(U.nl());
+
+            for (String msg : debugLog)
+                b.append("    ").append(msg).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Leaving nodes: ").append(U.nl());
+
+            for (TcpDiscoveryNode node : leavingNodes)
+                b.append("    ").append(node.id()).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Failed nodes: ").append(U.nl());
+
+            for (TcpDiscoveryNode node : failedNodes)
+                b.append("    ").append(node.id()).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Stats: ").append(spi.stats).append(U.nl());
+
+            U.quietAndInfo(log, b.toString());
+        }
+    }
+
+    /**
+     * @param msg Message.
+     */
+    private void debugLog(String msg) {
+        assert debugMode;
+
+        String msg0 = new SimpleDateFormat("[HH:mm:ss,SSS]").format(new Date(System.currentTimeMillis())) +
+            '[' + Thread.currentThread().getName() + "][" + getLocalNodeId() +
+            "-" + locNode.internalOrder() + "] " +
+            msg;
+
+        debugLog.add(msg0);
+
+        int delta = debugLog.size() - debugMsgHist;
+
+        for (int i = 0; i < delta && debugLog.size() > debugMsgHist; i++)
+            debugLog.poll();
+    }
+
+    /**
+     * @param msg Message.
+     * @return {@code True} if recordable in debug mode.
+     */
+    private boolean recordable(TcpDiscoveryAbstractMessage msg) {
+        return !(msg instanceof TcpDiscoveryHeartbeatMessage) &&
+            !(msg instanceof TcpDiscoveryStatusCheckMessage) &&
+            !(msg instanceof TcpDiscoveryDiscardMessage);
+    }
+
+    /**
+     * Checks if two given {@link SecurityPermissionSet} objects contain the same permissions.
+     * Each permission belongs to one of three groups : cache, task or system.
+     *
+     * @param locPerms The first set of permissions.
+     * @param rmtPerms The second set of permissions.
+     * @return {@code True} if given parameters contain the same permissions, {@code False} otherwise.
+     */
+    private boolean permissionsEqual(SecurityPermissionSet locPerms, SecurityPermissionSet rmtPerms) {
+        boolean dfltAllowMatch = !(locPerms.defaultAllowAll() ^ rmtPerms.defaultAllowAll());
+
+        boolean bothHaveSamePerms = F.eqNotOrdered(rmtPerms.systemPermissions(), locPerms.systemPermissions()) &&
+            F.eqNotOrdered(rmtPerms.cachePermissions(), locPerms.cachePermissions()) &&
+            F.eqNotOrdered(rmtPerms.taskPermissions(), locPerms.taskPermissions());
+
+        return dfltAllowMatch && bothHaveSamePerms;
+    }
+
+    /**
+     * @param msg Message.
+     * @param nodeId Node ID.
+     */
+    private static void removeMetrics(TcpDiscoveryHeartbeatMessage msg, UUID nodeId) {
+        msg.removeMetrics(nodeId);
+        msg.removeCacheMetrics(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ServerImpl.class, this);
+    }
+
+    /**
+     * Thread that sends heartbeats.
+     */
+    private class HeartbeatsSender extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private HeartbeatsSender() {
+            super(spi.ignite().name(), "tcp-disco-hb-sender", log);
+
+            setPriority(spi.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            while (!isLocalNodeCoordinator())
+                Thread.sleep(1000);
+
+            if (log.isDebugEnabled())
+                log.debug("Heartbeats sender has been started.");
+
+            while (!isInterrupted()) {
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping heartbeats sender (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getLocalNodeId());
+
+                msg.verify(getLocalNodeId());
+
+                msgWorker.addMessage(msg);
+
+                Thread.sleep(spi.hbFreq);
+            }
+        }
+    }
+
+    /**
+     * Thread that sends status check messages to next node if local node has not
+     * been receiving heartbeats ({@link TcpDiscoveryHeartbeatMessage})
+     * for {@link TcpDiscoverySpi#getMaxMissedHeartbeats()} *
+     * {@link TcpDiscoverySpi#getHeartbeatFrequency()}.
+     */
+    private class CheckStatusSender extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private CheckStatusSender() {
+            super(spi.ignite().name(), "tcp-disco-status-check-sender", log);
+
+            setPriority(spi.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("Status check sender has been started.");
+
+            // Only 1 heartbeat missing is acceptable. Add 50 ms to avoid false alarm.
+            long checkTimeout = (long)spi.maxMissedHbs * spi.hbFreq + 50;
+
+            long lastSent = 0;
+
+            while (!isInterrupted()) {
+                // 1. Determine timeout.
+                if (lastSent < locNode.lastUpdateTime())
+                    lastSent = locNode.lastUpdateTime();
+
+                long timeout = (lastSent + checkTimeout) - U.currentTimeMillis();
+
+                if (timeout > 0)
+                    Thread.sleep(timeout);
+
+                // 2. Check if SPI is still connected.
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping status check sender (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                // 3. Was there an update?
+                if (locNode.lastUpdateTime() > lastSent || !ring.hasRemoteNodes()) {
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping status check send " +
+                            "[locNodeLastUpdate=" + U.format(locNode.lastUpdateTime()) +
+                            ", hasRmts=" + ring.hasRemoteNodes() + ']');
+
+                    continue;
+                }
+
+                // 4. Send status check message.
+                lastSent = U.currentTimeMillis();
+
+                msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, null));
+            }
+        }
+    }
+
+    /**
+     * Thread that cleans IP finder and keeps it in the correct state, unregistering
+     * addresses of the nodes that has left the topology.
+     * <p>
+     * This thread should run only on coordinator node and will clean IP finder
+     * if and only if {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}.
+     */
+    private class IpFinderCleaner extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private IpFinderCleaner() {
+            super(spi.ignite().name(), "tcp-disco-ip-finder-cleaner", log);
+
+            setPriority(spi.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("IP finder cleaner has been started.");
+
+            while (!isInterrupted()) {
+                Thread.sleep(spi.ipFinderCleanFreq);
+
+                if (!isLocalNodeCoordinator())
+                    continue;
+
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping IP finder cleaner (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                if (spi.ipFinder.isShared())
+                    cleanIpFinder();
+            }
+        }
+
+        /**
+         * Cleans IP finder.
+         */
+        private void cleanIpFinder() {
+            assert spi.ipFinder.isShared();
+
+            try {
+                // Addresses that belongs to nodes in topology.
+                Collection<InetSocketAddress> currAddrs = F.flatCollections(
+                    F.viewReadOnly(
+                        ring.allNodes(),
+                        new C1<TcpDiscoveryNode, Collection<InetSocketAddress>>() {
+                            @Override public Collection<InetSocketAddress> apply(TcpDiscoveryNode node) {
+                                return !node.isClient() ? spi.getNodeAddresses(node) :
+                                    Collections.<InetSocketAddress>emptyList();
+                            }
+                        }
+                    )
+                );
+
+                // Addresses registered in IP finder.
+                Collection<InetSocketAddress> regAddrs = spi.registeredAddresses();
+
+                // Remove all addresses that belong to alive nodes, leave dead-node addresses.
+                Collection<InetSocketAddress> rmvAddrs = F.view(
+                    regAddrs,
+                    F.notContains(currAddrs),
+                    new P1<InetSocketAddress>() {
+                        private final Map<InetSocketAddress, Boolean> pingResMap =
+                            new HashMap<>();
+
+                        @Override public boolean apply(InetSocketAddress addr) {
+                            Boolean res = pingResMap.get(addr);
+
+                            if (res == null) {
+                                try {
+                                    res = pingNode(addr, null).get1() != null;
+                                }
+                                catch (IgniteCheckedException e) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Failed to ping node [addr=" + addr +
+                                            ", err=" + e.getMessage() + ']');
+
+                                    res = false;
+                                }
+                                finally {
+                                    pingResMap.put(addr, res);
+                                }
+                            }
+
+                            return !res;
+                        }
+                    }
+                );
+
+                // Unregister dead-nodes addresses.
+                if (!rmvAddrs.isEmpty()) {
+                    spi.ipFinder.unregisterAddresses(rmvAddrs);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Unregistered addresses from IP finder: " + rmvAddrs);
+                }
+
+                // Addresses that were removed by mistake (e.g. on segmentation).
+                Collection<InetSocketAddress> missingAddrs = F.view(
+                    currAddrs,
+                    F.notContains(regAddrs)
+                );
+
+                // Re-register missing addresses.
+                if (!missingAddrs.isEmpty()) {
+                    spi.ipFinder.registerAddresses(missingAddrs);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Registered missing addresses in IP finder: " + missingAddrs);
+                }
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to clean IP finder up.");
+            }
+        }
+    }
+
+    /**
+     * Pending messages container.
+     */
+    private static class PendingMessages {
+        /** */
+        private static final int MAX = 1024;
+
+        /** Pending messages. */
+        private final Queue<TcpDiscoveryAbstractMessage> msgs = new ArrayDeque<>(MAX * 2);
+
+        /** Discarded message ID. */
+        private IgniteUuid discardId;
+
+        /**
+         * Adds pending message and shrinks queue if it exceeds limit
+         * (messages that were not discarded yet are never removed).
+         *
+         * @param msg Message to add.
+         */
+        void add(TcpDiscoveryAbstractMessage msg) {
+            msgs.add(msg);
+
+            while (msgs.size() > MAX) {
+                TcpDiscoveryAbstractMessage polled = msgs.poll();
+
+                assert polled != null;
+
+                if (polled.id().equals(discardId))
+                    break;
+            }
+        }
+
+        /**
+         * Gets messages starting from provided ID (exclusive). If such
+         * message is not found, {@code null} is returned (this indicates
+         * a failure condition when it was already removed from queue).
+         *
+         * @param lastMsgId Last message ID.
+         * @return Collection of messages.
+         */
+        @Nullable Collection<TcpDiscoveryAbstractMessage> messages(IgniteUuid lastMsgId) {
+            assert lastMsgId != null;
+
+            Collection<TcpDiscoveryAbstractMessage> cp = new ArrayList<>(msgs.size());
+
+            boolean skip = true;
+
+            for (TcpDiscoveryAbstractMessage msg : msgs) {
+                if (skip) {
+                    if (msg.id().equals(lastMsgId))
+                        skip = false;
+                }
+                else
+                    cp.add(msg);
+            }
+
+            return !skip ? cp : null;
+        }
+
+        /**
+         * Resets pending messages.
+         *
+         * @param msgs Message.
+         * @param discardId Discarded message ID.
+         */
+        void reset(@Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardId) {
+            this.msgs.clear();
+
+            if (msgs != null)
+                this.msgs.addAll(msgs);
+
+            this.discardId = discardId;
+        }
+
+        /**
+         * Clears pending messages.
+         */
+        void clear() {
+            msgs.clear();
+
+            discardId = null;
+        }
+
+        /**
+         * Discards message with provided ID and all before it.
+         *
+         * @param id Discarded message ID.
+         */
+        void discard(IgniteUuid id) {
+            discardId = id;
+        }
+    }
+
+    /**
+     * Message worker thread for messages processing.
+     */
+    private class RingMessageWorker extends MessageWorkerAdapter {
+        /** Next node. */
+        @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+        private TcpDiscoveryNode next;
+
+        /** Pending messages. */
+        private final PendingMessages pendingMsgs = new PendingMessages();
+
+        /** Last message that updated topology. */
+        private TcpDiscoveryAbstractMessage lastMsg;
+
+        /** Force pending messages send. */
+        private boolean forceSndPending;
+
+        /** Socket. */
+        private Socket sock;
+
+        /**
+         */
+        protected RingMessageWorker() {
+            super("tcp-disco-msg-worker");
+        }
+
+        /**
+         * @param msg Message to process.
+         */
+        @Override protected void processMessage(TcpDiscoveryAbstractMessage msg) {
+            if (log.isDebugEnabled())
+                log.debug("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+
+            if (debugMode)
+                debugLog("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+
+            spi.stats.onMessageProcessingStarted(msg);
+
+            if (msg instanceof TcpDiscoveryJoinRequestMessage)
+                processJoinRequestMessage((TcpDiscoveryJoinRequestMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientReconnectMessage)
+                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
+                processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
+                processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
+                processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientHeartbeatMessage)
+                processClientHeartbeatMessage((TcpDiscoveryClientHeartbeatMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryHeartbeatMessage)
+                processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryStatusCheckMessage)
+                processStatusCheckMessage((TcpDiscoveryStatusCheckMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryDiscardMessage)
+                processDiscardMessage((TcpDiscoveryDiscardMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryCustomEventMessage)
+                processCustomMessage((TcpDiscoveryCustomEventMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientPingRequest)
+                processClientPingRequest((TcpDiscoveryClientPingRequest)msg);
+
+            else
+                assert false : "Unknown message type: " + msg.getClass().getSimpleName();
+
+            spi.stats.onMessageProcessingFinished(msg);
+        }
+
+        /**
+         * Sends message across the ring.
+         *
+         * @param msg Message to send
+         */
+        @SuppressWarnings({"BreakStatementWithLabel", "LabeledStatement", "ContinueStatementWithLabel"})
+        private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null;
+
+            assert ring.hasRemoteNodes();
+
+            for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : spi.sendMsgLsnrs)
+                msgLsnr.apply(msg);
+
+            if (redirectToClients(msg)) {
+                byte[] marshalledMsg = null;
+
+                for (ClientMessageWorker clientMsgWorker : clientMsgWorkers.values()) {
+                    // Send a clone to client to avoid ConcurrentModificationException
+                    TcpDiscoveryAbstractMessage msgClone;
+
+                    try {
+                        if (marshalledMsg == null)
+                            marshalledMsg = spi.marsh.marshal(msg);
+
+                        msgClone = spi.marsh.unmarshal(marshalledMsg, null);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to marshal message: " + msg, e);
+
+                        msgClone = msg;
+                    }
+
+                    clientMsgWorker.addMessage(msgClone);
+                }
+            }
+
+            Collection<TcpDiscoveryNode> failedNodes;
+
+            TcpDiscoverySpiState state;
+
+            synchronized (mux) {
+                failedNodes = U.arrayList(ServerImpl.this.failedNodes);
+
+                state = spiState;
+            }
+
+            Collection<Throwable> errs = null;
+
+            boolean sent = false;
+
+            boolean searchNext = true;
+
+            UUID locNodeId = getLocalNodeId();
+
+            while (true) {
+                if (searchNext) {
+                    TcpDiscoveryNode newNext = ring.nextNode(failedNodes);
+
+                    if (newNext == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("No next node in topology.");
+
+                        if (debugMode)
+                            debugLog("No next node in topology.");
+
+                        if (ring.hasRemoteNodes()) {
+                            msg.senderNodeId(locNodeId);
+
+                            addMessage(msg);
+                        }
+
+                        break;
+                    }
+
+                    if (!newNext.equals(next)) {
+                        if (log.isDebugEnabled())
+                            log.debug("New next node [newNext=" + newNext + ", formerNext=" + next +
+                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+
+                        if (debugMode)
+                            debugLog("New next node [newNext=" + newNext + ", formerNext=" + next +
+                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+
+                        U.closeQuiet(sock);
+
+                        sock = null;
+
+                        next = newNext;
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Next node remains the same [nextId=" + next.id() +
+                            ", nextOrder=" + next.internalOrder() + ']');
+                }
+
+                // Flag that shows whether next node exists and accepts incoming connections.
+                boolean nextNodeExists = sock != null;
+
+                final boolean sameHost = U.sameMacs(locNode, next);
+
+                List<InetSocketAddress> locNodeAddrs = U.arrayList(locNode.socketAddresses());
+
+                addr: for (InetSocketAddress addr : spi.getNodeAddresses(next, sameHost)) {
+                    long ackTimeout0 = spi.ackTimeout;
+
+                    if (locNodeAddrs.contains(addr)){
+                        if (log.isDebugEnabled())
+                            log.debug("Skip to send message to the local node (probably remote node has the same " +
+                                "loopback address that local node): " + addr);
+
+                        continue;
+                    }
+
+                    for (int i = 0; i < spi.reconCnt; i++) {
+                        if (sock == null) {
+                            nextNodeExists = false;
+
+                            boolean success = false;
+
+                            boolean openSock = false;
+
+                            // Restore ring.
+                            try {
+                                long tstamp = U.currentTimeMillis();
+
+                                sock = spi.openSocket(addr);
+
+                                openSock = true;
+
+                                // Handshake.
+                                writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
+
+                                TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, ackTimeout0);
+
+                                if (locNodeId.equals(res.creatorNodeId())) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Handshake response from local node: " + res);
+
+                                    U.closeQuiet(sock);
+
+                                    sock = null;
+
+                                    break;
+                                }
+
+                                spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                                UUID nextId = res.creatorNodeId();
+
+                                long nextOrder = res.order();
+
+                                if (!next.id().equals(nextId)) {
+                                    // Node with different ID has bounded to the same port.
+                                    if (log.isDebugEnabled())
+                                        log.debug("Failed to restore ring because next node ID received is not as " +
+                                            "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
+
+                                    if (debugMode)
+                                        debugLog("Failed to restore ring because next node ID received is not as " +
+                                            "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
+
+                                    break;
+                                }
+                                else {
+                                    // ID is as expected. Check node order.
+                                    if (nextOrder != next.internalOrder()) {
+                                        // Is next currently being added?
+                                        boolean nextNew = (msg instanceof TcpDiscoveryNodeAddedMessage &&
+                                            ((TcpDiscoveryNodeAddedMessage)msg).node().id().equals(nextId));
+
+                                        if (!nextNew) {
+                                            if (log.isDebugEnabled())
+                                                log.debug("Failed to restore ring because next node order received " +
+                                                    "is not as expected [expected=" + next.internalOrder() +
+                                                    ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
+
+                                            if (debugMode)
+                                                debugLog("Failed to restore ring because next node order received " +
+                                                    "is not as expected [expected=" + next.internalOrder() +
+                                                    ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
+
+                                            break;
+                                        }
+                                    }
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Initialized connection with next node: " + next.id());
+
+                                    if (debugMode)
+                                        debugLog("Initialized connection with next node: " + next.id());
+
+                                    errs = null;
+
+                                    success = true;
+                                }
+                            }
+                            catch (IOException | IgniteCheckedException e) {
+                                if (errs == null)
+                                    errs = new ArrayList<>();
+
+                                errs.add(e);
+
+                                if (log.isDebugEnabled())
+                                    U.error(log, "Failed to connect to next node [msg=" + msg
+                                        + ", err=" + e.getMessage() + ']', e);
+
+                                onException("Failed to connect to next node [msg=" + msg + ", err=" + e + ']', e);
+
+                                if (!openSock)
+                                    break; // Don't retry if we can not establish connection.
+
+                                if (e instanceof SocketTimeoutException ||
+                                    X.hasCause(e, SocketTimeoutException.class)) {
+                                    ackTimeout0 *= 2;
+
+                                    if (!checkAckTimeout(ackTimeout0))
+                                        break;
+                                }
+
+                                continue;
+                            }
+                            finally {
+                                if (!success) {
+                                    U.closeQuiet(sock);
+
+                                    sock = null;
+                                }
+                                else
+                                    // Next node exists and accepts incoming messages.
+                                    nextNodeExists = true;
+                            }
+                        }
+
+                        try {
+                            boolean failure;
+
+                            synchronized (mux) {
+                                failure = ServerImpl.this.failedNodes.size() < failedNodes.size();
+                            }
+
+                            assert !forceSndPending || msg instanceof TcpDiscoveryNodeLeftMessage;
+
+                            if (failure || forceSndPending) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Pending messages will be sent [failure=" + failure +
+                                        ", forceSndPending=" + forceSndPending + ']');
+
+                                if (debugMode)
+                                    debugLog("Pending messages will be sent [failure=" + failure +
+                                        ", forceSndPending=" + forceSndPending + ']');
+
+                                boolean skip = pendingMsgs.discardId != null;
+
+                                for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs.msgs) {
+                                    if (skip) {
+                                        if (pendingMsg.id().equals(pendingMsgs.discardId))
+                                            skip = false;
+
+                                        continue;
+                                    }
+
+                                    long tstamp = U.currentTimeMillis();
+
+                                    prepareNodeAddedMessage(pendingMsg, next.id(), pendingMsgs.msgs,
+                                        pendingMsgs.discardId);
+
+                                    try {
+                                        writeToSocket(sock, pendingMsg);
+                                    }
+                                    finally {
+                                        clearNodeAddedMessage(pendingMsg);
+                                    }
+
+                                    spi.stats.onMessageSent(pendingMsg, U.currentTimeMillis() - tstamp);
+
+                                    int res = spi.readReceipt(sock, ackTimeout0);
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Pending message has been sent to next node [msg=" + msg.id() +
+                                            ", pendingMsgId=" + pendingMsg + ", next=" + next.id() +
+                                            ", res=" + res + ']');
+
+                                    if (debugMode)
+                                        debugLog("Pending message has been sent to next node [msg=" + msg.id() +
+                                            ", pendingMsgId=" + pendingMsg + ", next=" + next.id() +
+                                            ", res=" + res + ']');
+                                }
+                            }
+
+                            prepareNodeAddedMessage(msg, next.id(), pendingMsgs.msgs, pendingMsgs.discardId);
+
+                            try {
+                                long tstamp = U.currentTimeMillis();
+
+                                writeToSocket(sock, msg);
+
+                                spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+
+                                int res = spi.readReceipt(sock, ackTimeout0);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Message has been sent to next node [msg=" + msg +
+                                        ", next=" + next.id() +
+                                        ", res=" + res + ']');
+
+                                if (debugMode)
+                                    debugLog("Message has been sent to next node [msg=" + msg +
+                                        ", next=" + next.id() +
+                                        ", res=" + res + ']');
+                            }
+                            finally {
+                                clearNodeAddedMessage(msg);
+                            }
+
+                            registerPendingMessage(msg);
+
+                            sent = true;
+
+                            break addr;
+                        }
+                        catch (IOException | IgniteCheckedException e) {
+                            if (errs == null)
+                                errs = new ArrayList<>();
+
+                            errs.add(e);
+
+                            if (log.isDebugEnabled())
+                                U.error(log, "Failed to send message to next node [next=" + next.id() + ", msg=" + msg +
+                                    ", err=" + e + ']', e);
+
+                            onException("Failed to send message to next node [next=" + next.id() + ", msg=" + msg + ']',
+                                e);
+
+                            if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
+                                ackTimeout0 *= 2;
+
+                                if (!checkAckTimeout(ackTimeout0))
+                                    break;
+                            }
+                        }
+                        finally {
+                            forceSndPending = false;
+
+                            if (!sent) {
+                                U.closeQuiet(sock);
+
+                                sock = null;
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Message has not been sent [next=" + next.id() + ", msg=" + msg +
+                                        ", i=" + i + ']');
+                            }
+                        }
+                    } // Try to reconnect.
+                } // Iterating node's addresses.
+
+                if (!sent) {
+                    if (!failedNodes.contains(next)) {
+                        failedNodes.add(next);
+
+                        if (state == CONNECTED) {
+                            Exception err = errs != null ?
+                                U.exceptionWithSuppressed("Failed to send message to next node [msg=" + msg +
+                                    ", next=" + U.toShortString(next) + ']', errs) :
+                                null;
+
+                            // If node existed on connection initialization we should check
+                            // whether it has not gone yet.
+                            if (nextNodeExists && pingNode(next))
+                                U.error(log, "Failed to send message to next node [msg=" + msg +
+                                    ", next=" + next + ']', err);
+                            else if (log.isDebugEnabled())
+                                log.debug("Failed to send message to next node [msg=" + msg + ", next=" + next +
+                                    ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']');
+                        }
+                    }
+
+                    if (msg instanceof TcpDiscoveryStatusCheckMessage) {
+                        TcpDiscoveryStatusCheckMessage msg0 = (TcpDiscoveryStatusCheckMessage)msg;
+
+                        if (next.id().equals(msg0.failedNodeId())) {
+                            next = null;
+
+                            if (log.isDebugEnabled())
+                                log.debug("Discarding status check since next node has indeed failed [next=" + next +
+                                    ", msg=" + msg + ']');
+
+                            // Discard status check message by exiting loop and handle failure.
+                            break;
+                        }
+                    }
+
+                    next = null;
+
+                    searchNext = true;
+
+                    errs = null;
+                }
+                else
+                    break;
+            }
+
+            synchronized (mux) {
+                failedNodes.removeAll(ServerImpl.this.failedNodes);
+            }
+
+            if (!failedNodes.isEmpty()) {
+                if (state == CONNECTED) {
+                    if (!sent && log.isDebugEnabled())
+                        // Message has not been sent due to some problems.
+                        log.debug("Message has not been sent: " + msg);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Detected failed nodes: " + failedNodes);
+                }
+
+                synchronized (mux) {
+                    ServerImpl.this.failedNodes.addAll(failedNodes);
+                }
+
+                for (TcpDiscoveryNode n : failedNodes)
+                    msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(locNodeId, n.id(), n.internalOrder()));
+            }
+        }
+
+        /**
+         * @param msg Message.
+         * @return Whether to redirect message to client nodes.
+         */
+        private boolean redirectToClients(TcpDiscoveryAbstractMessage msg) {
+            return msg.verified() && U.getAnnotation(msg.getClass(), TcpDiscoveryRedirectToClient.class) != null;
+        }
+
+        /**
+         * Registers pending message.
+         *
+         * @param msg Message to register.
+         */
+        private void registerPendingMessage(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null;
+
+            if (spi.ensured(msg)) {
+                pendingMsgs.add(msg);
+
+                spi.stats.onPendingMessageRegistered();
+
+                if (log.isDebugEnabled())
+                    log.debug("Pending message has been registered: " + msg.id());
+            }
+        }
+
+        /**
+         * Processes join request message.
+         *
+         * @param msg Join request message.
+         */
+        private void processJoinRequestMessage(TcpDiscoveryJoinRequestMessage msg) {
+            assert msg != null;
+
+            TcpDiscoveryNode node = msg.node();
+
+            UUID locNodeId = getLocalNodeId();
+
+            if (!msg.client()) {
+                boolean rmtHostLoopback = node.socketAddresses().size() == 1 &&
+                    node.socketAddresses().iterator().next().getAddress().isLoopbackAddress();
+
+                // This check is performed by the node joining node is connected to, but not by coordinator
+                // because loopback problem message is sent directly to the joining node which may be unavailable
+                // if coordinator resides on another host.
+                if (spi.locHost.isLoopbackAddress() != rmtHostLoopback) {
+                    String firstNode = rmtHostLoopback ? "remote" : "local";
+
+                    String secondNode = rmtHostLoopback ? "local" : "remote";
+
+                    String errMsg = "Failed to add node to topology because " + firstNode +
+                        " node is configured to use loopback address, but " + secondNode + " node is not " +
+                        "(consider changing 'localAddress' configuration parameter) " +
+                

<TRUNCATED>


[18/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 08a9937..733ae81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -176,9 +176,11 @@ public class GridNioRecoveryDescriptor {
         while (acked < rcvCnt) {
             GridNioFuture<?> fut = msgFuts.pollFirst();
 
-            assert fut != null;
+            assert fut != null : "Missed message future [rcvCnt=" + rcvCnt +
+                ", acked=" + acked +
+                ", desc=" + this + ']';
 
-            assert fut.isDone();
+            assert fut.isDone() : fut;
 
             acked++;
         }
@@ -239,9 +241,12 @@ public class GridNioRecoveryDescriptor {
      * @param rcvCnt Number of messages received by remote node.
      */
     public void onHandshake(long rcvCnt) {
-        ackReceived(rcvCnt);
+        synchronized (this) {
+            if (!nodeLeft)
+                ackReceived(rcvCnt);
 
-        resendCnt = msgFuts.size();
+            resendCnt = msgFuts.size();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
deleted file mode 100644
index 72c20f8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpCommunicationClient.java
+++ /dev/null
@@ -1,554 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.util.nio;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.plugin.extensions.communication.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.nio.*;
-import java.util.*;
-import java.util.concurrent.locks.*;
-
-/**
- * Grid client for NIO server.
- */
-public class GridTcpCommunicationClient extends GridAbstractCommunicationClient {
-    /** Socket. */
-    private final Socket sock;
-
-    /** Output stream. */
-    private final UnsafeBufferedOutputStream out;
-
-    /** Minimum buffered message count. */
-    private final int minBufferedMsgCnt;
-
-    /** Communication buffer size ratio. */
-    private final double bufSizeRatio;
-
-    /** */
-    private final ByteBuffer writeBuf;
-
-    /** */
-    private final MessageFormatter formatter;
-
-    /**
-     * @param metricsLsnr Metrics listener.
-     * @param addr Address.
-     * @param locHost Local address.
-     * @param connTimeout Connect timeout.
-     * @param tcpNoDelay Value for {@code TCP_NODELAY} socket option.
-     * @param sockRcvBuf Socket receive buffer.
-     * @param sockSndBuf Socket send buffer.
-     * @param bufSize Buffer size (or {@code 0} to disable buffer).
-     * @param minBufferedMsgCnt Minimum buffered message count.
-     * @param bufSizeRatio Communication buffer size ratio.
-     * @param formatter Message formatter.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridTcpCommunicationClient(
-        GridNioMetricsListener metricsLsnr,
-        InetSocketAddress addr,
-        InetAddress locHost,
-        long connTimeout,
-        boolean tcpNoDelay,
-        int sockRcvBuf,
-        int sockSndBuf,
-        int bufSize,
-        int minBufferedMsgCnt,
-        double bufSizeRatio,
-        MessageFormatter formatter
-    ) throws IgniteCheckedException {
-        super(metricsLsnr);
-
-        assert metricsLsnr != null;
-        assert addr != null;
-        assert locHost != null;
-        assert connTimeout >= 0;
-        assert bufSize >= 0;
-
-        A.ensure(minBufferedMsgCnt >= 0,
-            "Value of minBufferedMessageCount property cannot be less than zero.");
-        A.ensure(bufSizeRatio > 0 && bufSizeRatio < 1,
-            "Value of bufSizeRatio property must be between 0 and 1 (exclusive).");
-
-        this.minBufferedMsgCnt = minBufferedMsgCnt;
-        this.bufSizeRatio = bufSizeRatio;
-        this.formatter = formatter;
-
-        writeBuf = ByteBuffer.allocate(8 << 10);
-
-        writeBuf.order(ByteOrder.nativeOrder());
-
-        sock = new Socket();
-
-        boolean success = false;
-
-        try {
-            sock.bind(new InetSocketAddress(locHost, 0));
-
-            sock.setTcpNoDelay(tcpNoDelay);
-
-            if (sockRcvBuf > 0)
-                sock.setReceiveBufferSize(sockRcvBuf);
-
-            if (sockSndBuf > 0)
-                sock.setSendBufferSize(sockSndBuf);
-
-            sock.connect(addr, (int)connTimeout);
-
-            out = new UnsafeBufferedOutputStream(sock.getOutputStream(), bufSize);
-
-            success = true;
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to connect to remote host " +
-                "[addr=" + addr + ", localHost=" + locHost + ']', e);
-        }
-        finally {
-            if (!success)
-                U.closeQuiet(sock);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void doHandshake(IgniteInClosure2X<InputStream, OutputStream> handshakeC) throws IgniteCheckedException {
-        try {
-            handshakeC.applyx(sock.getInputStream(), sock.getOutputStream());
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to access IO streams when executing handshake with remote node: " +
-                sock.getRemoteSocketAddress(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean close() {
-        boolean res = super.close();
-
-        if (res) {
-            U.closeQuiet(out);
-            U.closeQuiet(sock);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void forceClose() {
-        super.forceClose();
-
-        try {
-            out.flush();
-        }
-        catch (IOException ignored) {
-            // No-op.
-        }
-
-        // Do not call (directly or indirectly) out.close() here
-        // since it may cause a deadlock.
-        out.forceClose();
-
-        U.closeQuiet(sock);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendMessage(byte[] data, int len) throws IgniteCheckedException {
-        if (closed())
-            throw new IgniteCheckedException("Client was closed: " + this);
-
-        try {
-            out.write(data, 0, len);
-
-            metricsLsnr.onBytesSent(len);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to send message to remote node: " + sock.getRemoteSocketAddress(), e);
-        }
-
-        markUsed();
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg)
-        throws IgniteCheckedException {
-        if (closed())
-            throw new IgniteCheckedException("Client was closed: " + this);
-
-        assert writeBuf.hasArray();
-
-        try {
-            int cnt = U.writeMessageFully(msg, out, writeBuf, formatter.writer());
-
-            metricsLsnr.onBytesSent(cnt);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException("Failed to send message to remote node: " + sock.getRemoteSocketAddress(), e);
-        }
-
-        markUsed();
-
-        return false;
-    }
-
-    /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    @Override public void flushIfNeeded(long timeout) throws IOException {
-        assert timeout > 0;
-
-        out.flushOnTimeout(timeout);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendMessage(ByteBuffer data) throws IgniteCheckedException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(GridTcpCommunicationClient.class, this, super.toString());
-    }
-
-    /**
-     *
-     */
-    private class UnsafeBufferedOutputStream extends FilterOutputStream {
-        /** The internal buffer where data is stored. */
-        private final byte buf[];
-
-        /** Current size. */
-        private int size;
-
-        /** Count. */
-        private int cnt;
-
-        /** Message count. */
-        private int msgCnt;
-
-        /** Total messages size. */
-        private int totalCnt;
-
-        /** Lock. */
-        private final ReentrantLock lock = new ReentrantLock();
-
-        /** Last flushed timestamp. */
-        private volatile long lastFlushed = U.currentTimeMillis();
-
-        /** Cached flush timeout. */
-        private volatile long flushTimeout;
-
-        /** Buffer adjusted timestamp. */
-        private long lastAdjusted = U.currentTimeMillis();
-
-        /**
-         * Creates a new buffered output stream to write data to the
-         * specified underlying output stream.
-         *
-         * @param out The underlying output stream.
-         */
-        UnsafeBufferedOutputStream(OutputStream out) {
-            this(out, 8192);
-        }
-
-        /**
-         * Creates a new buffered output stream to write data to the
-         * specified underlying output stream with the specified buffer
-         * size.
-         *
-         * @param out The underlying output stream.
-         * @param size The buffer size.
-         */
-        UnsafeBufferedOutputStream(OutputStream out, int size) {
-            super(out);
-
-            assert size >= 0;
-
-            this.size = size;
-            buf = size > 0 ? new byte[size] : null;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(int b) throws IOException {
-            throw new UnsupportedOperationException();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void write(byte[] b, int off, int len) throws IOException {
-            assert b != null;
-            assert off == 0;
-
-            // No buffering.
-            if (buf == null) {
-                lock.lock();
-
-                try {
-                    out.write(b, 0, len);
-                }
-                finally {
-                    lock.unlock();
-                }
-
-                return;
-            }
-
-            // Buffering is enabled.
-            lock.lock();
-
-            try {
-                msgCnt++;
-                totalCnt += len;
-
-                if (len >= size) {
-                    flushLocked();
-
-                    out.write(b, 0, len);
-
-                    lastFlushed = U.currentTimeMillis();
-
-                    adjustBufferIfNeeded();
-
-                    return;
-                }
-
-                if (cnt + len > size) {
-                    flushLocked();
-
-                    messageToBuffer0(b, off, len, buf, 0);
-
-                    cnt = len;
-
-                    assert cnt < size;
-
-                    adjustBufferIfNeeded();
-
-                    return;
-                }
-
-                messageToBuffer0(b, 0, len, buf, cnt);
-
-                cnt += len;
-
-                if (cnt == size)
-                    flushLocked();
-                else
-                    flushIfNeeded();
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @throws IOException If failed.
-         */
-        private void flushIfNeeded() throws IOException {
-            assert lock.isHeldByCurrentThread();
-            assert buf != null;
-
-            long flushTimeout0 = flushTimeout;
-
-            if (flushTimeout0 > 0)
-                flushOnTimeoutLocked(flushTimeout0);
-        }
-
-        /**
-         *
-         */
-        private void adjustBufferIfNeeded() {
-            assert lock.isHeldByCurrentThread();
-            assert buf != null;
-
-            long flushTimeout0 = flushTimeout;
-
-            if (flushTimeout0 > 0)
-                adjustBufferLocked(flushTimeout0);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void flush() throws IOException {
-            lock.lock();
-
-            try {
-                flushLocked();
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @param timeout Timeout.
-         * @throws IOException If failed.
-         */
-        public void flushOnTimeout(long timeout) throws IOException {
-            assert buf != null;
-            assert timeout > 0;
-
-            // Overwrite cached value.
-            flushTimeout = timeout;
-
-            if (lastFlushed + timeout > U.currentTimeMillis() || !lock.tryLock())
-                return;
-
-            try {
-                flushOnTimeoutLocked(timeout);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-
-        /**
-         * @param timeout Timeout.
-         * @throws IOException If failed.
-         */
-        private void flushOnTimeoutLocked(long timeout) throws IOException {
-            assert lock.isHeldByCurrentThread();
-            assert timeout > 0;
-
-            // Double check.
-            if (cnt == 0 || lastFlushed + timeout > U.currentTimeMillis())
-                return;
-
-            flushLocked();
-
-            adjustBufferLocked(timeout);
-        }
-
-        /**
-         * @param timeout Timeout.
-         */
-        private void adjustBufferLocked(long timeout) {
-            assert lock.isHeldByCurrentThread();
-            assert timeout > 0;
-
-            long time = U.currentTimeMillis();
-
-            if (lastAdjusted + timeout < time) {
-                if (msgCnt <= minBufferedMsgCnt)
-                    size = 0;
-                else {
-                    size = (int)(totalCnt * bufSizeRatio);
-
-                    if (size > buf.length)
-                        size = buf.length;
-                }
-
-                msgCnt = 0;
-                totalCnt = 0;
-
-                lastAdjusted = time;
-            }
-        }
-
-        /**
-         * @throws IOException If failed.
-         */
-        private void flushLocked() throws IOException {
-            assert lock.isHeldByCurrentThread();
-
-            if (buf != null && cnt > 0) {
-                out.write(buf, 0, cnt);
-
-                cnt = 0;
-            }
-
-            out.flush();
-
-            lastFlushed = U.currentTimeMillis();
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() throws IOException {
-            lock.lock();
-
-            try {
-                flushLocked();
-            }
-            finally {
-                try {
-                    out.close();
-                }
-                finally {
-                    lock.unlock();
-                }
-            }
-        }
-
-        /**
-         * Forcibly closes underlying stream ignoring any possible exception.
-         */
-        public void forceClose() {
-            try {
-                out.close();
-            }
-            catch (IOException ignored) {
-                // No-op.
-            }
-        }
-
-        /**
-         * @param b Buffer to copy from.
-         * @param off Offset in source buffer.
-         * @param len Length.
-         * @param resBuf Result buffer.
-         * @param resOff Result offset.
-         */
-        private void messageToBuffer(byte[] b, int off, int len, byte[] resBuf, int resOff) {
-            assert b.length == len;
-            assert off == 0;
-            assert resBuf.length >= resOff + len + 4;
-
-            U.intToBytes(len, resBuf, resOff);
-
-            U.arrayCopy(b, off, resBuf, resOff + 4, len);
-        }
-
-        /**
-         * @param b Buffer to copy from (length included).
-         * @param off Offset in source buffer.
-         * @param len Length.
-         * @param resBuf Result buffer.
-         * @param resOff Result offset.
-         */
-        private void messageToBuffer0(byte[] b, int off, int len, byte[] resBuf, int resOff) {
-            assert off == 0;
-            assert resBuf.length >= resOff + len;
-
-            U.arrayCopy(b, off, resBuf, resOff, len);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            lock.lock();
-
-            try {
-                return S.toString(UnsafeBufferedOutputStream.class, this);
-            }
-            finally {
-                lock.unlock();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 788a8e6..abad875 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -122,14 +122,6 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
         return false;
     }
 
-    /**
-     * @param timeout Timeout.
-     * @throws IOException If failed.
-     */
-    @Override public void flushIfNeeded(long timeout) throws IOException {
-        // No-op.
-    }
-
     /** {@inheritDoc} */
     @Override public boolean async() {
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index bd24ecf..9e15d2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -202,7 +202,7 @@ public class VisorCache implements Serializable {
         offHeapAllocatedSize = ca.offHeapAllocatedSize();
         offHeapEntriesCnt = ca.offHeapEntriesCount();
         partitions = ca.affinity().partitions();
-        metrics = VisorCacheMetrics.from(ignite, ca);
+        metrics = VisorCacheMetrics.from(ignite, cacheName);
 
         estimateMemorySize(ignite, ca, sample);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
index ef12424..c8913c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheConfigurationCollectorJob.java
@@ -45,17 +45,17 @@ public class VisorCacheConfigurationCollectorJob
 
     /** {@inheritDoc} */
     @Override protected Map<IgniteUuid, VisorCacheConfiguration> run(Collection<IgniteUuid> arg) {
-        Collection<GridCacheAdapter<?, ?>> caches = ignite.context().cache().internalCaches();
+        Collection<IgniteCacheProxy<?, ?>> caches = ignite.context().cache().jcaches();
 
         boolean all = arg == null || arg.isEmpty();
 
         Map<IgniteUuid, VisorCacheConfiguration> res = U.newHashMap(caches.size());
 
-        for (GridCacheAdapter<?, ?> cache : caches) {
+        for (IgniteCacheProxy<?, ?> cache : caches) {
             IgniteUuid deploymentId = cache.context().dynamicDeploymentId();
 
             if (all || arg.contains(deploymentId))
-                res.put(deploymentId, config(cache.configuration()));
+                res.put(deploymentId, config(cache.getConfiguration(CacheConfiguration.class)));
         }
 
         return res;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
index 30be424..c5d70a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetrics.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.visor.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -166,19 +167,21 @@ public class VisorCacheMetrics implements Serializable {
 
     /**
      * @param ignite Ignite.
-     * @param c Cache.
+     * @param cacheName Cache name.
      * @return Data transfer object for given cache metrics.
      */
-    public static VisorCacheMetrics from(IgniteEx ignite, IgniteInternalCache c) {
+    public static VisorCacheMetrics from(IgniteEx ignite, String cacheName) {
         VisorCacheMetrics cm = new VisorCacheMetrics();
 
-        CacheMetrics m = c.metrics();
-
         GridCacheProcessor cacheProcessor = ignite.context().cache();
 
-        cm.name = c.name();
-        cm.mode = cacheProcessor.cacheMode(c.name());
-        cm.sys = cacheProcessor.systemCache(c.name());
+        IgniteCache<Object, Object> c = cacheProcessor.jcache(cacheName);
+
+        cm.name = cacheName;
+        cm.mode = cacheProcessor.cacheMode(cacheName);
+        cm.sys = cacheProcessor.systemCache(cacheName);
+
+        CacheMetrics m = c.metrics();
 
         cm.size = m.getSize();
         cm.keySize = m.getKeySize();
@@ -208,7 +211,7 @@ public class VisorCacheMetrics implements Serializable {
         cm.commitsPerSec = perSecond(m.getAverageTxCommitTime());
         cm.rollbacksPerSec = perSecond(m.getAverageTxRollbackTime());
 
-        cm.qryMetrics = VisorCacheQueryMetrics.from(c.context().queries().metrics());
+        cm.qryMetrics = VisorCacheQueryMetrics.from(c.queryMetrics());
 
         cm.dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         cm.txThreadMapSize = m.getTxThreadMapSize();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
index 8fd42a0..23263c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheMetricsCollectorTask.java
@@ -99,17 +99,19 @@ public class VisorCacheMetricsCollectorTask extends VisorMultiNodeTask<IgniteBiT
 
             GridCacheProcessor cacheProcessor = ignite.context().cache();
 
-            Collection<GridCacheAdapter<?, ?>> caches = cacheProcessor.internalCaches();
+            Collection<IgniteCacheProxy<?, ?>> caches = cacheProcessor.jcaches();
 
             Collection<VisorCacheMetrics> res = new ArrayList<>(caches.size());
 
             boolean allCaches = cacheNames.isEmpty();
 
-            for (GridCacheAdapter ca : caches) {
+            for (IgniteCacheProxy ca : caches) {
                 if (ca.context().started()) {
-                    VisorCacheMetrics cm = VisorCacheMetrics.from(ignite, ca);
+                    String cacheName = ca.getName();
 
-                    if ((allCaches || cacheNames.contains(ca.name())) && (showSysCaches || !cm.system()))
+                    VisorCacheMetrics cm = VisorCacheMetrics.from(ignite, cacheName);
+
+                    if ((allCaches || cacheNames.contains(cacheName)) && (showSysCaches || !cm.system()))
                         res.add(cm);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
index 06dbfbf..ab24a3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheStoreConfiguration.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.visor.cache;
 
-import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.configuration.*;
@@ -72,10 +71,10 @@ public class VisorCacheStoreConfiguration implements Serializable {
      * @param ccfg Cache configuration.
      * @return Data transfer object for cache store configuration properties.
      */
-    public static VisorCacheStoreConfiguration from(Ignite ignite, CacheConfiguration ccfg) {
+    public static VisorCacheStoreConfiguration from(IgniteEx ignite, CacheConfiguration ccfg) {
         VisorCacheStoreConfiguration cfg = new VisorCacheStoreConfiguration();
 
-        GridCacheAdapter<Object, Object> c = ((IgniteKernal)ignite).internalCache(ccfg.getName());
+        IgniteCacheProxy<Object, Object> c = ignite.context().cache().jcache(ccfg.getName());
 
         CacheStore store = c != null && c.context().started() ? c.context().store().configuredStore() : null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
index fde871b..3b2d45c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTask.java
@@ -23,6 +23,7 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -87,7 +88,7 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
                 else {
                     // Ignore nodes that left topology.
                     if (!(unhandledEx instanceof ClusterGroupEmptyException))
-                        taskRes.unhandledEx().put(nid, unhandledEx);
+                        taskRes.unhandledEx().put(nid, new VisorExceptionWrapper(unhandledEx));
                 }
             }
         }
@@ -116,13 +117,13 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.events().addAll(jobRes.events());
 
         if (jobRes.eventsEx() != null)
-            taskRes.eventsEx().put(nid, jobRes.eventsEx());
+            taskRes.eventsEx().put(nid, new VisorExceptionWrapper(jobRes.eventsEx()));
 
         if (!jobRes.caches().isEmpty())
             taskRes.caches().put(nid, jobRes.caches());
 
         if (jobRes.cachesEx() != null)
-            taskRes.cachesEx().put(nid, jobRes.cachesEx());
+            taskRes.cachesEx().put(nid, new VisorExceptionWrapper(jobRes.cachesEx()));
 
         if (!jobRes.igfss().isEmpty())
             taskRes.igfss().put(nid, jobRes.igfss());
@@ -131,6 +132,6 @@ public class VisorNodeDataCollectorTask extends VisorMultiNodeTask<VisorNodeData
             taskRes.igfsEndpoints().put(nid, jobRes.igfsEndpoints());
 
         if (jobRes.igfssEx() != null)
-            taskRes.igfssEx().put(nid, jobRes.igfssEx());
+            taskRes.igfssEx().put(nid, new VisorExceptionWrapper(jobRes.igfssEx()));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
index 6485978..1a4eb02 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorTaskResult.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.visor.node;
 import org.apache.ignite.internal.visor.cache.*;
 import org.apache.ignite.internal.visor.event.*;
 import org.apache.ignite.internal.visor.igfs.*;
+import org.apache.ignite.internal.visor.util.*;
 
 import java.io.*;
 import java.util.*;
@@ -32,7 +33,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private static final long serialVersionUID = 0L;
 
     /** Unhandled exceptions from nodes. */
-    private final Map<UUID, Throwable> unhandledEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> unhandledEx = new HashMap<>();
 
     /** Nodes grid names. */
     private final Map<UUID, String> gridNames = new HashMap<>();
@@ -50,13 +51,13 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private final List<VisorGridEvent> evts = new ArrayList<>();
 
     /** Exceptions caught during collecting events from nodes. */
-    private final Map<UUID, Throwable> evtsEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> evtsEx = new HashMap<>();
 
     /** All caches collected from nodes. */
     private final Map<UUID, Collection<VisorCache>> caches = new HashMap<>();
 
     /** Exceptions caught during collecting caches from nodes. */
-    private final Map<UUID, Throwable> cachesEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> cachesEx = new HashMap<>();
 
     /** All IGFS collected from nodes. */
     private final Map<UUID, Collection<VisorIgfs>> igfss = new HashMap<>();
@@ -65,7 +66,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     private final Map<UUID, Collection<VisorIgfsEndpoint>> igfsEndpoints = new HashMap<>();
 
     /** Exceptions caught during collecting IGFS from nodes. */
-    private final Map<UUID, Throwable> igfssEx = new HashMap<>();
+    private final Map<UUID, VisorExceptionWrapper> igfssEx = new HashMap<>();
 
     /**
      * @return {@code true} If no data was collected.
@@ -88,7 +89,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Unhandled exceptions from nodes.
      */
-    public Map<UUID, Throwable> unhandledEx() {
+    public Map<UUID, VisorExceptionWrapper> unhandledEx() {
         return unhandledEx;
     }
 
@@ -123,7 +124,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting events from nodes.
      */
-    public Map<UUID, Throwable> eventsEx() {
+    public Map<UUID, VisorExceptionWrapper> eventsEx() {
         return evtsEx;
     }
 
@@ -137,7 +138,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting caches from nodes.
      */
-    public Map<UUID, Throwable> cachesEx() {
+    public Map<UUID, VisorExceptionWrapper> cachesEx() {
         return cachesEx;
     }
 
@@ -158,7 +159,7 @@ public class VisorNodeDataCollectorTaskResult implements Serializable {
     /**
      * @return Exceptions caught during collecting IGFS from nodes.
      */
-    public Map<UUID, Throwable> igfssEx() {
+    public Map<UUID, VisorExceptionWrapper> igfssEx() {
         return igfssEx;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
index 8b39d09..9fc1cc4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeSuppressedErrorsTask.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -83,12 +84,21 @@ public class VisorNodeSuppressedErrorsTask extends VisorMultiNodeTask<Map<UUID,
 
             List<IgniteExceptionRegistry.ExceptionInfo> errors = ignite.context().exceptionRegistry().getErrors(order);
 
+            List<IgniteExceptionRegistry.ExceptionInfo> wrapped = new ArrayList<>(errors.size());
+
             for (IgniteExceptionRegistry.ExceptionInfo error : errors) {
                 if (error.order() > order)
                     order = error.order();
+
+                wrapped.add(new IgniteExceptionRegistry.ExceptionInfo(error.order(),
+                    new VisorExceptionWrapper(error.error()),
+                    error.message(),
+                    error.threadId(),
+                    error.threadName(),
+                    error.time()));
             }
 
-            return new IgniteBiTuple<>(order, errors);
+            return new IgniteBiTuple<>(order, wrapped);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
index 4a9daad..e977d2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 
 import javax.cache.*;
@@ -36,7 +37,7 @@ import static org.apache.ignite.internal.visor.query.VisorQueryUtils.*;
 /**
  * Job for execute SCAN or SQL query and get first page of results.
  */
-public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? extends Exception, VisorQueryResultEx>> {
+public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -61,11 +62,11 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteBiTuple<? extends Exception, VisorQueryResultEx> run(VisorQueryArg arg) {
+    @Override protected IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx> run(VisorQueryArg arg) {
         try {
             UUID nid = ignite.localNode().id();
 
-            boolean scan = arg.queryTxt().toUpperCase().startsWith("SCAN");
+            boolean scan = arg.queryTxt() == null;
 
             String qryId = (scan ? SCAN_QRY_NAME : SQL_QRY_NAME) + "-" +
                 UUID.randomUUID();
@@ -110,8 +111,8 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 Collection<GridQueryFieldMetadata> meta = cur.fieldsMeta();
 
                 if (meta == null)
-                    return new IgniteBiTuple<Exception, VisorQueryResultEx>(
-                        new SQLException("Fail to execute query. No metadata available."), null);
+                    return new IgniteBiTuple<>(
+                        new VisorExceptionWrapper(new SQLException("Fail to execute query. No metadata available.")), null);
                 else {
                     List<VisorQueryField> names = new ArrayList<>(meta.size());
 
@@ -138,7 +139,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
             }
         }
         catch (Exception e) {
-            return new IgniteBiTuple<>(e, null);
+            return new IgniteBiTuple<>(VisorTaskUtils.wrap(e), null);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
index 4f2fda5..98c876a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
@@ -19,13 +19,14 @@ package org.apache.ignite.internal.visor.query;
 
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 
 /**
  * Task for execute SCAN or SQL query and get first page of results.
  */
 @GridInternal
-public class VisorQueryTask extends VisorOneNodeTask<VisorQueryArg, IgniteBiTuple<? extends Exception, VisorQueryResultEx>> {
+public class VisorQueryTask extends VisorOneNodeTask<VisorQueryArg, IgniteBiTuple<? extends VisorExceptionWrapper, VisorQueryResultEx>> {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
new file mode 100644
index 0000000..d2ae0e1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -0,0 +1,81 @@
+/*
+ * 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.visor.util;
+
+/**
+ * Exception wrapper for safe for transferring to Visor.
+ */
+public class VisorExceptionWrapper extends Throwable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Detail message string of this throwable */
+    private String detailMsg;
+
+    /** Simple class name of base throwable object. */
+    private String clsSimpleName;
+
+    /** Class name of base throwable object. */
+    private String clsName;
+
+    /**
+     * Wrap throwable by presented on Visor throwable object.
+     *
+     * @param cause Base throwable object.
+     */
+    public VisorExceptionWrapper(Throwable cause) {
+        assert cause != null;
+
+        clsSimpleName = cause.getClass().getSimpleName();
+        clsName = cause.getClass().getName();
+
+        detailMsg = cause.getMessage();
+
+        StackTraceElement[] stackTrace = cause.getStackTrace();
+
+        if (stackTrace != null)
+            setStackTrace(stackTrace);
+
+        if (cause.getCause() != null)
+            initCause(new VisorExceptionWrapper(cause.getCause()));
+    }
+
+    /**
+     * @return Class simple name of base throwable object.
+     */
+    public String getClassSimpleName() {
+        return clsSimpleName;
+    }
+
+    /**
+     * @return Class name of base throwable object.
+     */
+    public String getClassName() {
+        return clsName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getMessage() {
+        return detailMsg;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return (detailMsg != null) ? (clsName + ": " + detailMsg) : clsName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index e8ae76d..b0afbc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -867,4 +867,14 @@ public class VisorTaskUtils {
 
         return bos.toByteArray();
     }
+
+    /**
+     * Wrap throwable object of any type to presented on Visor throwable object.
+     *
+     * @param e Base throwable object.
+     * @return Wrapped throwable object.
+     */
+    public static VisorExceptionWrapper wrap(Throwable e) {
+        return new VisorExceptionWrapper(e);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
index 2ad07b5..5cdc72f 100644
--- a/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/CacheMetricsMXBean.java
@@ -100,14 +100,94 @@ public interface CacheMetricsMXBean extends CacheStatisticsMXBean, CacheMXBean,
     public long getOverflowSize();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Number of gets from off-heap memory.")
+    public long getOffHeapGets();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of puts to off-heap memory.")
+    public long getOffHeapPuts();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of removed entries from off-heap memory.")
+    public long getOffHeapRemovals();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of evictions from off-heap memory.")
+    public long getOffHeapEvictions();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of hits on off-heap memory.")
+    public long getOffHeapHits();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of hits on off-heap memory.")
+    public float getOffHeapHitPercentage();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of misses on off-heap memory.")
+    public long getOffHeapMisses();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of misses on off-heap memory.")
+    public float getOffHeapMissPercentage();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Number of entries stored in off-heap memory.")
     public long getOffHeapEntriesCount();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Number of primary entries stored in off-heap memory.")
+    public long getOffHeapPrimaryEntriesCount();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of backup stored in off-heap memory.")
+    public long getOffHeapBackupEntriesCount();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Memory size allocated in off-heap.")
     public long getOffHeapAllocatedSize();
 
     /** {@inheritDoc} */
+    @MXBeanDescription("Off-heap memory maximum size.")
+    public long getOffHeapMaxSize();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of gets from swap.")
+    public long getSwapGets();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of puts to swap.")
+    public long getSwapPuts();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of removed entries from swap.")
+    public long getSwapRemovals();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of hits on swap.")
+    public long getSwapHits();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of misses on swap.")
+    public long getSwapMisses();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of hits on swap.")
+    public float getSwapHitPercentage();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Percentage of misses on swap.")
+    public float getSwapMissPercentage();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Number of entries stored in swap.")
+    public long getSwapEntriesCount();
+
+    /** {@inheritDoc} */
+    @MXBeanDescription("Size of swap.")
+    public long getSwapSize();
+
+    /** {@inheritDoc} */
     @MXBeanDescription("Number of non-null values in the cache.")
     public int getSize();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
index 17bbc36..f064fde 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/PluginProvider.java
@@ -19,13 +19,22 @@ package org.apache.ignite.plugin;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.util.*;
 
 /**
- * Pluggable ignite component.
+ * Pluggable Ignite component.
+ * <p>
+ * Ignite plugins are loaded using JDK {@link ServiceLoader}.
+ * First method called to initialize plugin is {@link PluginProvider#initExtensions(PluginContext, ExtensionRegistry)}.
+ * If plugin requires configuration it can be set in {@link IgniteConfiguration} using
+ * {@link IgniteConfiguration#setPluginConfigurations(PluginConfiguration...)}.
+ *
+ * @see IgniteConfiguration#setPluginConfigurations(PluginConfiguration...)
+ * @see PluginContext
  */
 public interface PluginProvider<C extends PluginConfiguration> {
     /**
@@ -49,18 +58,21 @@ public interface PluginProvider<C extends PluginConfiguration> {
     public <T extends IgnitePlugin> T plugin();
 
     /**
+     * Registers extensions.
+     *
      * @param ctx Plugin context.
-     * @param cls Ignite component class.
-     * @return Ignite component or {@code null} if component is not supported.
+     * @param registry Extension registry.
      */
-    @Nullable public <T> T createComponent(PluginContext ctx, Class<T> cls);
+    public void initExtensions(PluginContext ctx, ExtensionRegistry registry);
 
     /**
-     * Register extensions.
+     * Creates Ignite component.
+     *
      * @param ctx Plugin context.
-     * @param registry Extension registry.
+     * @param cls Ignite component class.
+     * @return Ignite component or {@code null} if component is not supported.
      */
-    public void initExtensions(PluginContext ctx, ExtensionRegistry registry);
+    @Nullable public <T> T createComponent(PluginContext ctx, Class<T> cls);
 
     /**
      * Starts grid component.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 871512c..6e7a706 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -23,13 +23,14 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
+import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.swapspace.*;
+
 import org.jetbrains.annotations.*;
 
 import javax.management.*;
@@ -197,7 +198,7 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
      * Inject ignite instance.
      */
     @IgniteInstanceResource
-    protected void injectResources(Ignite ignite){
+    protected void injectResources(Ignite ignite) {
         this.ignite = ignite;
 
         if (ignite != null) {
@@ -453,19 +454,20 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
         boolean isSpiConsistent = false;
 
-        String tipStr = " (fix configuration or set " + "-D" + IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK + "=true system property)";
+        String tipStr = " (fix configuration or set " +
+            "-D" + IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK + "=true system property)";
 
         if (rmtCls == null) {
             if (!optional && starting)
-                throw new IgniteSpiException("Remote SPI with the same name is not configured" + tipStr + " [name=" + name +
-                    ", loc=" + locCls + ']');
+                throw new IgniteSpiException("Remote SPI with the same name is not configured" + tipStr +
+                    " [name=" + name + ", loc=" + locCls + ']');
 
             sb.a(format(">>> Remote SPI with the same name is not configured: " + name, locCls));
         }
         else if (!locCls.equals(rmtCls)) {
             if (!optional && starting)
-                throw new IgniteSpiException("Remote SPI with the same name is of different type" + tipStr + " [name=" + name +
-                    ", loc=" + locCls + ", rmt=" + rmtCls + ']');
+                throw new IgniteSpiException("Remote SPI with the same name is of different type" + tipStr +
+                    " [name=" + name + ", loc=" + locCls + ", rmt=" + rmtCls + ']');
 
             sb.a(format(">>> Remote SPI with the same name is of different type: " + name, locCls, rmtCls));
         }
@@ -542,9 +544,25 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
     }
 
     /**
+     * @param obj Timeout object.
+     * @see IgniteSpiContext#addTimeoutObject(IgniteSpiTimeoutObject)
+     */
+    protected void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+        spiCtx.addTimeoutObject(obj);
+    }
+
+    /**
+     * @param obj Timeout object.
+     * @see IgniteSpiContext#removeTimeoutObject(IgniteSpiTimeoutObject)
+     */
+    protected void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+        spiCtx.removeTimeoutObject(obj);
+    }
+
+    /**
      * Temporarily SPI context.
      */
-    private static class GridDummySpiContext implements IgniteSpiContext {
+    private class GridDummySpiContext implements IgniteSpiContext {
         /** */
         private final ClusterNode locNode;
 
@@ -627,27 +645,11 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public void writeToSwap(String spaceName, Object key, @Nullable Object val,
-            @Nullable ClassLoader ldr) {
-            /* No-op. */
-        }
-
-        /** {@inheritDoc} */
-        @Override public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
         @Override public int partition(String cacheName, Object key) {
             return -1;
         }
 
         /** {@inheritDoc} */
-        @Override public void removeFromSwap(String spaceName, Object key, @Nullable ClassLoader ldr) {
-            // No-op.
-        }
-
-        /** {@inheritDoc} */
         @Override public Collection<ClusterNode> nodes() {
             return  locNode == null  ? Collections.<ClusterNode>emptyList() : Collections.singletonList(locNode);
         }
@@ -713,12 +715,6 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public <T> T readValueFromOffheapAndSwap(@Nullable String spaceName, Object key,
-            @Nullable ClassLoader ldr) {
-            return null;
-        }
-
-        /** {@inheritDoc} */
         @Override public MessageFormatter messageFormatter() {
             return msgFormatter;
         }
@@ -737,5 +733,19 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         @Override public boolean tryFailNode(UUID nodeId) {
             return false;
         }
+
+        /** {@inheritDoc} */
+        @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+            assert ignite instanceof IgniteKernal : ignite;
+
+            ((IgniteKernal)ignite).context().timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+            assert ignite instanceof IgniteKernal : ignite;
+
+            ((IgniteKernal)ignite).context().timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index 6852b6d..f83326c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
-import org.apache.ignite.spi.swapspace.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
@@ -253,30 +252,6 @@ public interface IgniteSpiContext {
     public <K> boolean containsKey(String cacheName, K key);
 
     /**
-     * Writes object to swap.
-     *
-     * @param spaceName Swap space name.
-     * @param key Key.
-     * @param val Value.
-     * @param ldr Class loader (optional).
-     * @throws IgniteException If any exception occurs.
-     */
-    public void writeToSwap(String spaceName, Object key, @Nullable Object val, @Nullable ClassLoader ldr)
-        throws IgniteException;
-
-    /**
-     * Reads object from swap.
-     *
-     * @param spaceName Swap space name.
-     * @param key Key.
-     * @param ldr Class loader (optional).
-     * @return Swapped value.
-     * @throws IgniteException If any exception occurs.
-     */
-    @Nullable public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr)
-        throws IgniteException;
-
-    /**
      * Calculates partition number for given key.
      *
      * @param cacheName Cache name.
@@ -286,16 +261,6 @@ public interface IgniteSpiContext {
     public int partition(String cacheName, Object key);
 
     /**
-     * Removes object from swap.
-     *
-     * @param spaceName Swap space name.
-     * @param key Key.
-     * @param ldr Class loader (optional).
-     * @throws IgniteException If any exception occurs.
-     */
-    public void removeFromSwap(String spaceName, Object key, @Nullable ClassLoader ldr) throws IgniteException;
-
-    /**
      * Validates that new node can join grid topology, this method is called on coordinator
      * node before new node joins topology.
      *
@@ -322,18 +287,6 @@ public interface IgniteSpiContext {
     public SecuritySubject authenticatedSubject(UUID subjId) throws IgniteException;
 
     /**
-     * Reads swapped cache value from off-heap and swap.
-     *
-     * @param spaceName Off-heap space name.
-     * @param key Key.
-     * @param ldr Class loader for unmarshalling.
-     * @return Value.
-     * @throws IgniteException If any exception occurs.
-     */
-    @Nullable public <T> T readValueFromOffheapAndSwap(@Nullable String spaceName, Object key,
-        @Nullable ClassLoader ldr) throws IgniteException;
-
-    /**
      * Gets message formatter.
      *
      * @return Message formatter.
@@ -357,4 +310,14 @@ public interface IgniteSpiContext {
      * @return If node was failed.
      */
     public boolean tryFailNode(UUID nodeId);
+
+    /**
+     * @param c Timeout object.
+     */
+    public void addTimeoutObject(IgniteSpiTimeoutObject c);
+
+    /**
+     * @param c Timeout object.
+     */
+    public void removeTimeoutObject(IgniteSpiTimeoutObject c);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
new file mode 100644
index 0000000..b3fc28e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiTimeoutObject.java
@@ -0,0 +1,44 @@
+/*
+ * 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.spi;
+
+import org.apache.ignite.lang.*;
+
+/**
+ * Provides possibility to schedule delayed execution,
+ * see {@link IgniteSpiContext#addTimeoutObject(IgniteSpiTimeoutObject)}.
+ * <p>
+ * Note: all timeout objects are executed in single dedicated thread, so implementation
+ * of {@link #onTimeout()} should not use time consuming and blocking method.
+ */
+public interface IgniteSpiTimeoutObject {
+    /**
+     * @return Unique object ID.
+     */
+    public IgniteUuid id();
+
+    /**
+     * @return End time.
+     */
+    public long endTime();
+
+    /**
+     * Timeout callback.
+     */
+    public void onTimeout();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/noop/NoopCheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/noop/NoopCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/noop/NoopCheckpointSpi.java
index 460cff3..832d872 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/noop/NoopCheckpointSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/noop/NoopCheckpointSpi.java
@@ -51,8 +51,7 @@ public class NoopCheckpointSpi extends IgniteSpiAdapter implements CheckpointSpi
     }
 
     /** {@inheritDoc} */
-    @Override
-    public boolean saveCheckpoint(String key, byte[] state, long timeout, boolean overwrite) throws IgniteSpiException {
+    @Override public boolean saveCheckpoint(String key, byte[] state, long timeout, boolean overwrite) {
         return false;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index fd17791..359de1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -157,12 +157,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Default idle connection timeout (value is <tt>30000</tt>ms). */
     public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
 
-    /** Default value for connection buffer flush frequency (value is <tt>100</tt> ms). */
-    public static final long DFLT_CONN_BUF_FLUSH_FREQ = 100;
-
-    /** Default value for connection buffer size (value is <tt>0</tt>). */
-    public static final int DFLT_CONN_BUF_SIZE = 0;
-
     /** Default socket send and receive buffer size. */
     public static final int DFLT_SOCK_BUF_SIZE = 32 * 1024;
 
@@ -267,7 +261,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                                                 log.debug("Session was closed but there are unacknowledged messages, " +
                                                     "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
 
-                                            recoveryWorker.addReconnectRequest(recoveryData);
+                                            commWorker.addReconnectRequest(recoveryData);
                                         }
                                     }
                                     else
@@ -603,13 +597,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Idle connection timeout. */
     private long idleConnTimeout = DFLT_IDLE_CONN_TIMEOUT;
 
-    /** Connection buffer flush frequency. */
-    private volatile long connBufFlushFreq = DFLT_CONN_BUF_FLUSH_FREQ;
-
-    /** Connection buffer size. */
-    @SuppressWarnings("RedundantFieldInitialization")
-    private int connBufSize = DFLT_CONN_BUF_SIZE;
-
     /** Connect timeout. */
     private long connTimeout = DFLT_CONN_TIMEOUT;
 
@@ -647,17 +634,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Socket write timeout. */
     private long sockWriteTimeout = DFLT_SOCK_WRITE_TIMEOUT;
 
-    /** Idle client worker. */
-    private IdleClientWorker idleClientWorker;
-
-    /** Flush client worker. */
-    private ClientFlushWorker clientFlushWorker;
-
-    /** Socket timeout worker. */
-    private SocketTimeoutWorker sockTimeoutWorker;
-
-    /** Recovery worker. */
-    private RecoveryWorker recoveryWorker;
+    /** Recovery and idle clients handler. */
+    private CommunicationWorker commWorker;
 
     /** Clients. */
     private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
@@ -882,31 +860,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /**
      * Sets connection buffer size. If set to {@code 0} connection buffer is disabled.
-     * <p>
-     * If not provided, default value is {@link #DFLT_CONN_BUF_SIZE}.
      *
      * @param connBufSize Connection buffer size.
      * @see #setConnectionBufferFlushFrequency(long)
      */
     @IgniteSpiConfiguration(optional = true)
     public void setConnectionBufferSize(int connBufSize) {
-        this.connBufSize = connBufSize;
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public int getConnectionBufferSize() {
-        return connBufSize;
+        return 0;
     }
 
     /** {@inheritDoc} */
     @IgniteSpiConfiguration(optional = true)
     @Override public void setConnectionBufferFlushFrequency(long connBufFlushFreq) {
-        this.connBufFlushFreq = connBufFlushFreq;
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public long getConnectionBufferFlushFrequency() {
-        return connBufFlushFreq;
+        return 0;
     }
 
     /**
@@ -1174,8 +1150,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(locPort <= 0xffff, "locPort < 0xffff");
         assertParameter(locPortRange >= 0, "locPortRange >= 0");
         assertParameter(idleConnTimeout > 0, "idleConnTimeout > 0");
-        assertParameter(connBufFlushFreq > 0, "connBufFlushFreq > 0");
-        assertParameter(connBufSize >= 0, "connBufSize >= 0");
         assertParameter(sockRcvBuf >= 0, "sockRcvBuf >= 0");
         assertParameter(sockSndBuf >= 0, "sockSndBuf >= 0");
         assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
@@ -1245,8 +1219,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("idleConnTimeout", idleConnTimeout));
             log.debug(configInfo("directBuf", directBuf));
             log.debug(configInfo("directSendBuf", directSndBuf));
-            log.debug(configInfo("connBufSize", connBufSize));
-            log.debug(configInfo("connBufFlushFreq", connBufFlushFreq));
             log.debug(configInfo("selectorsCnt", selectorsCnt));
             log.debug(configInfo("tcpNoDelay", tcpNoDelay));
             log.debug(configInfo("sockSndBuf", sockSndBuf));
@@ -1261,11 +1233,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("unackedMsgsBufSize", unackedMsgsBufSize));
         }
 
-        if (connBufSize > 8192)
-            U.warn(log, "Specified communication IO buffer size is larger than recommended (ignore if done " +
-                "intentionally) [specified=" + connBufSize + ", recommended=8192]",
-                "Specified communication IO buffer size is larger than recommended (ignore if done intentionally).");
-
         if (!tcpNoDelay)
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
@@ -1274,23 +1241,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         nioSrvr.start();
 
-        idleClientWorker = new IdleClientWorker();
+        commWorker = new CommunicationWorker();
 
-        idleClientWorker.start();
-
-        recoveryWorker = new RecoveryWorker();
-
-        recoveryWorker.start();
-
-        if (connBufSize > 0) {
-            clientFlushWorker = new ClientFlushWorker();
-
-            clientFlushWorker.start();
-        }
-
-        sockTimeoutWorker = new SocketTimeoutWorker();
-
-        sockTimeoutWorker.start();
+        commWorker.start();
 
         // Ack start.
         if (log.isDebugEnabled())
@@ -1445,15 +1398,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (nioSrvr != null)
             nioSrvr.stop();
 
-        U.interrupt(idleClientWorker);
-        U.interrupt(clientFlushWorker);
-        U.interrupt(sockTimeoutWorker);
-        U.interrupt(recoveryWorker);
+        U.interrupt(commWorker);
 
-        U.join(idleClientWorker, log);
-        U.join(clientFlushWorker, log);
-        U.join(sockTimeoutWorker, log);
-        U.join(recoveryWorker, log);
+        U.join(commWorker, log);
 
         // Force closing on stop (safety).
         for (GridCommunicationClient client : clients.values())
@@ -1461,7 +1408,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         // Clear resources.
         nioSrvr = null;
-        idleClientWorker = null;
+        commWorker = null;
 
         boundTcpPort = -1;
 
@@ -1899,7 +1846,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     ) throws IgniteCheckedException {
         HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
 
-        sockTimeoutWorker.addTimeoutObject(obj);
+        addTimeoutObject(obj);
 
         long rcvCnt = 0;
 
@@ -2005,7 +1952,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             boolean cancelled = obj.cancel();
 
             if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
+                removeTimeoutObject(obj);
 
             // Ignoring whatever happened after timeout - reporting only timeout event.
             if (!cancelled)
@@ -2041,15 +1988,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (nioSrvr != null)
             nioSrvr.stop();
 
-        U.interrupt(idleClientWorker);
-        U.interrupt(clientFlushWorker);
-        U.interrupt(sockTimeoutWorker);
-        U.interrupt(recoveryWorker);
+        U.interrupt(commWorker);
 
-        U.join(idleClientWorker, log);
-        U.join(clientFlushWorker, log);
-        U.join(sockTimeoutWorker, log);
-        U.join(recoveryWorker, log);
+        U.join(commWorker, log);
 
         for (GridCommunicationClient client : clients.values())
             client.forceClose();
@@ -2156,80 +2097,95 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      *
      */
-    private class IdleClientWorker extends IgniteSpiThread {
+    private class CommunicationWorker extends IgniteSpiThread {
+        /** */
+        private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();
+
         /**
          *
          */
-        IdleClientWorker() {
-            super(gridName, "nio-idle-client-collector", log);
+        private CommunicationWorker() {
+            super(gridName, "tcp-comm-worker", log);
         }
 
         /** {@inheritDoc} */
-        @SuppressWarnings({"BusyWait"})
         @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("Tcp communication worker has been started.");
+
             while (!isInterrupted()) {
-                cleanupRecovery();
+                GridNioRecoveryDescriptor recoveryDesc = q.poll(idleConnTimeout, TimeUnit.MILLISECONDS);
 
-                for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
-                    UUID nodeId = e.getKey();
+                if (recoveryDesc != null)
+                    processRecovery(recoveryDesc);
+                else
+                    processIdle();
+            }
+        }
 
-                    GridCommunicationClient client = e.getValue();
+        /**
+         *
+         */
+        private void processIdle() {
+            cleanupRecovery();
 
-                    ClusterNode node = getSpiContext().node(nodeId);
+            for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
+                UUID nodeId = e.getKey();
 
-                    if (node == null) {
-                        if (log.isDebugEnabled())
-                            log.debug("Forcing close of non-existent node connection: " + nodeId);
+                GridCommunicationClient client = e.getValue();
 
-                        client.forceClose();
+                ClusterNode node = getSpiContext().node(nodeId);
 
-                        clients.remove(nodeId, client);
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Forcing close of non-existent node connection: " + nodeId);
 
-                        continue;
-                    }
+                    client.forceClose();
 
-                    GridNioRecoveryDescriptor recovery = null;
+                    clients.remove(nodeId, client);
 
-                    if (client instanceof GridTcpNioCommunicationClient) {
-                        recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
+                    continue;
+                }
 
-                        if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
-                            RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+                GridNioRecoveryDescriptor recovery = null;
 
-                            if (log.isDebugEnabled())
-                                log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
-                                    ", rcvCnt=" + msg.received() + ']');
+                if (client instanceof GridTcpNioCommunicationClient) {
+                    recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
 
-                            nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
+                    if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+                        RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
 
-                            recovery.lastAcknowledged(msg.received());
+                        if (log.isDebugEnabled())
+                            log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
+                                ", rcvCnt=" + msg.received() + ']');
 
-                            continue;
-                        }
-                    }
+                        nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
 
-                    long idleTime = client.getIdleTime();
+                        recovery.lastAcknowledged(msg.received());
 
-                    if (idleTime >= idleConnTimeout) {
-                        if (recovery != null &&
-                            recovery.nodeAlive(getSpiContext().node(nodeId)) &&
-                            !recovery.messagesFutures().isEmpty()) {
-                            if (log.isDebugEnabled())
-                                log.debug("Node connection is idle, but there are unacknowledged messages, " +
-                                    "will wait: " + nodeId);
+                        continue;
+                    }
+                }
 
-                            continue;
-                        }
+                long idleTime = client.getIdleTime();
 
+                if (idleTime >= idleConnTimeout) {
+                    if (recovery != null &&
+                        recovery.nodeAlive(getSpiContext().node(nodeId)) &&
+                        !recovery.messagesFutures().isEmpty()) {
                         if (log.isDebugEnabled())
-                            log.debug("Closing idle node connection: " + nodeId);
+                            log.debug("Node connection is idle, but there are unacknowledged messages, " +
+                                "will wait: " + nodeId);
 
-                        if (client.close() || client.closed())
-                            clients.remove(nodeId, client);
+                        continue;
                     }
-                }
 
-                Thread.sleep(idleConnTimeout);
+                    if (log.isDebugEnabled())
+                        log.debug("Closing idle node connection: " + nodeId);
+
+                    if (client.close() || client.closed())
+                        clients.remove(nodeId, client);
+                }
             }
         }
 
@@ -2264,212 +2220,39 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 }
             }
         }
-    }
-
-    /**
-     *
-     */
-    private class ClientFlushWorker extends IgniteSpiThread {
-        /**
-         *
-         */
-        ClientFlushWorker() {
-            super(gridName, "nio-client-flusher", log);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings({"BusyWait"})
-        @Override protected void body() throws InterruptedException {
-            while (!isInterrupted()) {
-                long connBufFlushFreq0 = connBufFlushFreq;
-
-                for (Map.Entry<UUID, GridCommunicationClient> entry : clients.entrySet()) {
-                    GridCommunicationClient client = entry.getValue();
-
-                    if (client.reserve()) {
-                        boolean err = true;
-
-                        try {
-                            client.flushIfNeeded(connBufFlushFreq0);
-
-                            err = false;
-                        }
-                        catch (IOException e) {
-                            if (getSpiContext().pingNode(entry.getKey()))
-                                U.error(log, "Failed to flush client: " + client, e);
-                            else {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to flush client (node left): " + client);
-
-                                onException("Failed to flush client (node left): " + client, e);
-                            }
-                        }
-                        finally {
-                            if (err)
-                                client.forceClose();
-                            else
-                                client.release();
-                        }
-                    }
-                }
-
-                Thread.sleep(connBufFlushFreq0);
-            }
-        }
-    }
-
-    /**
-     * Handles sockets timeouts.
-     */
-    private class SocketTimeoutWorker extends IgniteSpiThread {
-        /** Time-based sorted set for timeout objects. */
-        private final GridConcurrentSkipListSet<HandshakeTimeoutObject> timeoutObjs =
-            new GridConcurrentSkipListSet<>(new Comparator<HandshakeTimeoutObject>() {
-                @Override public int compare(HandshakeTimeoutObject o1, HandshakeTimeoutObject o2) {
-                    long time1 = o1.endTime();
-                    long time2 = o2.endTime();
-
-                    long id1 = o1.id();
-                    long id2 = o2.id();
-
-                    return time1 < time2 ? -1 : time1 > time2 ? 1 :
-                        id1 < id2 ? -1 : id1 > id2 ? 1 : 0;
-                }
-            });
-
-        /** Mutex. */
-        private final Object mux0 = new Object();
-
-        /**
-         *
-         */
-        SocketTimeoutWorker() {
-            super(gridName, "tcp-comm-sock-timeout-worker", log);
-        }
-
-        /**
-         * @param timeoutObj Timeout object to add.
-         */
-        @SuppressWarnings({"NakedNotify"})
-        public void addTimeoutObject(HandshakeTimeoutObject timeoutObj) {
-            assert timeoutObj != null && timeoutObj.endTime() > 0 && timeoutObj.endTime() != Long.MAX_VALUE;
-
-            timeoutObjs.add(timeoutObj);
-
-            if (timeoutObjs.firstx() == timeoutObj) {
-                synchronized (mux0) {
-                    mux0.notifyAll();
-                }
-            }
-        }
 
         /**
-         * @param timeoutObj Timeout object to remove.
+         * @param recoveryDesc Recovery descriptor.
          */
-        public void removeTimeoutObject(HandshakeTimeoutObject timeoutObj) {
-            assert timeoutObj != null;
-
-            timeoutObjs.remove(timeoutObj);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            if (log.isDebugEnabled())
-                log.debug("Socket timeout worker has been started.");
+        private void processRecovery(GridNioRecoveryDescriptor recoveryDesc) {
+            ClusterNode node = recoveryDesc.node();
 
-            while (!isInterrupted()) {
-                long now = U.currentTimeMillis();
-
-                for (Iterator<HandshakeTimeoutObject> iter = timeoutObjs.iterator(); iter.hasNext(); ) {
-                    HandshakeTimeoutObject timeoutObj = iter.next();
-
-                    if (timeoutObj.endTime() <= now) {
-                        iter.remove();
+            if (clients.containsKey(node.id()) || !recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+                return;
 
-                        timeoutObj.onTimeout();
-                    }
-                    else
-                        break;
-                }
-
-                synchronized (mux0) {
-                    while (true) {
-                        // Access of the first element must be inside of
-                        // synchronization block, so we don't miss out
-                        // on thread notification events sent from
-                        // 'addTimeoutObject(..)' method.
-                        HandshakeTimeoutObject first = timeoutObjs.firstx();
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
 
-                        if (first != null) {
-                            long waitTime = first.endTime() - U.currentTimeMillis();
+                GridCommunicationClient client = reserveClient(node);
 
-                            if (waitTime > 0)
-                                mux0.wait(waitTime);
-                            else
-                                break;
-                        }
-                        else
-                            mux0.wait(5000);
-                    }
-                }
+                client.release();
             }
-        }
-    }
-
-    /**
-     *
-     */
-    private class RecoveryWorker extends IgniteSpiThread {
-        /** */
-        private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();
-
-        /**
-         *
-         */
-        private RecoveryWorker() {
-            super(gridName, "tcp-comm-recovery-worker", log);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            if (log.isDebugEnabled())
-                log.debug("Recovery worker has been started.");
-
-            while (!isInterrupted()) {
-                GridNioRecoveryDescriptor recoveryDesc = q.take();
-
-                assert recoveryDesc != null;
-
-                ClusterNode node = recoveryDesc.node();
-
-                if (clients.containsKey(node.id()) || !recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
-                    continue;
-
-                try {
+            catch (IgniteCheckedException | IgniteException e) {
+                if (recoveryDesc.nodeAlive(getSpiContext().node(node.id()))) {
                     if (log.isDebugEnabled())
-                        log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
-
-                    GridCommunicationClient client = reserveClient(node);
+                        log.debug("Recovery reconnect failed, will retry " +
+                            "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
 
-                    client.release();
+                    addReconnectRequest(recoveryDesc);
                 }
-                catch (IgniteCheckedException e) {
-                    if (recoveryDesc.nodeAlive(getSpiContext().node(node.id()))) {
-                        if (log.isDebugEnabled())
-                            log.debug("Recovery reconnect failed, will retry " +
-                                "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
-
-                        addReconnectRequest(recoveryDesc);
-                    }
-                    else {
-                        if (log.isDebugEnabled())
-                            log.debug("Recovery reconnect failed, " +
-                                "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
-
-                        onException("Recovery reconnect failed, node left [rmtNode=" + recoveryDesc.node().id() + "]",
-                            e);
-                    }
+                else {
+                    if (log.isDebugEnabled())
+                        log.debug("Recovery reconnect failed, " +
+                            "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
 
+                    onException("Recovery reconnect failed, node left [rmtNode=" + recoveryDesc.node().id() + "]",
+                        e);
                 }
             }
         }
@@ -2497,12 +2280,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      *
      */
-    private static class HandshakeTimeoutObject<T> {
-        /** */
-        private static final AtomicLong idGen = new AtomicLong();
-
+    private static class HandshakeTimeoutObject<T> implements IgniteSpiTimeoutObject {
         /** */
-        private final long id = idGen.incrementAndGet();
+        private final IgniteUuid id = IgniteUuid.randomUuid();
 
         /** */
         private final T obj;
@@ -2533,34 +2313,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             return done.compareAndSet(false, true);
         }
 
-        /**
-         * @return {@code True} if object has not yet been canceled.
-         */
-        boolean onTimeout() {
+        /** {@inheritDoc} */
+        @Override public void onTimeout() {
             if (done.compareAndSet(false, true)) {
                 // Close socket - timeout occurred.
                 if (obj instanceof GridCommunicationClient)
                     ((GridCommunicationClient)obj).forceClose();
                 else
                     U.closeQuiet((AbstractInterruptibleChannel)obj);
-
-                return true;
             }
-
-            return false;
         }
 
-        /**
-         * @return End time.
-         */
-        long endTime() {
+        /** {@inheritDoc} */
+        @Override public long endTime() {
             return endTime;
         }
 
-        /**
-         * @return ID.
-         */
-        long id() {
+        /** {@inheritDoc} */
+        @Override public IgniteUuid id() {
             return id;
         }
 



[05/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
new file mode 100644
index 0000000..12b6458
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultinodeCreateCacheTest.java
@@ -0,0 +1,97 @@
+/*
+ * 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.datastreamer;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class DataStreamerMultinodeCreateCacheTest 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);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setSocketTimeout(50);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setAckTimeout(50);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCreateCacheAndStream() throws Exception {
+        final int THREADS = 5;
+
+        startGrids(THREADS);
+
+        final AtomicInteger idx = new AtomicInteger();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int threadIdx = idx.getAndIncrement();
+
+                long stopTime = System.currentTimeMillis() + 60_000;
+
+                Ignite ignite = grid(threadIdx);
+
+                int iter = 0;
+
+                while (System.currentTimeMillis() < stopTime) {
+                    String cacheName = "cache-" + threadIdx + "-" + (iter % 10);
+
+                    try (IgniteCache<Integer, String> cache = ignite.getOrCreateCache(cacheName)) {
+                        try (IgniteDataStreamer<Object, Object> stmr = ignite.dataStreamer(cacheName)) {
+                            ((DataStreamerImpl<Object, Object>)stmr).maxRemapCount(0);
+
+                            for (int i = 0; i < 1000; i++)
+                                stmr.addData(i, i);
+                        }
+                    }
+
+                    iter++;
+                }
+
+                return null;
+            }
+        }, THREADS, "create-cache");
+
+        fut.get(2 * 60_000);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
index d983302..9cda1b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsClientCacheSelfTest.java
@@ -85,13 +85,16 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
         cfg.setCacheConfiguration(cacheConfiguration(META_CACHE_NAME), cacheConfiguration(DATA_CACHE_NAME),
             cacheConfiguration(CACHE_NAME));
 
-        if (!gridName.equals(getTestGridName(0)))
-            cfg.setClientMode(true);
-
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(IP_FINDER);
 
+        if (!gridName.equals(getTestGridName(0))) {
+            cfg.setClientMode(true);
+
+            disco.setForceServerMode(true);
+        }
+
         cfg.setDiscoverySpi(disco);
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
@@ -114,11 +117,12 @@ public class IgfsClientCacheSelfTest extends IgfsAbstractSelfTest {
 
         cacheCfg.setName(cacheName);
 
+        cacheCfg.setNearConfiguration(null);
+
         if (META_CACHE_NAME.equals(cacheName))
             cacheCfg.setCacheMode(REPLICATED);
         else {
             cacheCfg.setCacheMode(PARTITIONED);
-            cacheCfg.setNearConfiguration(null);
 
             cacheCfg.setBackups(0);
             cacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(128));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCommonAbstractTest.java
index fa7f048..e879130 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsCommonAbstractTest.java
@@ -46,16 +46,6 @@ public class IgfsCommonAbstractTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(IgniteTestResources rsrcs) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(rsrcs);
-
-        cfg.setPeerClassLoadingEnabled(false);
-        cfg.setLocalHost("127.0.0.1");
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName, IgniteTestResources rsrcs) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName, rsrcs);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
index 49ddb03..3498cd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
@@ -52,11 +52,9 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(true);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi()
+            .setForceServerMode(true)
+            .setIpFinder(new TcpDiscoveryVmIpFinder(true)));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index 761f00f..faccc9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -53,11 +53,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(new OptimizedMarshaller(false));
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
 
         cfg.setCacheConfiguration();
 
@@ -88,6 +84,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
             srvNames.add(getTestGridName(i));
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute().broadcast(new IgniteCallable<String>() {
@@ -113,6 +111,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testClientClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forClients()).
@@ -138,6 +138,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testCustomClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forPredicate(F.<ClusterNode>alwaysTrue())).
@@ -161,6 +163,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services().deployNodeSingleton(SINGLETON_NAME, new TestService());
@@ -194,6 +198,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services(ignite.cluster().forClients()).deployNodeSingleton(SINGLETON_NAME, new TestService());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
new file mode 100644
index 0000000..404c32b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceClientNodeTest.java
@@ -0,0 +1,81 @@
+/*
+ * 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.service;
+
+import org.apache.ignite.*;
+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.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class GridServiceClientNodeTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 3;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeployFromClient() throws Exception {
+        Ignite ignite = ignite(NODE_CNT - 1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        String svcName = "testService";
+
+        CountDownLatch latch = new CountDownLatch(1);
+
+        DummyService.exeLatch(svcName, latch);
+
+        ignite.services().deployClusterSingleton(svcName, new DummyService());
+
+        assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
new file mode 100644
index 0000000..cbf7d89
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioDelimitedBufferSelfTest.java
@@ -0,0 +1,112 @@
+/*
+ * 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.util.nio;
+
+import junit.framework.TestCase;
+
+import java.nio.*;
+import java.util.*;
+
+/**
+ * Tests for {@link GridNioDelimitedBuffer}.
+ */
+public class GridNioDelimitedBufferSelfTest extends TestCase {
+    /** */
+    private static final String ASCII = "ASCII";
+
+    /**
+     * Tests simple delimiter (excluded from alphabet)
+     */
+    public void testReadZString() throws Exception {
+        Random rnd = new Random();
+
+        int buffSize = 0;
+
+        byte[] delim = new byte[] {0};
+
+        List<String> strs = new ArrayList<>(50);
+
+        for (int i = 0; i < 50; i++) {
+            int len = rnd.nextInt(128) + 1;
+
+            buffSize += len + delim.length;
+
+            StringBuilder sb = new StringBuilder(len);
+
+            for (int j = 0; j < len; j++)
+                sb.append((char)(rnd.nextInt(26) + 'a'));
+
+
+            strs.add(sb.toString());
+        }
+
+        ByteBuffer buff = ByteBuffer.allocate(buffSize);
+
+        for (String str : strs) {
+            buff.put(str.getBytes(ASCII));
+            buff.put(delim);
+        }
+
+        buff.flip();
+
+        byte[] msg;
+
+        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
+
+        List<String> res = new ArrayList<>(strs.size());
+
+        while ((msg = delimBuff.read(buff)) != null)
+            res.add(new String(msg, ASCII));
+
+        assertEquals(strs, res);
+    }
+
+    /**
+     * Tests compound delimiter (included to alphabet)
+     */
+    public void testDelim() throws Exception {
+        byte[] delim = "aabb".getBytes(ASCII);
+
+        List<String> strs = Arrays.asList("za", "zaa", "zaab", "zab", "zaabaababbbbabaab");
+
+        int buffSize = 0;
+
+        for (String str : strs)
+            buffSize += str.length() + delim.length;
+
+        ByteBuffer buff = ByteBuffer.allocate(buffSize);
+
+        for (String str : strs) {
+            buff.put(str.getBytes(ASCII));
+            buff.put(delim);
+        }
+
+        buff.flip();
+
+        byte[] msg;
+
+        GridNioDelimitedBuffer delimBuff = new GridNioDelimitedBuffer(delim);
+
+        List<String> res = new ArrayList<>(strs.size());
+
+        while ((msg = delimBuff.read(buff)) != null)
+            res.add(new String(msg, ASCII));
+
+        assertEquals(strs, res);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
index e3baeb0..bdf9929 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
@@ -1286,7 +1286,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Test client to use instead of {@link GridTcpCommunicationClient}
+     * Test client to use instead of {@link GridTcpNioCommunicationClient}
      */
     private static class TestClient implements AutoCloseable {
         /** Socket implementation to use. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
index d106f5b..0030ce1 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/GridCacheMultiNodeLoadTest.java
@@ -65,7 +65,10 @@ public class GridCacheMultiNodeLoadTest extends GridCommonAbstractTest {
         cacheCfg.setStartSize(10);
         cacheCfg.setWriteSynchronizationMode(FULL_SYNC);
 
-        cacheCfg.setEvictionPolicy(new LruEvictionPolicy(100000));
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(100000);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setBackups(1);
 
         cacheCfg.setRebalanceMode(SYNC);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
index 6960fa1..0d9ec8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/datastructures/GridCachePartitionedAtomicLongLoadTest.java
@@ -70,7 +70,11 @@ public class GridCachePartitionedAtomicLongLoadTest extends GridCommonAbstractTe
         cc.setStartSize(200);
         cc.setRebalanceMode(CacheRebalanceMode.SYNC);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictionPolicy(new LruEvictionPolicy<>(1000));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setBackups(1);
         cc.setAffinity(new RendezvousAffinityFunction(true));
         cc.setEvictSynchronized(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index 0b0f099..77d3905 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -54,11 +54,13 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheMvccManager(),
                 new GridCacheDeploymentManager<K, V>(),
                 new GridCachePartitionExchangeManager<K, V>(),
-                new GridCacheIoManager()
+                new GridCacheIoManager(),
+                null
             ),
             defaultCacheConfiguration(),
             CacheType.USER,
             true,
+            true,
             new GridCacheEventManager(),
             new GridCacheSwapManager(false),
             new CacheOsStoreManager(null, new CacheConfiguration()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java b/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
index 1561b77..62066da 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/swap/GridSwapEvictAllBenchmark.java
@@ -266,7 +266,11 @@ public class GridSwapEvictAllBenchmark {
 
         ccfg.setSwapEnabled(true);
         ccfg.setEvictSynchronized(false);
-        ccfg.setEvictionPolicy(new FifoEvictionPolicy(EVICT_PLC_SIZE));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_PLC_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
 
         if (store != null) {
             ccfg.setCacheStoreFactory(new FactoryBuilder.SingletonFactory(store));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
index 07fd9e3..35abf7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/OptimizedMarshallerNodeFailoverTest.java
@@ -113,7 +113,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
 
         cache = true;
 
-        Ignite ignite = startGrid(2); // Check can start on more cache node.
+        Ignite ignite = startGrid(2); // Check can start one more cache node.
 
         assertNotNull(ignite.cache(null));
     }
@@ -122,7 +122,7 @@ public class OptimizedMarshallerNodeFailoverTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     public void testRestartAllNodes() throws Exception {
-        cache = false;
+        cache = true;
 
         String home = U.getIgniteHome();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
index 610ce64..37d2ec7 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingNoPeerClassLoadingSelfTest.java
@@ -31,6 +31,9 @@ import java.util.concurrent.atomic.*;
  * peer class loading.
  */
 public class GridMessagingNoPeerClassLoadingSelfTest extends GridMessagingSelfTest {
+    /** */
+    private static CountDownLatch rcvLatch;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -56,9 +59,9 @@ public class GridMessagingNoPeerClassLoadingSelfTest extends GridMessagingSelfTe
 
         final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
 
-        final CountDownLatch rcvLatch = new CountDownLatch(1);
+        rcvLatch = new CountDownLatch(1);
 
-        ignite2.message().remoteListen("", new P2<UUID, Object>() {
+        ignite2.message().remoteListen(null, new P2<UUID, Object>() {
             @Override public boolean apply(UUID nodeId, Object msg) {
                 try {
                     log.info("Received new message [msg=" + msg + ", senderNodeId=" + nodeId + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
index c033750..b7838be 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
@@ -45,7 +45,7 @@ import static org.apache.ignite.testframework.GridTestUtils.*;
 /**
  * Various tests for Messaging public API.
  */
-public class GridMessagingSelfTest extends GridCommonAbstractTest {
+public class GridMessagingSelfTest extends GridCommonAbstractTest implements Serializable {
     /** */
     private static final String MSG_1 = "MSG-1";
 
@@ -74,7 +74,10 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
     public static final String EXT_RESOURCE_CLS_NAME = "org.apache.ignite.tests.p2p.TestUserResource";
 
     /** Shared IP finder. */
-    private final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+    private final transient TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected static CountDownLatch rcvLatch;
 
     /**
      * A test message topic.
@@ -609,7 +612,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
     public void testRemoteListen() throws Exception {
         final Collection<Object> rcvMsgs = new GridConcurrentHashSet<>();
 
-        final CountDownLatch rcvLatch = new CountDownLatch(4);
+        rcvLatch = new CountDownLatch(4);
 
         ignite2.message().remoteListen(null, new P2<UUID, Object>() {
             @Override public boolean apply(UUID nodeId, Object msg) {
@@ -746,7 +749,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
 
-        final CountDownLatch rcvLatch = new CountDownLatch(3);
+        rcvLatch = new CountDownLatch(3);
 
         ignite2.message().remoteListen(S_TOPIC_1, new P2<UUID, Object>() {
             @Override public boolean apply(UUID nodeId, Object msg) {
@@ -795,7 +798,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean error = new AtomicBoolean(false); //to make it modifiable
 
-        final CountDownLatch rcvLatch = new CountDownLatch(3);
+        rcvLatch = new CountDownLatch(3);
 
         ignite2.message().remoteListen(I_TOPIC_1, new P2<UUID, Object>() {
             @IgniteInstanceResource

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
new file mode 100644
index 0000000..09abcdb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/IgniteMessagingWithClientTest.java
@@ -0,0 +1,166 @@
+/*
+ * 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.messaging;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+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.marshaller.optimized.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteMessagingWithClientTest extends GridCommonAbstractTest implements Serializable {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Message topic. */
+    private enum TOPIC {
+        /** */
+        ORDERED
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(new OptimizedMarshaller(false));
+
+        if (gridName.equals(getTestGridName(2))) {
+            cfg.setClientMode(true);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+        }
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMessageSendWithClientJoin() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-996");
+
+        startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        ClusterGroup rmts = ignite1.cluster().forRemotes();
+
+        IgniteMessaging msg = ignite1.message(rmts);
+
+        msg.localListen(TOPIC.ORDERED, new LocalListener());
+
+        msg.remoteListen(TOPIC.ORDERED, new RemoteListener());
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                int iter = 0;
+
+                while (!stop.get()) {
+                    if (iter % 10 == 0)
+                        log.info("Client start/stop iteration: " + iter);
+
+                    iter++;
+
+                    try (Ignite ignite = startGrid(2)) {
+                        assertTrue(ignite.configuration().isClientMode());
+                    }
+                }
+
+                return null;
+            }
+        }, 1, "client-start-stop");
+
+        try {
+            long stopTime = U.currentTimeMillis() + 30_000;
+
+            int iter = 0;
+
+            while (System.currentTimeMillis() < stopTime) {
+                try {
+                    ignite1.message(rmts).sendOrdered(TOPIC.ORDERED, Integer.toString(iter), 0);
+                }
+                catch (IgniteException e) {
+                    log.info("Message send failed: " + e);
+                }
+
+                iter++;
+
+                if (iter % 100 == 0)
+                    Thread.sleep(5);
+            }
+        }
+        finally {
+            stop.set(true);
+        }
+
+        fut.get();
+    }
+
+    /**
+     *
+     */
+    private static class LocalListener implements IgniteBiPredicate<UUID, String> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, String s) {
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class RemoteListener implements IgniteBiPredicate<UUID, String> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID nodeId, String msg) {
+            ignite.message(ignite.cluster().forNodeId(nodeId)).send(TOPIC.ORDERED, msg);
+
+            return true;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index ea51aff..8d27485 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -64,7 +64,7 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
 
         // Test idle clients remove.
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             assertEquals(2, clients.size());
 
@@ -77,7 +77,7 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         super.afterTest();
 
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             for (int i = 0; i < 20 && !clients.isEmpty(); i++) {
                 info("Check failed for SPI [grid=" +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index c038ee7..2d175f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -256,7 +256,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
                     assertTrue(latch.await(10, TimeUnit.SECONDS));
 
                     for (CommunicationSpi spi : spis) {
-                        ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+                        ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
                         assertEquals(1, clients.size());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index 34fa610..c4a0916 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -32,8 +32,6 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "localPort", 65636);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "localPortRange", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "idleConnectionTimeout", 0);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionBufferSize", -1);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionBufferFlushFrequency", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketReceiveBuffer", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketSendBuffer", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "messageQueueLimit", -1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index e7ae957..3916f02 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -501,7 +501,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         }
 
         for (CommunicationSpi spi : spis.values()) {
-            final ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+            final ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
 
             assert GridTestUtils.waitForCondition(new PA() {
                 @Override public boolean apply() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
index 3c61f00..61bb944 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
@@ -22,9 +22,11 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.config.*;
 import org.apache.ignite.testframework.junits.*;
 import org.apache.ignite.testframework.junits.spi.*;
+import org.jetbrains.annotations.*;
 
 import javax.management.*;
 import java.io.*;
@@ -132,7 +134,7 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
 
         /** {@inheritDoc} */
         @Override public void onDiscovery(int type, long topVer, ClusterNode node, Collection<ClusterNode> topSnapshot,
-            Map<Long, Collection<ClusterNode>> topHist, Serializable data) {
+            Map<Long, Collection<ClusterNode>> topHist, @Nullable DiscoverySpiCustomMessage data) {
             if (type == EVT_NODE_METRICS_UPDATED)
                 isMetricsUpdate = true;
         }
@@ -205,7 +207,7 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
             DiscoverySpiListener locHeartbeatLsnr = new DiscoverySpiListener() {
                 @Override public void onDiscovery(int type, long topVer, ClusterNode node,
                     Collection<ClusterNode> topSnapshot, Map<Long, Collection<ClusterNode>> topHist,
-                    Serializable data) {
+                    @Nullable DiscoverySpiCustomMessage data) {
                     // If METRICS_UPDATED came from local node
                     if (type == EVT_NODE_METRICS_UPDATED
                         && node.id().equals(spi.getLocalNode().id()))
@@ -266,9 +268,8 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
 
             Collection<UUID> nodeIds = new HashSet<>();
 
-            for (IgniteTestResources rsrc : spiRsrcs) {
+            for (IgniteTestResources rsrc : spiRsrcs)
                 nodeIds.add(rsrc.getNodeId());
-            }
 
             for (ClusterNode node : spi.getRemoteNodes()) {
                 if (nodeIds.contains(node.id())) {
@@ -369,7 +370,8 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                 spi.setListener(new DiscoverySpiListener() {
                     @SuppressWarnings({"NakedNotify"})
                     @Override public void onDiscovery(int type, long topVer, ClusterNode node,
-                        Collection<ClusterNode> topSnapshot, Map<Long, Collection<ClusterNode>> topHist, Serializable data) {
+                        Collection<ClusterNode> topSnapshot, Map<Long, Collection<ClusterNode>> topHist,
+                        @Nullable DiscoverySpiCustomMessage data) {
                         info("Discovery event [type=" + type + ", node=" + node + ']');
 
                         synchronized (mux) {
@@ -388,6 +390,10 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                     }
                 });
 
+                GridSpiTestContext ctx = initSpiContext();
+
+                GridTestUtils.setFieldValue(spi, IgniteSpiAdapter.class, "spiCtx", ctx);
+
                 spi.spiStart(getTestGridName() + i);
 
                 spis.add(spi);
@@ -395,7 +401,7 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
                 spiRsrcs.add(rsrcMgr);
 
                 // Force to use test context instead of default dummy context.
-                spi.onContextInitialized(initSpiContext());
+                spi.onContextInitialized(ctx);
             }
         }
         catch (Throwable e) {
@@ -436,9 +442,8 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
             spi.spiStop();
         }
 
-        for (IgniteTestResources rscrs : spiRsrcs) {
+        for (IgniteTestResources rscrs : spiRsrcs)
             rscrs.stopThreads();
-        }
 
         // Clear.
         spis.clear();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
new file mode 100644
index 0000000..016854a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.marshaller.jdk.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+/**
+ * Test for {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi}.
+ */
+public class TcpClientDiscoveryMarshallerCheckSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        if (gridName.endsWith("0"))
+            cfg.setMarshaller(new JdkMarshaller());
+        else {
+            cfg.setClientMode(true);
+
+            cfg.setMarshaller(new OptimizedMarshaller());
+        }
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshallerInConsistency() throws Exception {
+        startGrid(0);
+
+        try {
+            startGrid(1);
+
+            fail("Expected SPI exception was not thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            Throwable ex = e.getCause().getCause();
+
+            assertTrue(ex instanceof IgniteSpiException);
+            assertTrue(ex.getMessage().contains("Local node's marshaller differs from remote node's marshaller"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
deleted file mode 100644
index 0c9f2f2..0000000
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
+++ /dev/null
@@ -1,700 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spi.discovery.tcp;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.resources.*;
-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.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.events.EventType.*;
-
-/**
- * Client-based discovery tests.
- */
-public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private static final AtomicInteger srvIdx = new AtomicInteger();
-
-    /** */
-    private static final AtomicInteger clientIdx = new AtomicInteger();
-
-    /** */
-    private static Collection<UUID> srvNodeIds;
-
-    /** */
-    private static Collection<UUID> clientNodeIds;
-
-    /** */
-    private static int clientsPerSrv;
-
-    /** */
-    private static CountDownLatch srvJoinedLatch;
-
-    /** */
-    private static CountDownLatch srvLeftLatch;
-
-    /** */
-    private static CountDownLatch srvFailedLatch;
-
-    /** */
-    private static CountDownLatch clientJoinedLatch;
-
-    /** */
-    private static CountDownLatch clientLeftLatch;
-
-    /** */
-    private static CountDownLatch clientFailedLatch;
-
-    /** */
-    private static CountDownLatch msgLatch;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setLocalHost("127.0.0.1");
-
-        if (gridName.startsWith("server")) {
-            TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-            disco.setIpFinder(IP_FINDER);
-
-            cfg.setDiscoverySpi(disco);
-        }
-        else if (gridName.startsWith("client")) {
-            TcpClientDiscoverySpi disco = new TcpClientDiscoverySpi();
-
-            TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder();
-
-            String addr = new ArrayList<>(IP_FINDER.getRegisteredAddresses()).
-                get((clientIdx.get() - 1) / clientsPerSrv).toString();
-
-            if (addr.startsWith("/"))
-                addr = addr.substring(1);
-
-            ipFinder.setAddresses(Arrays.asList(addr));
-
-            disco.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(disco);
-        }
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        Collection<InetSocketAddress> addrs = IP_FINDER.getRegisteredAddresses();
-
-        if (!F.isEmpty(addrs))
-            IP_FINDER.unregisterAddresses(addrs);
-
-        srvIdx.set(0);
-        clientIdx.set(0);
-
-        srvNodeIds = new GridConcurrentHashSet<>();
-        clientNodeIds = new GridConcurrentHashSet<>();
-
-        clientsPerSrv = 2;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTest() throws Exception {
-        stopAllClients(true);
-        stopAllServers(true);
-
-        assert G.allGrids().isEmpty();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeJoin() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvJoinedLatch = new CountDownLatch(3);
-        clientJoinedLatch = new CountDownLatch(3);
-
-        attachListeners(3, 3);
-
-        startClientNodes(1);
-
-        await(srvJoinedLatch);
-        await(clientJoinedLatch);
-
-        checkNodes(3, 4);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeLeave() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvLeftLatch = new CountDownLatch(3);
-        clientLeftLatch = new CountDownLatch(2);
-
-        attachListeners(3, 3);
-
-        stopGrid("client-2");
-
-        await(srvLeftLatch);
-        await(clientLeftLatch);
-
-        checkNodes(3, 2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeFail() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvFailedLatch = new CountDownLatch(3);
-        clientFailedLatch = new CountDownLatch(2);
-
-        attachListeners(3, 3);
-
-        failClient(2);
-
-        await(srvFailedLatch);
-        await(clientFailedLatch);
-
-        checkNodes(3, 2);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testServerNodeJoin() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvJoinedLatch = new CountDownLatch(3);
-        clientJoinedLatch = new CountDownLatch(3);
-
-        attachListeners(3, 3);
-
-        startServerNodes(1);
-
-        await(srvJoinedLatch);
-        await(clientJoinedLatch);
-
-        checkNodes(4, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testServerNodeLeave() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvLeftLatch = new CountDownLatch(2);
-        clientLeftLatch = new CountDownLatch(3);
-
-        attachListeners(3, 3);
-
-        stopGrid("server-2");
-
-        await(srvLeftLatch);
-        await(clientLeftLatch);
-
-        checkNodes(2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testServerNodeFail() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        srvFailedLatch = new CountDownLatch(2);
-        clientFailedLatch = new CountDownLatch(3);
-
-        attachListeners(3, 3);
-
-        assert U.<Map>field(G.ignite("server-2").configuration().getDiscoverySpi(), "clientMsgWorkers").isEmpty();
-
-        failServer(2);
-
-        await(srvFailedLatch);
-        await(clientFailedLatch);
-
-        checkNodes(2, 3);
-    }
-
-    /**
-     * TODO: IGNITE-587.
-     * @throws Exception If failed.
-     */
-    public void testClientReconnect() throws Exception {
-        fail("ignite-587");
-
-        clientsPerSrv = 1;
-
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        resetClientIpFinder(2);
-
-        srvFailedLatch = new CountDownLatch(2);
-        clientFailedLatch = new CountDownLatch(3);
-
-        attachListeners(2, 3);
-
-        failServer(2);
-
-        await(srvFailedLatch);
-        await(clientFailedLatch);
-
-        checkNodes(2, 3);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeJoinOneServer() throws Exception {
-        startServerNodes(1);
-
-        srvJoinedLatch = new CountDownLatch(1);
-
-        attachListeners(1, 0);
-
-        startClientNodes(1);
-
-        await(srvJoinedLatch);
-
-        checkNodes(1, 1);
-    }
-
-    /**
-     * TODO: IGNITE-587.
-     * @throws Exception If failed.
-     */
-    public void testClientNodeLeaveOneServer() throws Exception {
-        fail("ignite-587");
-
-        startServerNodes(1);
-        startClientNodes(1);
-
-        checkNodes(1, 1);
-
-        srvLeftLatch = new CountDownLatch(1);
-
-        attachListeners(1, 0);
-
-        stopGrid("client-0");
-
-        await(srvLeftLatch);
-
-        checkNodes(1, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testClientNodeFailOneServer() throws Exception {
-        startServerNodes(1);
-        startClientNodes(1);
-
-        checkNodes(1, 1);
-
-        srvFailedLatch = new CountDownLatch(1);
-
-        attachListeners(1, 0);
-
-        failClient(0);
-
-        await(srvFailedLatch);
-
-        checkNodes(1, 0);
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMetrics() throws Exception {
-        startServerNodes(3);
-        startClientNodes(3);
-
-        checkNodes(3, 3);
-
-        attachListeners(3, 3);
-
-        assertTrue(checkMetrics(3, 3, 0));
-
-        G.ignite("client-0").compute().broadcast(F.noop());
-
-        assertTrue(GridTestUtils.waitForCondition(new PA() {
-            @Override public boolean apply() {
-                return checkMetrics(3, 3, 1);
-            }
-        }, 10000));
-
-        checkMetrics(3, 3, 1);
-
-        G.ignite("server-0").compute().broadcast(F.noop());
-
-        assertTrue(GridTestUtils.waitForCondition(new PA() {
-            @Override public boolean apply() {
-                return checkMetrics(3, 3, 2);
-            }
-        }, 10000));
-    }
-
-    /**
-     * @param srvCnt Number of Number of server nodes.
-     * @param clientCnt Number of client nodes.
-     * @param execJobsCnt Expected number of executed jobs.
-     * @return Whether metrics are correct.
-     */
-    private boolean checkMetrics(int srvCnt, int clientCnt, int execJobsCnt) {
-        for (int i = 0; i < srvCnt; i++) {
-            Ignite g = G.ignite("server-" + i);
-
-            for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
-                    return false;
-            }
-        }
-
-        for (int i = 0; i < clientCnt; i++) {
-            Ignite g = G.ignite("client-" + i);
-
-            for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
-                    return false;
-            }
-        }
-
-        return true;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDataExchangeFromServer() throws Exception {
-        testDataExchange("server-0");
-    }
-
-    /**
-     * TODO: IGNITE-587.
-     *
-     * @throws Exception If failed.
-     */
-    public void testDataExchangeFromClient() throws Exception {
-        fail("ignite-587");
-
-        testDataExchange("client-0");
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void testDataExchange(String masterName) throws Exception {
-        startServerNodes(2);
-        startClientNodes(2);
-
-        checkNodes(2, 2);
-
-        IgniteMessaging msg = grid(masterName).message();
-
-        UUID id = null;
-
-        try {
-            id = msg.remoteListen(null, new MessageListener());
-
-            msgLatch = new CountDownLatch(4);
-
-            msg.send(null, "Message 1");
-
-            await(msgLatch);
-
-            startServerNodes(1);
-            startClientNodes(1);
-
-            checkNodes(3, 3);
-
-            msgLatch = new CountDownLatch(6);
-
-            msg.send(null, "Message 2");
-
-            await(msgLatch);
-        }
-        finally {
-            if (id != null)
-                msg.stopRemoteListen(id);
-        }
-    }
-
-    /**
-     * @param idx Index.
-     * @throws Exception In case of error.
-     */
-    private void resetClientIpFinder(int idx) throws Exception {
-        TcpClientDiscoverySpi disco =
-            (TcpClientDiscoverySpi)G.ignite("client-" + idx).configuration().getDiscoverySpi();
-
-        TcpDiscoveryVmIpFinder ipFinder = (TcpDiscoveryVmIpFinder)disco.getIpFinder();
-
-        String addr = IP_FINDER.getRegisteredAddresses().iterator().next().toString();
-
-        if (addr.startsWith("/"))
-            addr = addr.substring(1);
-
-        ipFinder.setAddresses(Arrays.asList(addr));
-    }
-
-    /**
-     * @param cnt Number of nodes.
-     * @throws Exception In case of error.
-     */
-    private void startServerNodes(int cnt) throws Exception {
-        for (int i = 0; i < cnt; i++) {
-            Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
-
-            srvNodeIds.add(g.cluster().localNode().id());
-        }
-    }
-
-    /**
-     * @param cnt Number of nodes.
-     * @throws Exception In case of error.
-     */
-    private void startClientNodes(int cnt) throws Exception {
-        for (int i = 0; i < cnt; i++) {
-            Ignite g = startGrid("client-" + clientIdx.getAndIncrement());
-
-            clientNodeIds.add(g.cluster().localNode().id());
-        }
-    }
-
-    /**
-     * @param idx Index.
-     */
-    private void failServer(int idx) {
-        ((TcpDiscoverySpi)G.ignite("server-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
-    }
-
-    /**
-     * @param idx Index.
-     */
-    private void failClient(int idx) {
-        ((TcpClientDiscoverySpi)G.ignite("client-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
-    }
-
-    /**
-     * @param srvCnt Number of server nodes.
-     * @param clientCnt Number of client nodes.
-     */
-    private void attachListeners(int srvCnt, int clientCnt) throws Exception {
-        if (srvJoinedLatch != null) {
-            for (int i = 0; i < srvCnt; i++) {
-                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Joined event fired on server: " + evt);
-
-                        srvJoinedLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_JOINED);
-            }
-        }
-
-        if (srvLeftLatch != null) {
-            for (int i = 0; i < srvCnt; i++) {
-                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Left event fired on server: " + evt);
-
-                        srvLeftLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_LEFT);
-            }
-        }
-
-        if (srvFailedLatch != null) {
-            for (int i = 0; i < srvCnt; i++) {
-                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Failed event fired on server: " + evt);
-
-                        srvFailedLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_FAILED);
-            }
-        }
-
-        if (clientJoinedLatch != null) {
-            for (int i = 0; i < clientCnt; i++) {
-                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Joined event fired on client: " + evt);
-
-                        clientJoinedLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_JOINED);
-            }
-        }
-
-        if (clientLeftLatch != null) {
-            for (int i = 0; i < clientCnt; i++) {
-                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Left event fired on client: " + evt);
-
-                        clientLeftLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_LEFT);
-            }
-        }
-
-        if (clientFailedLatch != null) {
-            for (int i = 0; i < clientCnt; i++) {
-                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
-                    @Override public boolean apply(Event evt) {
-                        info("Failed event fired on client: " + evt);
-
-                        clientFailedLatch.countDown();
-
-                        return true;
-                    }
-                }, EVT_NODE_FAILED);
-            }
-        }
-    }
-
-    /**
-     * @param srvCnt Number of server nodes.
-     * @param clientCnt Number of client nodes.
-     */
-    private void checkNodes(int srvCnt, int clientCnt) {
-        for (int i = 0; i < srvCnt; i++) {
-            Ignite g = G.ignite("server-" + i);
-
-            assertTrue(srvNodeIds.contains(g.cluster().localNode().id()));
-
-            assertFalse(g.cluster().localNode().isClient());
-
-            checkRemoteNodes(g, srvCnt + clientCnt - 1);
-        }
-
-        for (int i = 0; i < clientCnt; i++) {
-            Ignite g = G.ignite("client-" + i);
-
-            assertTrue(clientNodeIds.contains(g.cluster().localNode().id()));
-
-            assertTrue(g.cluster().localNode().isClient());
-
-            checkRemoteNodes(g, srvCnt + clientCnt - 1);
-        }
-    }
-
-    /**
-     * @param ignite Grid.
-     * @param expCnt Expected nodes count.
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    private void checkRemoteNodes(Ignite ignite, int expCnt) {
-        Collection<ClusterNode> nodes = ignite.cluster().forRemotes().nodes();
-
-        assertEquals(expCnt, nodes.size());
-
-        for (ClusterNode node : nodes) {
-            UUID id = node.id();
-
-            if (clientNodeIds.contains(id))
-                assertTrue(node.isClient());
-            else if (srvNodeIds.contains(id))
-                assertFalse(node.isClient());
-            else
-                assert false : "Unexpected node ID: " + id;
-        }
-    }
-
-    /**
-     * @param latch Latch.
-     * @throws InterruptedException If interrupted.
-     */
-    private void await(CountDownLatch latch) throws InterruptedException {
-        assertTrue("Latch count: " + latch.getCount(), latch.await(10000, MILLISECONDS));
-    }
-
-    /**
-     */
-    private static class MessageListener implements IgniteBiPredicate<UUID, Object> {
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /** {@inheritDoc} */
-        @Override public boolean apply(UUID uuid, Object msg) {
-            X.println(">>> Received [locNodeId=" + ignite.configuration().getNodeId() + ", msg=" + msg + ']');
-
-            msgLatch.countDown();
-
-            return true;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
new file mode 100644
index 0000000..d1b6232
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMulticastTest.java
@@ -0,0 +1,129 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class TcpClientDiscoverySpiMulticastTest extends GridCommonAbstractTest {
+    /** */
+    private boolean forceSrv;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setLocalHost("127.0.0.1");
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(new TcpDiscoveryMulticastIpFinder());
+
+        if (getTestGridName(1).equals(gridName)) {
+            cfg.setClientMode(true);
+
+            spi.setForceServerMode(forceSrv);
+        }
+
+        cfg.setDiscoverySpi(spi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWithMulticast() throws Exception {
+        joinWithMulticast();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWithMulticastForceServer() throws Exception {
+        forceSrv = true;
+
+        joinWithMulticast();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void joinWithMulticast() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        assertSpi(ignite0, false);
+
+        Ignite ignite1 = startGrid(1);
+
+        assertSpi(ignite1, !forceSrv);
+
+        assertTrue(ignite1.configuration().isClientMode());
+
+        assertEquals(2, ignite0.cluster().nodes().size());
+        assertEquals(2, ignite1.cluster().nodes().size());
+
+        Ignite ignite2 = startGrid(2);
+
+        assertSpi(ignite2, false);
+
+        assertEquals(3, ignite0.cluster().nodes().size());
+        assertEquals(3, ignite1.cluster().nodes().size());
+        assertEquals(3, ignite2.cluster().nodes().size());
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @param client Expected client mode flag.
+     */
+    private void assertSpi(Ignite ignite, boolean client) {
+        DiscoverySpi spi = ignite.configuration().getDiscoverySpi();
+
+        assertSame(TcpDiscoverySpi.class, spi.getClass());
+
+        TcpDiscoverySpi spi0 = (TcpDiscoverySpi)spi;
+
+        assertSame(TcpDiscoveryMulticastIpFinder.class, spi0.getIpFinder().getClass());
+
+        assertEquals(client, spi0.isClientMode());
+
+        Collection<Object> addrSnds = GridTestUtils.getFieldValue(spi0.getIpFinder(), "addrSnds");
+
+        assertNotNull(addrSnds);
+
+        if (client)
+            assertTrue(addrSnds.isEmpty()); // Check client does not send its address.
+        else
+            assertFalse(addrSnds.isEmpty());
+    }
+}


[17/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 5c80e6e..6f5a738 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -171,8 +171,6 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * This frequency defines how often system will advice to flush
      * connection buffer.
      * <p>
-     * If not provided, default value is {@link TcpCommunicationSpi#DFLT_CONN_BUF_FLUSH_FREQ}.
-     * <p>
      * This property is used only if {@link #getConnectionBufferSize()} is greater than {@code 0}.
      *
      * @param connBufFlushFreq Flush frequency.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index 7560999..b952087 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -17,12 +17,13 @@
 
 package org.apache.ignite.spi.discovery;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.jetbrains.annotations.*;
 
-import java.io.*;
 import java.util.*;
 
 /**
@@ -102,7 +103,7 @@ public interface DiscoverySpi extends IgniteSpi {
      *
      * @param exchange Discovery data exchange handler.
      */
-    public void setDataExchange(DiscoverySpiDataExchange exchange);
+    public TcpDiscoverySpi setDataExchange(DiscoverySpiDataExchange exchange);
 
     /**
      * Sets discovery metrics provider. Use metrics provided by
@@ -111,7 +112,7 @@ public interface DiscoverySpi extends IgniteSpi {
      *
      * @param metricsProvider Provider of metrics data.
      */
-    public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
+    public TcpDiscoverySpi setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
 
     /**
      * Tells discovery SPI to disconnect from topology. This is very close to calling
@@ -141,9 +142,10 @@ public interface DiscoverySpi extends IgniteSpi {
 
     /**
      * Sends custom message across the ring.
-     * @param evt Event.
+     * @param msg Custom message.
+     * @throws IgniteException if failed to marshal evt.
      */
-    public void sendCustomEvent(Serializable evt);
+    public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;
 
     /**
      * Initiates failure of provided node.
@@ -151,4 +153,12 @@ public interface DiscoverySpi extends IgniteSpi {
      * @param nodeId Node ID.
      */
     public void failNode(UUID nodeId);
+
+    /**
+     * Whether or not discovery is started in client mode.
+     *
+     * @return {@code true} if node is in client mode.
+     * @throws IllegalStateException If discovery SPI has not started.
+     */
+    public boolean isClientMode() throws IllegalStateException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
new file mode 100644
index 0000000..15e943b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiCustomMessage.java
@@ -0,0 +1,40 @@
+/*
+ * 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.spi.discovery;
+
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Message to send across ring.
+ *
+ * @see org.apache.ignite.internal.managers.discovery.GridDiscoveryManager#sendCustomEvent(
+ * org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage)
+ */
+public interface DiscoverySpiCustomMessage extends Serializable {
+    /**
+     * Called when message passed the ring.
+     */
+    @Nullable public DiscoverySpiCustomMessage ackMessage();
+
+    /**
+     * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+     */
+    public boolean isMutable();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiListener.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiListener.java
index 7f17fe4..f46869d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiListener.java
@@ -18,10 +18,9 @@
 package org.apache.ignite.spi.discovery;
 
 import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.events.*;
 import org.jetbrains.annotations.*;
 
-import java.io.*;
 import java.util.*;
 
 /**
@@ -47,5 +46,5 @@ public interface DiscoverySpiListener {
         ClusterNode node,
         Collection<ClusterNode> topSnapshot,
         @Nullable Map<Long, Collection<ClusterNode>> topHist,
-        @Nullable Serializable data);
+        @Nullable DiscoverySpiCustomMessage data);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
new file mode 100644
index 0000000..d064c8d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -0,0 +1,1478 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
+
+/**
+ *
+ */
+class ClientImpl extends TcpDiscoveryImpl {
+    /** */
+    private static final Object JOIN_TIMEOUT = "JOIN_TIMEOUT";
+
+    /** */
+    private static final Object SPI_STOP = "SPI_STOP";
+
+    /** */
+    private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED";
+
+    /** Remote nodes. */
+    private final ConcurrentMap<UUID, TcpDiscoveryNode> rmtNodes = new ConcurrentHashMap8<>();
+
+    /** Topology history. */
+    private final NavigableMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
+
+    /** Remote nodes. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Boolean>> pingFuts = new ConcurrentHashMap8<>();
+
+    /** Socket writer. */
+    private SocketWriter sockWriter;
+
+    /** */
+    private SocketReader sockReader;
+
+    /** */
+    private boolean segmented;
+
+    /** Last message ID. */
+    private volatile IgniteUuid lastMsgId;
+
+    /** Current topology version. */
+    private volatile long topVer;
+
+    /** Join error. Contains error what occurs on join process. */
+    private IgniteSpiException joinErr;
+
+    /** Joined latch. */
+    private final CountDownLatch joinLatch = new CountDownLatch(1);
+
+    /** Left latch. */
+    private final CountDownLatch leaveLatch = new CountDownLatch(1);
+
+    /** */
+    private final Timer timer = new Timer("TcpDiscoverySpi.timer");
+
+    /** */
+    protected MessageWorker msgWorker;
+
+    /**
+     * @param adapter Adapter.
+     */
+    ClientImpl(TcpDiscoverySpi adapter) {
+        super(adapter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dumpDebugInfo(IgniteLogger log) {
+        StringBuilder b = new StringBuilder(U.nl());
+
+        b.append(">>>").append(U.nl());
+        b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl());
+        b.append(">>>").append(U.nl());
+
+        b.append("Local node ID: ").append(getLocalNodeId()).append(U.nl()).append(U.nl());
+        b.append("Local node: ").append(locNode).append(U.nl()).append(U.nl());
+
+        b.append("Internal threads: ").append(U.nl());
+
+        b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
+        b.append("    Socket reader: ").append(threadStatus(sockReader)).append(U.nl());
+        b.append("    Socket writer: ").append(threadStatus(sockWriter)).append(U.nl());
+
+        b.append(U.nl());
+
+        b.append("Nodes: ").append(U.nl());
+
+        for (ClusterNode node : allVisibleNodes())
+            b.append("    ").append(node.id()).append(U.nl());
+
+        b.append(U.nl());
+
+        b.append("Stats: ").append(spi.stats).append(U.nl());
+
+        U.quietAndInfo(log, b.toString());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+
+        if (sockWriter.isOnline())
+            return "connected";
+
+        return "disconnected";
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return msgWorker.queueSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID getCoordinator() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        spi.initLocalNode(0, true);
+
+        locNode = spi.locNode;
+
+        sockWriter = new SocketWriter();
+        sockWriter.start();
+
+        sockReader = new SocketReader();
+        sockReader.start();
+
+        msgWorker = new MessageWorker();
+        msgWorker.start();
+
+        if (spi.ipFinder.isShared())
+            registerLocalNodeAddress();
+
+        try {
+            joinLatch.await();
+
+            if (joinErr != null)
+                throw joinErr;
+        }
+        catch (InterruptedException e) {
+            throw new IgniteSpiException("Thread has been interrupted.", e);
+        }
+
+        timer.schedule(new HeartbeatSender(), spi.hbFreq, spi.hbFreq);
+
+        spi.printStartInfo();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        timer.cancel();
+
+        if (msgWorker != null && msgWorker.isAlive()) { // Should always be alive
+            msgWorker.addMessage(SPI_STOP);
+
+            try {
+                if (!leaveLatch.await(spi.netTimeout, MILLISECONDS))
+                    U.warn(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
+            }
+            catch (InterruptedException ignored) {
+
+            }
+        }
+
+        for (GridFutureAdapter<Boolean> fut : pingFuts.values())
+            fut.onDone(false);
+
+        rmtNodes.clear();
+
+        U.interrupt(msgWorker);
+        U.interrupt(sockWriter);
+        U.interrupt(sockReader);
+
+        U.join(msgWorker, log);
+        U.join(sockWriter, log);
+        U.join(sockReader, log);
+
+        spi.printStopInfo();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return U.arrayList(rmtNodes.values(), TcpDiscoveryNodesRing.VISIBLE_NODES);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        if (getLocalNodeId().equals(nodeId))
+            return locNode;
+
+        TcpDiscoveryNode node = rmtNodes.get(nodeId);
+
+        return node != null && node.visible() ? node : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(@NotNull final UUID nodeId) {
+        if (nodeId.equals(getLocalNodeId()))
+            return true;
+
+        TcpDiscoveryNode node = rmtNodes.get(nodeId);
+
+        if (node == null || !node.visible())
+            return false;
+
+        GridFutureAdapter<Boolean> fut = pingFuts.get(nodeId);
+
+        if (fut == null) {
+            fut = new GridFutureAdapter<>();
+
+            GridFutureAdapter<Boolean> oldFut = pingFuts.putIfAbsent(nodeId, fut);
+
+            if (oldFut != null)
+                fut = oldFut;
+            else {
+                if (spi.getSpiContext().isStopping()) {
+                    if (pingFuts.remove(nodeId, fut))
+                        fut.onDone(false);
+
+                    return false;
+                }
+
+                final GridFutureAdapter<Boolean> finalFut = fut;
+
+                timer.schedule(new TimerTask() {
+                    @Override public void run() {
+                        if (pingFuts.remove(nodeId, finalFut))
+                            finalFut.onDone(false);
+                    }
+                }, spi.netTimeout);
+
+                sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
+            }
+        }
+
+        try {
+            return fut.get();
+        }
+        catch (IgniteInterruptedCheckedException ignored) {
+            return false;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException(e); // Should newer occur
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        U.interrupt(msgWorker);
+        U.interrupt(sockWriter);
+        U.interrupt(sockReader);
+
+        U.join(msgWorker, log);
+        U.join(sockWriter, log);
+        U.join(sockReader, log);
+
+        leaveLatch.countDown();
+        joinLatch.countDown();
+
+        spi.getSpiContext().deregisterPorts();
+
+        Collection<ClusterNode> rmts = getRemoteNodes();
+
+        // This is restart/disconnection and remote nodes are not empty.
+        // We need to fire FAIL event for each.
+        DiscoverySpiListener lsnr = spi.lsnr;
+
+        if (lsnr != null) {
+            for (ClusterNode n : rmts) {
+                rmtNodes.remove(n.id());
+
+                Collection<ClusterNode> top = updateTopologyHistory(topVer + 1);
+
+                lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, new TreeMap<>(topHist), null);
+            }
+        }
+
+        rmtNodes.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
+        if (segmented)
+            throw new IgniteException("Failed to send custom message: client is disconnected");
+
+        try {
+            sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt,
+                spi.marsh.marshal(evt)));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        ClusterNode node = rmtNodes.get(nodeId);
+
+        if (node != null) {
+            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
+                node.id(), node.order());
+
+            msgWorker.addMessage(msg);
+        }
+    }
+
+    /**
+     * @return Opened socket or {@code null} if timeout.
+     * @see TcpDiscoverySpi#joinTimeout
+     */
+    @SuppressWarnings("BusyWait")
+    @Nullable private Socket joinTopology(boolean recon) throws IgniteSpiException, InterruptedException {
+        Collection<InetSocketAddress> addrs = null;
+
+        long startTime = U.currentTimeMillis();
+
+        while (true) {
+            if (Thread.currentThread().isInterrupted())
+                throw new InterruptedException();
+
+            while (addrs == null || addrs.isEmpty()) {
+                addrs = spi.resolvedAddresses();
+
+                if (!F.isEmpty(addrs)) {
+                    if (log.isDebugEnabled())
+                        log.debug("Resolved addresses from IP finder: " + addrs);
+                }
+                else {
+                    U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + spi.ipFinder);
+
+                    if (spi.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > spi.joinTimeout)
+                        return null;
+
+                    Thread.sleep(2000);
+                }
+            }
+
+            Collection<InetSocketAddress> addrs0 = new ArrayList<>(addrs);
+
+            Iterator<InetSocketAddress> it = addrs.iterator();
+
+            while (it.hasNext()) {
+                if (Thread.currentThread().isInterrupted())
+                    throw new InterruptedException();
+
+                InetSocketAddress addr = it.next();
+
+                Socket sock = null;
+
+                try {
+                    long ts = U.currentTimeMillis();
+
+                    IgniteBiTuple<Socket, UUID> t = initConnection(addr);
+
+                    sock = t.get1();
+
+                    UUID rmtNodeId = t.get2();
+
+                    spi.stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
+
+                    locNode.clientRouterNodeId(rmtNodeId);
+
+                    TcpDiscoveryAbstractMessage msg = recon ?
+                        new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId,
+                            lastMsgId) :
+                        new TcpDiscoveryJoinRequestMessage(locNode, spi.collectExchangeData(getLocalNodeId()));
+
+                    msg.client(true);
+
+                    spi.writeToSocket(sock, msg);
+
+                    int res = spi.readReceipt(sock, spi.ackTimeout);
+
+                    switch (res) {
+                        case RES_OK:
+                            return sock;
+
+                        case RES_CONTINUE_JOIN:
+                        case RES_WAIT:
+                            U.closeQuiet(sock);
+
+                            break;
+
+                        default:
+                            if (log.isDebugEnabled())
+                                log.debug("Received unexpected response to join request: " + res);
+
+                            U.closeQuiet(sock);
+                    }
+                }
+                catch (IOException | IgniteCheckedException e) {
+                    if (log.isDebugEnabled())
+                        U.error(log, "Failed to establish connection with address: " + addr, e);
+
+                    U.closeQuiet(sock);
+
+                    it.remove();
+                }
+            }
+
+            if (addrs.isEmpty()) {
+                U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
+                    "in 2000ms): " + addrs0);
+
+                if (spi.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > spi.joinTimeout)
+                    return null;
+
+                Thread.sleep(2000);
+            }
+        }
+    }
+
+    /**
+     * @param topVer New topology version.
+     * @return Latest topology snapshot.
+     */
+    private NavigableSet<ClusterNode> updateTopologyHistory(long topVer) {
+        this.topVer = topVer;
+
+        NavigableSet<ClusterNode> allNodes = allVisibleNodes();
+
+        if (!topHist.containsKey(topVer)) {
+            assert topHist.isEmpty() || topHist.lastKey() == topVer - 1 :
+                "lastVer=" + topHist.lastKey() + ", newVer=" + topVer;
+
+            topHist.put(topVer, allNodes);
+
+            if (topHist.size() > spi.topHistSize)
+                topHist.pollFirstEntry();
+
+            assert topHist.lastKey() == topVer;
+            assert topHist.size() <= spi.topHistSize;
+        }
+
+        return allNodes;
+    }
+
+    /**
+     * @return All nodes.
+     */
+    private NavigableSet<ClusterNode> allVisibleNodes() {
+        NavigableSet<ClusterNode> allNodes = new TreeSet<>();
+
+        for (TcpDiscoveryNode node : rmtNodes.values()) {
+            if (node.visible())
+                allNodes.add(node);
+        }
+
+        allNodes.add(locNode);
+
+        return allNodes;
+    }
+
+    /**
+     * @param addr Address.
+     * @return Remote node ID.
+     * @throws IOException In case of I/O error.
+     * @throws IgniteCheckedException In case of other error.
+     */
+    private IgniteBiTuple<Socket, UUID> initConnection(InetSocketAddress addr) throws IOException, IgniteCheckedException {
+        assert addr != null;
+
+        Socket sock = spi.openSocket(addr);
+
+        TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId());
+
+        req.client(true);
+
+        spi.writeToSocket(sock, req);
+
+        TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, spi.ackTimeout);
+
+        UUID nodeId = res.creatorNodeId();
+
+        assert nodeId != null;
+        assert !getLocalNodeId().equals(nodeId);
+
+        return F.t(sock, nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override void simulateNodeFailure() {
+        U.warn(log, "Simulating client node failure: " + getLocalNodeId());
+
+        U.interrupt(sockWriter);
+        U.interrupt(msgWorker);
+
+        U.join(sockWriter, log);
+        U.join(msgWorker, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void brakeConnection() {
+        U.closeQuiet(msgWorker.currSock);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteSpiThread workerThread() {
+        return msgWorker;
+    }
+
+    /**
+     * FOR TEST PURPOSE ONLY!
+     */
+    @SuppressWarnings("BusyWait")
+    public void waitForClientMessagePrecessed() {
+        Object last = msgWorker.queue.peekLast();
+
+        while (last != null && msgWorker.isAlive() && msgWorker.queue.contains(last)) {
+            try {
+                Thread.sleep(10);
+            }
+            catch (InterruptedException ignored) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    /**
+     * Heartbeat sender.
+     */
+    private class HeartbeatSender extends TimerTask {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            if (!spi.getSpiContext().isStopping() && sockWriter.isOnline()) {
+                TcpDiscoveryClientHeartbeatMessage msg = new TcpDiscoveryClientHeartbeatMessage(getLocalNodeId(),
+                    spi.metricsProvider.metrics());
+
+                msg.client(true);
+
+                sockWriter.sendMessage(msg);
+            }
+        }
+    }
+
+    /**
+     * Socket reader.
+     */
+    private class SocketReader extends IgniteSpiThread {
+        /** */
+        private final Object mux = new Object();
+
+        /** */
+        private Socket sock;
+
+        /** */
+        private UUID rmtNodeId;
+
+        /**
+         */
+        protected SocketReader() {
+            super(spi.ignite().name(), "tcp-client-disco-sock-reader", log);
+        }
+
+        /**
+         * @param sock Socket.
+         * @param rmtNodeId Rmt node id.
+         */
+        public void setSocket(Socket sock, UUID rmtNodeId) {
+            synchronized (mux) {
+                this.sock = sock;
+
+                this.rmtNodeId = rmtNodeId;
+
+                mux.notifyAll();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            while (!isInterrupted()) {
+                Socket sock;
+                UUID rmtNodeId;
+
+                synchronized (mux) {
+                    if (this.sock == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+
+                    sock = this.sock;
+                    rmtNodeId = this.rmtNodeId;
+                }
+
+                try {
+                    InputStream in = new BufferedInputStream(sock.getInputStream());
+
+                    sock.setKeepAlive(true);
+                    sock.setTcpNoDelay(true);
+
+                    while (!isInterrupted()) {
+                        TcpDiscoveryAbstractMessage msg;
+
+                        try {
+                            msg = spi.marsh.unmarshal(in, U.gridClassLoader());
+                        }
+                        catch (IgniteCheckedException e) {
+                            if (log.isDebugEnabled())
+                                U.error(log, "Failed to read message [sock=" + sock + ", " +
+                                    "locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']', e);
+
+                            IOException ioEx = X.cause(e, IOException.class);
+
+                            if (ioEx != null)
+                                throw ioEx;
+
+                            ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
+
+                            if (clsNotFoundEx != null)
+                                LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+                                    "(make sure same versions of all classes are available on all nodes) " +
+                                    "[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
+                            else
+                                LT.error(log, e, "Failed to read message [sock=" + sock + ", locNodeId=" +
+                                    getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']');
+
+                            continue;
+                        }
+
+                        msg.senderNodeId(rmtNodeId);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Message has been received: " + msg);
+
+                        spi.stats.onMessageReceived(msg);
+
+                        if (spi.ensured(msg))
+                            lastMsgId = msg.id();
+
+                        msgWorker.addMessage(msg);
+                    }
+                }
+                catch (IOException e) {
+                    msgWorker.addMessage(new SocketClosedMessage(sock));
+
+                    if (log.isDebugEnabled())
+                        U.error(log, "Connection failed [sock=" + sock + ", locNodeId=" + getLocalNodeId() + ']', e);
+                }
+                finally {
+                    U.closeQuiet(sock);
+
+                    synchronized (mux) {
+                        if (this.sock == sock) {
+                            this.sock = null;
+                            this.rmtNodeId = null;
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private class SocketWriter extends IgniteSpiThread {
+        /** */
+        private final Object mux = new Object();
+
+        /** */
+        private Socket sock;
+
+        /** */
+        private final Queue<TcpDiscoveryAbstractMessage> queue = new ArrayDeque<>();
+
+        /**
+         *
+         */
+        protected SocketWriter() {
+            super(spi.ignite().name(), "tcp-client-disco-sock-writer", log);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void sendMessage(TcpDiscoveryAbstractMessage msg) {
+            synchronized (mux) {
+                queue.add(msg);
+
+                mux.notifyAll();
+            }
+        }
+
+        /**
+         * @param sock Socket.
+         */
+        private void setSocket(Socket sock) {
+            synchronized (mux) {
+                this.sock = sock;
+
+                mux.notifyAll();
+            }
+        }
+
+        /**
+         *
+         */
+        public boolean isOnline() {
+            synchronized (mux) {
+                return sock != null;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            TcpDiscoveryAbstractMessage msg = null;
+
+            while (!Thread.currentThread().isInterrupted()) {
+                Socket sock;
+
+                synchronized (mux) {
+                    sock = this.sock;
+
+                    if (sock == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+
+                    if (msg == null)
+                        msg = queue.poll();
+
+                    if (msg == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+                }
+
+                for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : spi.sendMsgLsnrs)
+                    msgLsnr.apply(msg);
+
+                try {
+                    spi.writeToSocket(sock, msg);
+
+                    msg = null;
+                }
+                catch (IOException e) {
+                    if (log.isDebugEnabled())
+                        U.error(log, "Failed to send node left message (will stop anyway) [sock=" + sock + ']', e);
+
+                    U.closeQuiet(sock);
+
+                    synchronized (mux) {
+                        if (sock == this.sock)
+                            this.sock = null; // Connection has dead.
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + msg, e);
+
+                    msg = null;
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private class Reconnector extends IgniteSpiThread {
+        /** */
+        private volatile Socket sock;
+
+        /**
+         *
+         */
+        protected Reconnector() {
+            super(spi.ignite().name(), "tcp-client-disco-msg-worker", log);
+        }
+
+        /**
+         *
+         */
+        public void cancel() {
+            interrupt();
+
+            U.closeQuiet(sock);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            assert !segmented;
+
+            boolean success = false;
+
+            try {
+                sock = joinTopology(true);
+
+                if (sock == null) {
+                    U.error(log, "Failed to reconnect to cluster: timeout.");
+
+                    return;
+                }
+
+                if (isInterrupted())
+                    throw new InterruptedException();
+
+                InputStream in = new BufferedInputStream(sock.getInputStream());
+
+                sock.setKeepAlive(true);
+                sock.setTcpNoDelay(true);
+
+                // Wait for
+                while (!isInterrupted()) {
+                    TcpDiscoveryAbstractMessage msg = spi.marsh.unmarshal(in, U.gridClassLoader());
+
+                    if (msg instanceof TcpDiscoveryClientReconnectMessage) {
+                        TcpDiscoveryClientReconnectMessage res = (TcpDiscoveryClientReconnectMessage)msg;
+
+                        if (res.creatorNodeId().equals(getLocalNodeId())) {
+                            if (res.success()) {
+                                msgWorker.addMessage(res);
+
+                                success = true;
+                            }
+
+                            break;
+                        }
+                    }
+
+                }
+            }
+            catch (IOException | IgniteCheckedException e) {
+                U.error(log, "Failed to reconnect", e);
+            }
+            finally {
+                if (!success) {
+                    U.closeQuiet(sock);
+
+                    msgWorker.addMessage(SPI_RECONNECT_FAILED);
+                }
+            }
+        }
+    }
+
+    /**
+     * Message worker.
+     */
+    protected class MessageWorker extends IgniteSpiThread {
+        /** Message queue. */
+        private final BlockingDeque<Object> queue = new LinkedBlockingDeque<>();
+
+        /** */
+        private Socket currSock;
+
+        /** Indicates that pending messages are currently processed. */
+        private boolean pending;
+
+        /** */
+        private Reconnector reconnector;
+
+        /**
+         *
+         */
+        private MessageWorker() {
+            super(spi.ignite().name(), "tcp-client-disco-msg-worker", log);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("InfiniteLoopStatement")
+        @Override protected void body() throws InterruptedException {
+            spi.stats.onJoinStarted();
+
+            try {
+                final Socket sock = joinTopology(false);
+
+                if (sock == null) {
+                    joinErr = new IgniteSpiException("Join process timed out");
+
+                    joinLatch.countDown();
+
+                    return;
+                }
+
+                currSock = sock;
+
+                sockWriter.setSocket(sock);
+
+                timer.schedule(new TimerTask() {
+                    @Override public void run() {
+                        if (joinLatch.getCount() > 0)
+                            queue.add(JOIN_TIMEOUT);
+                    }
+                }, spi.netTimeout);
+
+                sockReader.setSocket(sock, locNode.clientRouterNodeId());
+
+                while (true) {
+                    Object msg = queue.take();
+
+                    if (msg == JOIN_TIMEOUT) {
+                        if (joinLatch.getCount() > 0) {
+                            joinErr = new IgniteSpiException("Join process timed out [sock=" + sock +
+                                ", timeout=" + spi.netTimeout + ']');
+
+                            joinLatch.countDown();
+
+                            break;
+                        }
+                    }
+                    else if (msg == SPI_STOP) {
+                        assert spi.getSpiContext().isStopping();
+
+                        if (currSock != null) {
+                            TcpDiscoveryAbstractMessage leftMsg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId());
+
+                            leftMsg.client(true);
+
+                            sockWriter.sendMessage(leftMsg);
+                        }
+                        else
+                            leaveLatch.countDown();
+                    }
+                    else if (msg instanceof SocketClosedMessage) {
+                        if (((SocketClosedMessage)msg).sock == currSock) {
+                            currSock = null;
+
+                            if (joinLatch.getCount() > 0) {
+                                joinErr = new IgniteSpiException("Failed to connect to cluster: socket closed.");
+
+                                joinLatch.countDown();
+
+                                break;
+                            }
+                            else {
+                                if (spi.getSpiContext().isStopping() || segmented)
+                                    leaveLatch.countDown();
+                                else {
+                                    assert reconnector == null;
+
+                                    final Reconnector reconnector = new Reconnector();
+                                    this.reconnector = reconnector;
+                                    reconnector.start();
+
+                                    timer.schedule(new TimerTask() {
+                                        @Override public void run() {
+                                            if (reconnector.isAlive())
+                                                reconnector.cancel();
+                                        }
+                                    }, spi.netTimeout);
+                                }
+                            }
+                        }
+                    }
+                    else if (msg == SPI_RECONNECT_FAILED) {
+                        if (!segmented) {
+                            segmented = true;
+
+                            reconnector.cancel();
+                            reconnector.join();
+
+                            notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                        }
+                    }
+                    else {
+                        TcpDiscoveryAbstractMessage discoMsg = (TcpDiscoveryAbstractMessage)msg;
+
+                        if (joinLatch.getCount() > 0) {
+                            IgniteSpiException err = null;
+
+                            if (discoMsg instanceof TcpDiscoveryDuplicateIdMessage)
+                                err = spi.duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
+                            else if (discoMsg instanceof TcpDiscoveryAuthFailedMessage)
+                                err = spi.authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
+                            else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage)
+                                err = spi.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
+
+                            if (err != null) {
+                                joinErr = err;
+
+                                joinLatch.countDown();
+
+                                break;
+                            }
+                        }
+
+                        processDiscoveryMessage((TcpDiscoveryAbstractMessage)msg);
+                    }
+                }
+            }
+            finally {
+                U.closeQuiet(currSock);
+
+                if (joinLatch.getCount() > 0) {
+                    // This should not occurs.
+                    joinErr = new IgniteSpiException("Some error occurs in joinig process");
+
+                    joinLatch.countDown();
+                }
+
+                if (reconnector != null) {
+                    reconnector.cancel();
+
+                    reconnector.join();
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        protected void processDiscoveryMessage(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null;
+            assert msg.verified() || msg.senderNodeId() == null;
+
+            spi.stats.onMessageProcessingStarted(msg);
+
+            if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
+                processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
+                processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
+                processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
+            else if (msg instanceof TcpDiscoveryHeartbeatMessage)
+                processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
+            else if (msg instanceof TcpDiscoveryClientReconnectMessage)
+                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
+            else if (msg instanceof TcpDiscoveryCustomEventMessage)
+                processCustomMessage((TcpDiscoveryCustomEventMessage)msg);
+            else if (msg instanceof TcpDiscoveryClientPingResponse)
+                processClientPingResponse((TcpDiscoveryClientPingResponse)msg);
+            else if (msg instanceof TcpDiscoveryPingRequest)
+                processPingRequest();
+
+            spi.stats.onMessageProcessingFinished(msg);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
+            if (spi.getSpiContext().isStopping())
+                return;
+
+            TcpDiscoveryNode node = msg.node();
+
+            UUID newNodeId = node.id();
+
+            if (getLocalNodeId().equals(newNodeId)) {
+                if (joinLatch.getCount() > 0) {
+                    Collection<TcpDiscoveryNode> top = msg.topology();
+
+                    if (top != null) {
+                        spi.gridStartTime = msg.gridStartTime();
+
+                        for (TcpDiscoveryNode n : top) {
+                            if (n.order() > 0)
+                                n.visible(true);
+
+                            rmtNodes.put(n.id(), n);
+                        }
+
+                        topHist.clear();
+
+                        if (msg.topologyHistory() != null)
+                            topHist.putAll(msg.topologyHistory());
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Discarding node added message with empty topology: " + msg);
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("Discarding node added message (this message has already been processed) " +
+                        "[msg=" + msg + ", locNode=" + locNode + ']');
+            }
+            else {
+                boolean topChanged = rmtNodes.putIfAbsent(newNodeId, node) == null;
+
+                if (topChanged) {
+                    if (log.isDebugEnabled())
+                        log.debug("Added new node to topology: " + node);
+
+                    Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
+
+                    if (data != null)
+                        spi.onExchange(newNodeId, newNodeId, data, null);
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
+            if (spi.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.nodeId())) {
+                if (joinLatch.getCount() > 0) {
+                    Map<UUID, Map<Integer, byte[]>> dataMap = msg.clientDiscoData();
+
+                    if (dataMap != null) {
+                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
+                            spi.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
+                    }
+
+                    locNode.setAttributes(msg.clientNodeAttributes());
+                    locNode.visible(true);
+
+                    long topVer = msg.topologyVersion();
+
+                    locNode.order(topVer);
+
+                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer));
+
+                    joinErr = null;
+
+                    joinLatch.countDown();
+
+                    spi.stats.onJoinFinished();
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("Discarding node add finished message (this message has already been processed) " +
+                        "[msg=" + msg + ", locNode=" + locNode + ']');
+            }
+            else {
+                TcpDiscoveryNode node = rmtNodes.get(msg.nodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node add finished message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                long topVer = msg.topologyVersion();
+
+                node.order(topVer);
+                node.visible(true);
+
+                if (spi.locNodeVer.equals(node.version()))
+                    node.version(spi.locNodeVer);
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(topVer);
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node add finished message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_JOINED, topVer, node, top);
+
+                spi.stats.onNodeJoined();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeLeftMessage(TcpDiscoveryNodeLeftMessage msg) {
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                if (log.isDebugEnabled())
+                    log.debug("Received node left message for local node: " + msg);
+
+                leaveLatch.countDown();
+            }
+            else {
+                if (spi.getSpiContext().isStopping())
+                    return;
+
+                TcpDiscoveryNode node = rmtNodes.remove(msg.creatorNodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node left message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node left message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_LEFT, msg.topologyVersion(), node, top);
+
+                spi.stats.onNodeLeft();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) {
+            if (spi.getSpiContext().isStopping()) {
+                if (!getLocalNodeId().equals(msg.creatorNodeId()) && getLocalNodeId().equals(msg.failedNodeId())) {
+                    if (leaveLatch.getCount() > 0) {
+                        log.debug("Remote node fail this node while node is stopping [locNode=" + getLocalNodeId()
+                            + ", rmtNode=" + msg.creatorNodeId() + ']');
+
+                        leaveLatch.countDown();
+                    }
+                }
+
+                return;
+            }
+
+            if (!getLocalNodeId().equals(msg.creatorNodeId())) {
+                TcpDiscoveryNode node = rmtNodes.remove(msg.failedNodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node failed message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node failed message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
+
+                spi.stats.onNodeFailed();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) {
+            if (spi.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                assert msg.senderNodeId() != null;
+
+                if (log.isDebugEnabled())
+                    log.debug("Received heartbeat response: " + msg);
+            }
+            else {
+                long tstamp = U.currentTimeMillis();
+
+                if (msg.hasMetrics()) {
+                    for (Map.Entry<UUID, TcpDiscoveryHeartbeatMessage.MetricsSet> e : msg.metrics().entrySet()) {
+                        UUID nodeId = e.getKey();
+
+                        TcpDiscoveryHeartbeatMessage.MetricsSet metricsSet = e.getValue();
+
+                        Map<Integer, CacheMetrics> cacheMetrics = msg.hasCacheMetrics() ?
+                            msg.cacheMetrics().get(nodeId) : Collections.<Integer, CacheMetrics>emptyMap();
+
+                        updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp);
+
+                        for (T2<UUID, ClusterMetrics> t : metricsSet.clientMetrics())
+                            updateMetrics(t.get1(), t.get2(), cacheMetrics, tstamp);
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) {
+            if (spi.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                assert msg.success();
+
+                currSock = reconnector.sock;
+
+                sockWriter.setSocket(currSock);
+                sockReader.setSocket(currSock, locNode.clientRouterNodeId());
+
+                reconnector = null;
+
+                pending = true;
+
+                try {
+                    for (TcpDiscoveryAbstractMessage pendingMsg : msg.pendingMessages())
+                        processDiscoveryMessage(pendingMsg);
+                }
+                finally {
+                    pending = false;
+                }
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Discarding reconnect message for another client: " + msg);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
+            if (msg.verified() && joinLatch.getCount() == 0) {
+                DiscoverySpiListener lsnr = spi.lsnr;
+
+                if (lsnr != null) {
+                    UUID nodeId = msg.creatorNodeId();
+
+                    TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
+
+                    if (node != null && node.visible()) {
+                        try {
+                            DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh);
+
+                            notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj);
+                        }
+                        catch (Throwable e) {
+                            U.error(log, "Failed to unmarshal discovery custom message.", e);
+                        }
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Received metrics from unknown node: " + nodeId);
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processClientPingResponse(TcpDiscoveryClientPingResponse msg) {
+            GridFutureAdapter<Boolean> fut = pingFuts.remove(msg.nodeToPing());
+
+            if (fut != null)
+                fut.onDone(msg.result());
+        }
+
+        /**
+         * Router want to ping this client.
+         */
+        private void processPingRequest() {
+            TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(getLocalNodeId());
+
+            res.client(true);
+
+            sockWriter.sendMessage(res);
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @param metrics Metrics.
+         * @param cacheMetrics Cache metrics.
+         * @param tstamp Timestamp.
+         */
+        private void updateMetrics(UUID nodeId,
+            ClusterMetrics metrics,
+            Map<Integer, CacheMetrics> cacheMetrics,
+            long tstamp)
+        {
+            assert nodeId != null;
+            assert metrics != null;
+            assert cacheMetrics != null;
+
+            TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
+
+            if (node != null && node.visible()) {
+                node.setMetrics(metrics);
+                node.setCacheMetrics(cacheMetrics);
+
+                node.lastUpdateTime(tstamp);
+
+                notifyDiscovery(EVT_NODE_METRICS_UPDATED, topVer, node, allVisibleNodes());
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Received metrics from unknown node: " + nodeId);
+        }
+
+        /**
+         * @param type Event type.
+         * @param topVer Topology version.
+         * @param node Node.
+         * @param top Topology snapshot.
+         */
+        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top) {
+            notifyDiscovery(type, topVer, node, top, null);
+        }
+
+        /**
+         * @param type Event type.
+         * @param topVer Topology version.
+         * @param node Node.
+         * @param top Topology snapshot.
+         */
+        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top,
+            @Nullable DiscoverySpiCustomMessage data) {
+            DiscoverySpiListener lsnr = spi.lsnr;
+
+            if (lsnr != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
+                        ", topVer=" + topVer + ']');
+
+                lsnr.onDiscovery(type, topVer, node, top, new TreeMap<>(topHist), data);
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Skipped discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
+                    ", topVer=" + topVer + ']');
+        }
+
+        /**
+         * @param msg Message.
+         */
+        public void addMessage(Object msg) {
+            queue.add(msg);
+        }
+
+        /**
+         *
+         */
+        public int queueSize() {
+            return queue.size();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SocketClosedMessage {
+        /** */
+        private final Socket sock;
+
+        /**
+         * @param sock Socket.
+         */
+        private SocketClosedMessage(Socket sock) {
+            this.sock = sock;
+        }
+    }
+}


[08/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
new file mode 100644
index 0000000..a8da54b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/EvictionAbstractTest.java
@@ -0,0 +1,1056 @@
+/*
+ * 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.eviction;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import java.lang.reflect.*;
+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.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.processors.cache.eviction.EvictionAbstractTest.EvictionPolicyProxy.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ * Base class for eviction tests.
+ */
+public abstract class EvictionAbstractTest<T extends EvictionPolicy<?, ?>>
+    extends GridCommonAbstractTest {
+    /** IP finder. */
+    protected static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Put entry size. */
+    protected static final int PUT_ENTRY_SIZE = 10;
+
+    /** Replicated cache. */
+    protected CacheMode mode = REPLICATED;
+
+    /** Near enabled flag. */
+    protected boolean nearEnabled;
+
+    /** Evict backup sync. */
+    protected boolean evictSync;
+
+    /** Evict near sync. */
+    protected boolean evictNearSync = true;
+
+    /** Policy max. */
+    protected int plcMax = 10;
+
+    /** Policy batch size. */
+    protected int plcBatchSize = 1;
+
+    /** Policy max memory size. */
+    protected long plcMaxMemSize = 0;
+
+    /** Near policy max. */
+    protected int nearMax = 3;
+
+    /** Synchronous commit. */
+    protected boolean syncCommit;
+
+    /** */
+    protected int gridCnt = 2;
+
+    /** */
+    protected EvictionFilter<?, ?> filter;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(mode);
+        cc.setEvictionPolicy(createPolicy(plcMax));
+        cc.setEvictSynchronized(evictSync);
+        cc.setSwapEnabled(false);
+        cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC);
+        cc.setStartSize(plcMax);
+        cc.setAtomicityMode(TRANSACTIONAL);
+
+        if (nearEnabled) {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+            nearCfg.setNearEvictionPolicy(createNearPolicy(nearMax));
+
+            cc.setNearConfiguration(nearCfg);
+        }
+        else
+            cc.setNearConfiguration(null);
+
+        if (mode == PARTITIONED)
+            cc.setBackups(1);
+
+        if (filter != null)
+            cc.setEvictionFilter(filter);
+
+        c.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        c.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
+
+        c.setIncludeProperties();
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        filter = null;
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePolicy() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestPolicy();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePolicyWithBatch() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestPolicyWithBatch();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePolicy() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 3 * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 1;
+
+        doTestPolicy();
+    }
+
+    /**
+     * Batch ignored when {@code maxSize > 0} and {@code maxMemSize > 0}.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePolicyWithBatch() throws Exception {
+        plcMax = 3;
+        plcMaxMemSize = 10 * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 2;
+
+        doTestPolicy();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeMemory() throws Exception {
+        int max = 10;
+
+        plcMax = max;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeMemoryWithBatch() throws Exception {
+        int max = 10;
+
+        plcMax = max;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeMemory() throws Exception {
+        int max = 10;
+
+        plcMax = 0;
+        plcMaxMemSize = max * MockEntry.ENTRY_SIZE;
+        plcBatchSize = 1;
+
+        doTestMemory(max);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeRandom() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeRandomWithBatch() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeRandom() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 10 * MockEntry.KEY_SIZE;
+        plcBatchSize = 1;
+
+        doTestRandom();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeAllowEmptyEntries() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 1;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizeAllowEmptyEntriesWithBatch() throws Exception {
+        plcMax = 10;
+        plcMaxMemSize = 0;
+        plcBatchSize = 2;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizeAllowEmptyEntries() throws Exception {
+        plcMax = 0;
+        plcMaxMemSize = 10 * MockEntry.KEY_SIZE;
+        plcBatchSize = 1;
+
+        doTestAllowEmptyEntries();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePut() throws Exception {
+        plcMax = 100;
+        plcBatchSize = 1;
+        plcMaxMemSize = 0;
+
+        doTestPut(plcMax);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxSizePutWithBatch() throws Exception {
+        plcMax = 100;
+        plcBatchSize = 2;
+        plcMaxMemSize = 0;
+
+        doTestPut(plcMax);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMaxMemSizePut() throws Exception {
+        int max = 100;
+
+        plcMax = 0;
+        plcBatchSize = 2;
+        plcMaxMemSize = max * PUT_ENTRY_SIZE;
+
+        doTestPut(max);
+    }
+
+    /**
+     * Tests policy behaviour.
+     *
+     * @throws Exception If failed.
+     */
+    protected abstract void doTestPolicy() throws Exception;
+
+    /**
+     * Tests policy behaviour with batch enabled.
+     *
+     * @throws Exception If failed.
+     */
+    protected abstract void doTestPolicyWithBatch() throws Exception;
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestAllowEmptyEntries() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1");
+            MockEntry e2 = new MockEntry("2");
+            MockEntry e3 = new MockEntry("3");
+            MockEntry e4 = new MockEntry("4");
+            MockEntry e5 = new MockEntry("5");
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            p.onEntryAccessed(false, e1);
+
+            assertFalse(e1.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e2);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+
+            p.onEntryAccessed(false, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            check(p.queue().size(), MockEntry.KEY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestMemory(int max) throws Exception {
+        try {
+            startGrid();
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            int cnt = max + plcBatchSize;
+
+            for (int i = 0; i < cnt; i++)
+                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
+
+            info(p);
+
+            check(max, MockEntry.ENTRY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestRandom() throws Exception {
+        try {
+            startGrid();
+
+            EvictionPolicyProxy p = proxy(policy());
+
+            int max = 10;
+
+            Random rand = new Random();
+
+            int keys = 31;
+
+            MockEntry[] entries = new MockEntry[keys];
+
+            for (int i = 0; i < entries.length; i++)
+                entries[i] = new MockEntry(Integer.toString(i));
+
+            int runs = 5000000;
+
+            for (int i = 0; i < runs; i++) {
+                boolean rmv = rand.nextBoolean();
+
+                int j = rand.nextInt(entries.length);
+
+                MockEntry e = entry(entries, j);
+
+                if (rmv)
+                    entries[j] = new MockEntry(Integer.toString(j));
+
+                p.onEntryAccessed(rmv, e);
+            }
+
+            info(p);
+
+            assertTrue(p.getCurrentSize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize));
+            assertTrue(p.getCurrentMemorySize() <= (plcMaxMemSize > 0 ? max : max + plcBatchSize) * MockEntry.KEY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void doTestPut(int max) throws Exception {
+        mode = LOCAL;
+        syncCommit = true;
+
+        try {
+            Ignite ignite = startGrid();
+
+            IgniteCache<Object, Object> cache = ignite.cache(null);
+
+            int cnt = 500;
+
+            int min = Integer.MAX_VALUE;
+
+            int minIdx = 0;
+
+            for (int i = 0; i < cnt; i++) {
+                cache.put(i, i);
+
+                int cacheSize = cache.size();
+
+                if (i > max && cacheSize < min) {
+                    min = cacheSize;
+                    minIdx = i;
+                }
+            }
+
+            assertTrue("Min cache size is too small: " + min, min >= max);
+
+            check(max, PUT_ENTRY_SIZE);
+
+            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
+            info("Current cache size " + cache.size());
+            info("Current cache key size " + cache.size());
+
+            min = Integer.MAX_VALUE;
+
+            minIdx = 0;
+
+            // Touch.
+            for (int i = cnt; --i > cnt - max;) {
+                cache.get(i);
+
+                int cacheSize = cache.size();
+
+                if (cacheSize < min) {
+                    min = cacheSize;
+                    minIdx = i;
+                }
+            }
+
+            info("----");
+            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
+            info("Current cache size " + cache.size());
+            info("Current cache key size " + cache.size());
+
+            check(max, PUT_ENTRY_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param arr Array.
+     * @param idx Index.
+     * @return Entry at the index.
+     */
+    protected MockEntry entry(MockEntry[] arr, int idx) {
+        MockEntry e = arr[idx];
+
+        if (e.isEvicted())
+            e = arr[idx] = new MockEntry(e.getKey());
+
+        return e;
+    }
+
+    /**
+     * @param prefix Prefix.
+     * @param p Policy.
+     */
+    protected void info(String prefix, EvictionPolicy<?, ?> p) {
+        info(prefix + ": " + p.toString());
+    }
+
+    /** @param p Policy. */
+    protected void info(EvictionPolicy<?, ?> p) {
+        info(p.toString());
+    }
+
+    /**
+     * @param c1 Policy collection.
+     * @param c2 Expected list.
+     */
+    protected static void check(Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
+        check(c1, F.asList(c2));
+    }
+
+    /**
+     * @param expSize Expected size.
+     * @param entrySize Entry size.
+     */
+    protected void check(int expSize, int entrySize) {
+        EvictionPolicyProxy proxy = proxy(policy());
+
+        assertEquals(expSize, proxy.getCurrentSize());
+        assertEquals(expSize * entrySize, proxy.getCurrentMemorySize());
+    }
+
+    /**
+     * @param entrySize Entry size.
+     * @param c1 Closure 1.
+     * @param c2 Closure 2.
+     */
+    protected void check(int entrySize, Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
+        check(c2.length, entrySize);
+
+        check(c1, c2);
+    }
+
+    /** @return Policy. */
+    @SuppressWarnings({"unchecked"})
+    protected T policy() {
+        return (T)grid().cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T policy(int i) {
+        return (T)grid(i).cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
+    }
+
+    /**
+     * @param i Grid index.
+     * @return Policy.
+     */
+    @SuppressWarnings({"unchecked"})
+    protected T nearPolicy(int i) {
+        CacheConfiguration cfg = grid(i).cache(null).getConfiguration(CacheConfiguration.class);
+
+        NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
+
+        return (T)(nearCfg == null ? null : nearCfg.getNearEvictionPolicy());
+    }
+
+    /**
+     * @param c1 Policy collection.
+     * @param c2 Expected list.
+     */
+    protected static void check(Collection<EvictableEntry<String, String>> c1, List<MockEntry> c2) {
+        assert c1.size() == c2.size() : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
+
+        assert c1.containsAll(c2) : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
+
+        int i = 0;
+
+        // Check order.
+        for (Cache.Entry<String, String> e : c1)
+            assertEquals(e, c2.get(i++));
+    }
+
+    /**
+     * @param c Collection.
+     * @return String.
+     */
+    @SuppressWarnings("unchecked")
+    protected static String string(Iterable<? extends Cache.Entry> c) {
+        return "[" +
+            F.fold(
+                c,
+                "",
+                new C2<Cache.Entry, String, String>() {
+                    @Override public String apply(Cache.Entry e, String b) {
+                        return b.isEmpty() ? e.getKey().toString() : b + ", " + e.getKey();
+                    }
+                }) +
+            "]]";
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxSizePartitionedNearDisabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 10;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxSizePartitionedNearDisabledWithBatch() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 10;
+        plcBatchSize = 2;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testMaxMemSizePartitionedNearDisabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 0;
+        plcMaxMemSize = 100;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabled() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        nearMax = 3;
+        plcMax = 10;
+        evictNearSync = true;
+        syncCommit = true;
+
+        gridCnt = 2;
+
+        checkPartitioned(); // Near size is 0 because of backups present.
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearDisabledMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 100;
+        evictSync = false;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = false;
+        plcMax = 100;
+        evictSync = true;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabledMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        plcMax = 10;
+        evictSync = false;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /** @throws Exception If failed. */
+    public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
+        mode = PARTITIONED;
+        nearEnabled = true;
+        plcMax = 10;
+        evictSync = true;
+
+        gridCnt = 2;
+
+        checkPartitionedMultiThreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void checkPartitioned() throws Exception {
+
+        int endSize = nearEnabled ? 0 : plcMax;
+
+        int endPlcSize = nearEnabled ? 0 : plcMax;
+
+        startGridsMultiThreaded(gridCnt);
+
+        try {
+            Random rand = new Random();
+
+            int cnt = 500;
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(rand.nextInt(2)).cache(null);
+
+                int key = rand.nextInt(100);
+                String val = Integer.toString(key);
+
+                cache.put(key, val);
+
+                if (i % 100 == 0)
+                    info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
+            }
+
+            if (nearEnabled) {
+                for (int i = 0; i < gridCnt; i++)
+                    assertEquals(endSize, near(i).nearSize());
+
+                if (endPlcSize >= 0)
+                    checkNearPolicies(endPlcSize);
+            }
+            else {
+                if (plcMaxMemSize > 0) {
+                    for (int i = 0; i < gridCnt; i++) {
+                        GridDhtColocatedCache<Object, Object> cache = colocated(i);
+
+                        int memSize = 0;
+
+                        for (Cache.Entry<Object, Object> entry : cache.entrySet())
+                            memSize += entry.unwrap(EvictableEntry.class).size();
+
+                        EvictionPolicyProxy plc = proxy(policy(i));
+
+                        assertTrue(plc.getCurrentMemorySize() <= memSize);
+                    }
+                }
+
+                if (plcMax > 0) {
+                    for (int i = 0; i < gridCnt; i++) {
+                        int actual = colocated(i).size();
+
+                        assertTrue("Cache size is greater then policy size [expected=" + endSize + ", actual=" + actual + ']',
+                            actual <= endSize + (plcMaxMemSize > 0 ? 1 : plcBatchSize));
+                    }
+                }
+
+                checkPolicies();
+            }
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    protected void checkPartitionedMultiThreaded() throws Exception {
+        try {
+            startGridsMultiThreaded(gridCnt);
+
+            final Random rand = new Random();
+
+            final AtomicInteger cntr = new AtomicInteger();
+
+            multithreaded(new Callable() {
+                @Nullable @Override public Object call() throws Exception {
+                    int cnt = 100;
+
+                    for (int i = 0; i < cnt && !Thread.currentThread().isInterrupted(); i++) {
+                        IgniteEx grid = grid(rand.nextInt(2));
+
+                        IgniteCache<Integer, String> cache = grid.cache(null);
+
+                        int key = rand.nextInt(1000);
+                        String val = Integer.toString(key);
+
+                        try (Transaction tx = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                            String v = cache.get(key);
+
+                            assert v == null || v.equals(Integer.toString(key)) : "Invalid value for key [key=" + key +
+                                ", val=" + v + ']';
+
+                            cache.put(key, val);
+
+                            tx.commit();
+                        }
+
+                        if (cntr.incrementAndGet() % 100 == 0)
+                            info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
+                    }
+
+                    return null;
+                }
+            }, 10);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @param plcMax Policy max.
+     * @return Policy.
+     */
+    protected abstract T createPolicy(int plcMax);
+
+    /**
+     * @param nearMax Near max.
+     * @return Policy.
+     */
+    protected abstract T createNearPolicy(int nearMax);
+
+    /**
+     * Performs after-test near policy check.
+     *
+     * @param nearMax Near max.
+     */
+    protected void checkNearPolicies(int nearMax) {
+        for (int i = 0; i < gridCnt; i++) {
+
+            EvictionPolicyProxy proxy = proxy(nearPolicy(i));
+
+            for (EvictableEntry e : proxy.queue())
+                assert !e.isCached() : "Invalid near policy size: " + proxy.queue();
+        }
+    }
+
+    /**
+     * Performs after-test policy check.
+     */
+    protected void checkPolicies() {
+        for (int i = 0; i < gridCnt; i++) {
+            if (plcMaxMemSize > 0) {
+                int size = 0;
+
+                for (EvictableEntry entry : proxy(policy(i)).queue())
+                    size += entry.size();
+
+                assertEquals(size, proxy(policy(i)).getCurrentMemorySize());
+            }
+            else
+                assertTrue(proxy(policy(i)).queue().size() <= plcMax + plcBatchSize);
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings({"PublicConstructorInNonPublicClass"})
+    protected static class MockEntry extends GridCacheMockEntry<String, String> {
+        /** Key size. */
+        public static final int KEY_SIZE = 1;
+
+        /** Value size. */
+        public static final int VALUE_SIZE = 1;
+
+        /** Entry size. */
+        public static final int ENTRY_SIZE = KEY_SIZE + VALUE_SIZE;
+
+        /** */
+        private IgniteCache<String, String> parent;
+
+        /** Entry value. */
+        private String val;
+
+        /** @param key Key. */
+        public MockEntry(String key) {
+            super(key);
+        }
+
+        /**
+         * @param key Key.
+         * @param val Value.
+         */
+        public MockEntry(String key, String val) {
+            super(key);
+
+            this.val = val;
+        }
+
+        /**
+         * @param key Key.
+         * @param parent Parent.
+         */
+        public MockEntry(String key, @Nullable IgniteCache<String, String> parent) {
+            super(key);
+
+            this.parent = parent;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("unchecked")
+        @Override public <T> T unwrap(Class<T> clazz) {
+            if (clazz.isAssignableFrom(IgniteCache.class))
+                return (T)parent;
+
+            return super.unwrap(clazz);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getValue() throws IllegalStateException {
+            return val;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int size() {
+            return val == null ? KEY_SIZE : ENTRY_SIZE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(MockEntry.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Rvicition policy proxy.
+     */
+    public static class EvictionPolicyProxy implements EvictionPolicy {
+        /** Policy. */
+        private final EvictionPolicy plc;
+
+        /**
+         * @param plc Policy.
+         */
+        private EvictionPolicyProxy(EvictionPolicy plc) {
+            this.plc = plc;
+        }
+
+        /**
+         * @param plc Policy.
+         */
+        public static EvictionPolicyProxy proxy(EvictionPolicy plc) {
+            return new EvictionPolicyProxy(plc);
+        }
+
+        /**
+         * Get current size.
+         */
+        public int getCurrentSize() {
+            try {
+                return (Integer)plc.getClass().getDeclaredMethod("getCurrentSize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * Current memory size.
+         */
+        public long getCurrentMemorySize() {
+            try {
+                return (Long)plc.getClass().getDeclaredMethod("getCurrentMemorySize").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * Current queue.
+         */
+        public Collection<EvictableEntry> queue() {
+            try {
+                return (Collection<EvictableEntry>)plc.getClass().getDeclaredMethod("queue").invoke(plc);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * @param rmv Remove.
+         * @param entry Entry.
+         */
+        @Override public void onEntryAccessed(boolean rmv, EvictableEntry entry) {
+            try {
+                plc.getClass()
+                    .getDeclaredMethod("onEntryAccessed", boolean.class, EvictableEntry.class)
+                    .invoke(plc, rmv, entry);
+            }
+            catch (NoSuchMethodException | InvocationTargetException | IllegalAccessException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
index 36598fd..da5017a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
@@ -90,7 +90,10 @@ public class GridCacheBatchEvictUnswapSelfTest extends GridCacheAbstractSelfTest
         cacheCfg.setWriteThrough(true);
         cacheCfg.setLoadPreviousValue(true);
 
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(EVICT_PLC_SIZE));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_PLC_SIZE);
+
+        cacheCfg.setEvictionPolicy(plc);
         cacheCfg.setSwapEnabled(true);
         cacheCfg.setEvictSynchronized(false);
         cacheCfg.setNearConfiguration(null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
index 4daa908..921422e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.cache.eviction.sorted.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -100,7 +101,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocalTwoKeys() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(1);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
 
         keyCnt = 2;
         threadCnt = Runtime.getRuntime().availableProcessors() / 2;
@@ -112,7 +116,25 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocalTwoKeys() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(1);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
+
+        keyCnt = 2;
+        threadCnt = Runtime.getRuntime().availableProcessors() / 2;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocalTwoKeys() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(1);
+
+        this.plc = plc;
 
         keyCnt = 2;
         threadCnt = Runtime.getRuntime().availableProcessors() / 2;
@@ -124,7 +146,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocalFewKeys() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE + 5;
 
@@ -135,7 +160,24 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocalFewKeys() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
+
+        keyCnt = POLICY_QUEUE_SIZE + 5;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocalFewKeys() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE + 5;
 
@@ -146,7 +188,10 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyFifoLocal() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE * 10;
 
@@ -157,7 +202,24 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testPolicyConsistencyLruLocal() throws Exception {
-        plc = new LruEvictionPolicy<Object, Object>(POLICY_QUEUE_SIZE);
+        LruEvictionPolicy<Object, Object> plc = new LruEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
+
+        keyCnt = POLICY_QUEUE_SIZE * 10;
+
+        checkPolicyConsistency();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPolicyConsistencySortedLocal() throws Exception {
+        SortedEvictionPolicy<Object, Object> plc = new SortedEvictionPolicy<>();
+        plc.setMaxSize(POLICY_QUEUE_SIZE);
+
+        this.plc = plc;
 
         keyCnt = POLICY_QUEUE_SIZE * 10;
 
@@ -177,8 +239,7 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
 
             IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
-                    @Override
-                    public Object call() throws Exception {
+                    @Override public Object call() throws Exception {
                         final Random rnd = new Random();
 
                         for (int i = 0; i < ITERATION_CNT; i++) {
@@ -258,6 +319,11 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
 
             return plc0.queue();
         }
+        else if (plc instanceof SortedEvictionPolicy) {
+            SortedEvictionPolicy<Integer, Integer> plc0 = (SortedEvictionPolicy<Integer, Integer>)plc;
+
+            return plc0.queue();
+        }
 
         assert false : "Unexpected policy type: " + plc.getClass().getName();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
index 3cadc45..270f991 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.cache.eviction.sorted.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -98,7 +99,11 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
      */
     public void testConcurrentPutsFifoLocal() throws Exception {
         mode = LOCAL;
-        plc = new FifoEvictionPolicy<Object, Object>(1000);
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
         warmUpPutsCnt = 100000;
         iterCnt = 100000;
 
@@ -110,7 +115,27 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
      */
     public void testConcurrentPutsLruLocal() throws Exception {
         mode = LOCAL;
-        plc = new LruEvictionPolicy<Object, Object>(1000);
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
+        warmUpPutsCnt = 100000;
+        iterCnt = 100000;
+
+        checkConcurrentPuts();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentPutsSortedLocal() throws Exception {
+        mode = LOCAL;
+
+        SortedEvictionPolicy plc = new SortedEvictionPolicy();
+        plc.setMaxSize(1000);
+
+        this.plc = plc;
         warmUpPutsCnt = 100000;
         iterCnt = 100000;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
index 396f360..75c4dfe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheDistributedEvictionsSelfTest.java
@@ -86,7 +86,10 @@ public class GridCacheDistributedEvictionsSelfTest extends GridCommonAbstractTes
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
 
         // Set only DHT policy, leave default near policy.
-        cc.setEvictionPolicy(new FifoEvictionPolicy<>(10));
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10);
+
+        cc.setEvictionPolicy(plc);
         cc.setEvictSynchronized(evictSync);
         cc.setEvictSynchronizedKeyBufferSize(1);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
index 768bced..8e17eed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEmptyEntriesAbstractSelfTest.java
@@ -120,14 +120,19 @@ public abstract class GridCacheEmptyEntriesAbstractSelfTest extends GridCommonAb
      * @throws Exception If failed.
      */
     public void testFifo() throws Exception {
-        plc = new FifoEvictionPolicy(50);
-        nearPlc = new FifoEvictionPolicy(50);
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(50);
+        this.plc = plc;
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(50);
+        this.nearPlc = nearPlc;
 
         checkPolicy();
     }
 
     /**
-     * Checks policy with and without store set.
+     * Checks policy with and without store queue.
      *
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
deleted file mode 100644
index e0dab7d..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionAbstractTest.java
+++ /dev/null
@@ -1,484 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
-import org.apache.ignite.testframework.junits.common.*;
-import org.apache.ignite.transactions.*;
-import org.jetbrains.annotations.*;
-
-import javax.cache.*;
-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.*;
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.transactions.TransactionConcurrency.*;
-import static org.apache.ignite.transactions.TransactionIsolation.*;
-
-/**
- * Base class for eviction tests.
- */
-public abstract class GridCacheEvictionAbstractTest<T extends EvictionPolicy<?, ?>>
-    extends GridCommonAbstractTest {
-    /** IP finder. */
-    protected static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Replicated cache. */
-    protected CacheMode mode = REPLICATED;
-
-    /** Near enabled flag. */
-    protected boolean nearEnabled;
-
-    /** Evict backup sync. */
-    protected boolean evictSync;
-
-    /** Evict near sync. */
-    protected boolean evictNearSync = true;
-
-    /** Policy batch size. */
-    protected int plcBatchSize = 0;
-
-    /** Policy max. */
-    protected int plcMax = 10;
-
-    /** Near policy max. */
-    protected int nearMax = 3;
-
-    /** Synchronous commit. */
-    protected boolean syncCommit;
-
-    /** */
-    protected int gridCnt = 2;
-
-    /** */
-    protected EvictionFilter<?, ?> filter;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        CacheConfiguration cc = defaultCacheConfiguration();
-
-        cc.setCacheMode(mode);
-        cc.setEvictionPolicy(createPolicy(plcMax));
-        cc.setEvictSynchronized(evictSync);
-        cc.setSwapEnabled(false);
-        cc.setWriteSynchronizationMode(syncCommit ? FULL_SYNC : FULL_ASYNC);
-        cc.setStartSize(plcMax);
-        cc.setAtomicityMode(TRANSACTIONAL);
-
-        if (nearEnabled) {
-            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-
-            nearCfg.setNearEvictionPolicy(createNearPolicy(nearMax));
-
-            cc.setNearConfiguration(nearCfg);
-        }
-        else
-            cc.setNearConfiguration(null);
-
-        if (mode == PARTITIONED)
-            cc.setBackups(1);
-
-        if (filter != null)
-            cc.setEvictionFilter(filter);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        c.setIncludeEventTypes(EVT_TASK_FAILED, EVT_TASK_FINISHED, EVT_JOB_MAPPED);
-
-        c.setIncludeProperties();
-
-        return c;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        filter = null;
-
-        super.afterTestsStopped();
-    }
-
-    /**
-     * @param arr Array.
-     * @param idx Index.
-     * @return Entry at the index.
-     */
-    protected MockEntry entry(MockEntry[] arr, int idx) {
-        MockEntry e = arr[idx];
-
-        if (e.isEvicted())
-            e = arr[idx] = new MockEntry(e.getKey());
-
-        return e;
-    }
-
-    /**
-     * @param prefix Prefix.
-     * @param p Policy.
-     */
-    protected void info(String prefix, EvictionPolicy<?, ?> p) {
-        info(prefix + ": " + p.toString());
-    }
-
-    /** @param p Policy. */
-    protected void info(EvictionPolicy<?, ?> p) {
-        info(p.toString());
-    }
-
-    /**
-     * @param c1 Policy collection.
-     * @param c2 Expected list.
-     */
-    protected void check(Collection<EvictableEntry<String, String>> c1, MockEntry... c2) {
-        check(c1, F.asList(c2));
-    }
-
-    /** @return Policy. */
-    @SuppressWarnings({"unchecked"})
-    protected T policy() {
-        return (T)grid().cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
-    }
-
-    /**
-     * @param i Grid index.
-     * @return Policy.
-     */
-    @SuppressWarnings({"unchecked"})
-    protected T policy(int i) {
-        return (T)grid(i).cache(null).getConfiguration(CacheConfiguration.class).getEvictionPolicy();
-    }
-
-    /**
-     * @param i Grid index.
-     * @return Policy.
-     */
-    @SuppressWarnings({"unchecked"})
-    protected T nearPolicy(int i) {
-        CacheConfiguration cfg = grid(i).cache(null).getConfiguration(CacheConfiguration.class);
-
-        NearCacheConfiguration nearCfg = cfg.getNearConfiguration();
-
-        return (T)(nearCfg == null ? null : nearCfg.getNearEvictionPolicy());
-    }
-
-    /**
-     * @param c1 Policy collection.
-     * @param c2 Expected list.
-     */
-    protected void check(Collection<EvictableEntry<String, String>> c1, List<MockEntry> c2) {
-        assert c1.size() == c2.size() : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
-
-        assert c1.containsAll(c2) : "Mismatch [actual=" + string(c1) + ", expected=" + string(c2) + ']';
-
-        int i = 0;
-
-        // Check order.
-        for (Cache.Entry<String, String> e : c1)
-            assertEquals(e, c2.get(i++));
-    }
-
-    /**
-     * @param c Collection.
-     * @return String.
-     */
-    @SuppressWarnings("unchecked")
-    protected String string(Iterable<? extends Cache.Entry> c) {
-        return "[" +
-            F.fold(
-                c,
-                "",
-                new C2<Cache.Entry, String, String>() {
-                    @Override public String apply(Cache.Entry e, String b) {
-                        return b.isEmpty() ? e.getKey().toString() : b + ", " + e.getKey();
-                    }
-                }) +
-            "]]";
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabled() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 10;
-        syncCommit = true;
-
-        gridCnt = 2;
-
-        checkPartitioned(plcMax, plcMax, false);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabled() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        nearMax = 3;
-        plcMax = 10;
-        evictNearSync = true;
-        syncCommit = true;
-
-        gridCnt = 2;
-
-        checkPartitioned(0, 0, true); // Near size is 0 because of backups present.
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabledMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 100;
-        evictSync = false;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = false;
-        plcMax = 100;
-        evictSync = true;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabledMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        plcMax = 10;
-        evictSync = false;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /** @throws Exception If failed. */
-    public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
-        mode = PARTITIONED;
-        nearEnabled = true;
-        plcMax = 10;
-        evictSync = true;
-
-        gridCnt = 2;
-
-        checkPartitionedMultiThreaded(gridCnt);
-    }
-
-    /**
-     * @param endSize Final near size.
-     * @param endPlcSize Final near policy size.
-     * @throws Exception If failed.
-     */
-    private void checkPartitioned(int endSize, int endPlcSize, boolean near) throws Exception {
-        startGridsMultiThreaded(gridCnt);
-
-        try {
-            Random rand = new Random();
-
-            int cnt = 500;
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(rand.nextInt(2)).cache(null);
-
-                int key = rand.nextInt(100);
-                String val = Integer.toString(key);
-
-                cache.put(key, val);
-
-                if (i % 100 == 0)
-                    info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
-            }
-
-            if (near) {
-                for (int i = 0; i < gridCnt; i++)
-                    assertEquals(endSize, near(i).nearSize());
-
-                if (endPlcSize >= 0)
-                    checkNearPolicies(endPlcSize);
-            }
-            else {
-                for (int i = 0; i < gridCnt; i++) {
-                    int actual = colocated(i).size();
-
-                    assertTrue("Cache size is greater then policy size [expected=" + endSize + ", actual=" + actual + ']',
-                        actual <= endSize + plcBatchSize);
-                }
-
-                checkPolicies(endPlcSize);
-            }
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param gridCnt Grid count.
-     * @throws Exception If failed.
-     */
-    protected void checkPartitionedMultiThreaded(int gridCnt) throws Exception {
-        try {
-            startGridsMultiThreaded(gridCnt);
-
-            final Random rand = new Random();
-
-            final AtomicInteger cntr = new AtomicInteger();
-
-            multithreaded(new Callable() {
-                @Nullable @Override public Object call() throws Exception {
-                    int cnt = 100;
-
-                    for (int i = 0; i < cnt && !Thread.currentThread().isInterrupted(); i++) {
-                        IgniteEx grid = grid(rand.nextInt(2));
-
-                        IgniteCache<Integer, String> cache = grid.cache(null);
-
-                        int key = rand.nextInt(1000);
-                        String val = Integer.toString(key);
-
-                        try (Transaction tx = grid.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
-                            String v = cache.get(key);
-
-                            assert v == null || v.equals(Integer.toString(key)) : "Invalid value for key [key=" + key +
-                                ", val=" + v + ']';
-
-                            cache.put(key, val);
-
-                            tx.commit();
-                        }
-
-                        if (cntr.incrementAndGet() % 100 == 0)
-                            info("Stored cache object for key [key=" + key + ", idx=" + i + ']');
-                    }
-
-                    return null;
-                }
-            }, 10);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param plcMax Policy max.
-     * @return Policy.
-     */
-    protected abstract T createPolicy(int plcMax);
-
-    /**
-     * @param nearMax Near max.
-     * @return Policy.
-     */
-    protected abstract T createNearPolicy(int nearMax);
-
-    /**
-     * Performs after-test near policy check.
-     *
-     * @param nearMax Near max.
-     */
-    protected abstract void checkNearPolicies(int nearMax);
-
-    /**
-     * Performs after-test policy check.
-     *
-     * @param plcMax Maximum allowed size of ploicy.
-     */
-    protected abstract void checkPolicies(int plcMax);
-
-    /**
-     *
-     */
-    @SuppressWarnings({"PublicConstructorInNonPublicClass"})
-    protected static class MockEntry extends GridCacheMockEntry<String, String> {
-        /** */
-        private IgniteCache<String, String> parent;
-
-        /** Entry value. */
-        private String val;
-
-        /** @param key Key. */
-        public MockEntry(String key) {
-            super(key);
-        }
-
-        /**
-         * @param key Key.
-         * @param val Value.
-         */
-        public MockEntry(String key, String val) {
-            super(key);
-
-            this.val = val;
-        }
-
-        /**
-         * @param key Key.
-         * @param parent Parent.
-         */
-        public MockEntry(String key, @Nullable IgniteCache<String, String> parent) {
-            super(key);
-
-            this.parent = parent;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public <T> T unwrap(Class<T> clazz) {
-            if (clazz.isAssignableFrom(IgniteCache.class))
-                return (T)parent;
-
-            return super.unwrap(clazz);
-        }
-
-        /** {@inheritDoc} */
-        @Override public String getValue() throws IllegalStateException {
-            return val;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(MockEntry.class, this, super.toString());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
index 9873ce6..55b8d63 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheEvictionTouchSelfTest.java
@@ -110,7 +110,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testPolicyConsistency() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -136,7 +139,7 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
                         info("Stats [iterCnt=" + i + ", size=" + cache.size() + ']');
                 }
 
-                FifoEvictionPolicy<Integer, Integer> plc0 = (FifoEvictionPolicy<Integer, Integer>) plc;
+                FifoEvictionPolicy<Integer, Integer> plc0 = (FifoEvictionPolicy<Integer, Integer>)this.plc;
 
                 if (!plc0.queue().isEmpty()) {
                     for (Cache.Entry<Integer, Integer> e : plc0.queue())
@@ -162,7 +165,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testEvictSingle() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -189,7 +195,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testEvictAll() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(500);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(500);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);
@@ -221,7 +230,10 @@ public class GridCacheEvictionTouchSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testReload() throws Exception {
-        plc = new FifoEvictionPolicy<Object, Object>(100);
+        FifoEvictionPolicy<Object, Object> plc = new FifoEvictionPolicy<>();
+        plc.setMaxSize(100);
+
+        this.plc = plc;
 
         try {
             Ignite ignite = startGrid(1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
index 5898ed9..5d183bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
@@ -83,6 +83,11 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
         return !evicted;
     }
 
+    /** {@inheritDoc} */
+    @Override public int size() {
+        return 0;
+    }
+
     /**
      *
      * @return Evicted or not.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
new file mode 100644
index 0000000..6b6ba51
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/FifoEvictionPolicySelfTest.java
@@ -0,0 +1,262 @@
+/*
+ * 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.eviction.fifo;
+
+import org.apache.ignite.cache.eviction.fifo.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * FIFO eviction policy tests.
+ */
+public class FifoEvictionPolicySelfTest extends
+    EvictionAbstractTest<FifoEvictionPolicy<String, String>> {
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            FifoEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assert !e1.isEvicted();
+            assert !e2.isEvicted();
+            assert !e3.isEvicted();
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assert e1.isEvicted();
+            assert !e2.isEvicted();
+            assert !e3.isEvicted();
+            assert !e4.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assert e2.isEvicted();
+            assert !e3.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert e3.isEvicted();
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assert !e1.isEvicted();
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assert !e4.isEvicted();
+            assert !e5.isEvicted();
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assert !e5.isEvicted();
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            FifoEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted.
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e3.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
+        FifoEvictionPolicy<String, String> plc = new FifoEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        FifoEvictionPolicy<String, String> plc = new FifoEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
deleted file mode 100644
index f907c36..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoBatchEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,384 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.fifo;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * FIFO batch eviction test.
- */
-public class GridCacheFifoBatchEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<FifoEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e1, e2, e3, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            assertEquals(4, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e5);
-
-            // Batch evicted.
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            check(p.queue(), e3, e4, e5, e1);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-            p.setBatchSize(batchSize);
-
-            int cnt = max + batchSize;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(cnt - batchSize, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-
-            p.setBatchSize(batchSize);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize < max + batchSize :
-                "curSize < max + batchSize [curSize=" + curSize + ", max=" + max + ", batchSize=" + batchSize + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 10;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        plcBatchSize = 2;
-
-        super.testPartitionedNearDisabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new FifoEvictionPolicy<>(10, 2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new FifoEvictionPolicy<>(nearMax, 2);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax + policy(i).getBatchSize();
-    }
-}


[26/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index d54e06f..5cbe377 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -121,11 +121,7 @@ public class IgnitionEx {
     };
 
     /** */
-    private static ThreadLocal<Boolean> clientMode = new ThreadLocal<Boolean>() {
-        @Override protected Boolean initialValue() {
-            return null;
-        }
-    };
+    private static ThreadLocal<Boolean> clientMode = new ThreadLocal<>();
 
     /**
      * Checks runtime version to be 1.7.x or 1.8.x.
@@ -196,7 +192,7 @@ public class IgnitionEx {
      * @return Client mode flag.
      */
     public static boolean isClientMode() {
-        return clientMode.get();
+        return clientMode.get() == null ? false : clientMode.get();
     }
 
     /**
@@ -1458,8 +1454,9 @@ public class IgnitionEx {
                 DFLT_PUBLIC_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP));
 
-            // Pre-start all threads as they are guaranteed to be needed.
-            ((ThreadPoolExecutor) execSvc).prestartAllCoreThreads();
+            if (!myCfg.isClientMode())
+                // Pre-start all threads as they are guaranteed to be needed.
+                ((ThreadPoolExecutor)execSvc).prestartAllCoreThreads();
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
@@ -1471,7 +1468,7 @@ public class IgnitionEx {
                 new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
 
             // Pre-start all threads as they are guaranteed to be needed.
-            ((ThreadPoolExecutor) sysExecSvc).prestartAllCoreThreads();
+            ((ThreadPoolExecutor)sysExecSvc).prestartAllCoreThreads();
 
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
@@ -1764,20 +1761,14 @@ public class IgnitionEx {
         public void initializeDefaultCacheConfiguration(IgniteConfiguration cfg) throws IgniteCheckedException {
             List<CacheConfiguration> cacheCfgs = new ArrayList<>();
 
-            boolean clientDisco = cfg.getDiscoverySpi() instanceof TcpClientDiscoverySpi;
-
-            // Add marshaller and utility caches.
-            if (!clientDisco) {
-                cacheCfgs.add(marshallerSystemCache());
+            cacheCfgs.add(marshallerSystemCache());
 
-                cacheCfgs.add(utilitySystemCache());
-            }
+            cacheCfgs.add(utilitySystemCache());
 
             if (IgniteComponentType.HADOOP.inClassPath())
                 cacheCfgs.add(CU.hadoopSystemCache());
 
-            if (cfg.getAtomicConfiguration() != null && !clientDisco)
-                cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
+            cacheCfgs.add(atomicsSystemCache(cfg.getAtomicConfiguration()));
 
             CacheConfiguration[] userCaches = cfg.getCacheConfiguration();
 
@@ -1854,7 +1845,7 @@ public class IgnitionEx {
             if (cfg.getSwapSpaceSpi() == null) {
                 boolean needSwap = false;
 
-                if (cfg.getCacheConfiguration() != null) {
+                if (cfg.getCacheConfiguration() != null && !Boolean.TRUE.equals(cfg.isClientMode())) {
                     for (CacheConfiguration c : cfg.getCacheConfiguration()) {
                         if (c.isSwapEnabled()) {
                             needSwap = true;
@@ -2005,7 +1996,6 @@ public class IgnitionEx {
             ccfg.setWriteSynchronizationMode(FULL_SYNC);
             ccfg.setCacheMode(cfg.getCacheMode());
             ccfg.setNodeFilter(CacheConfiguration.ALL_NODES);
-            ccfg.setNearConfiguration(new NearCacheConfiguration());
 
             if (cfg.getCacheMode() == PARTITIONED)
                 ccfg.setBackups(cfg.getBackups());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
index 5dca2f2..21f2264 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.jsr166.*;
@@ -49,10 +50,29 @@ public abstract class MarshallerContextAdapter implements MarshallerContext {
 
             Enumeration<URL> urls = ldr.getResources(CLS_NAMES_FILE);
 
-            while (urls.hasMoreElements())
+            boolean foundClsNames = false;
+
+            while (urls.hasMoreElements()) {
                 processResource(urls.nextElement());
 
-            processResource(ldr.getResource(JDK_CLS_NAMES_FILE));
+                foundClsNames = true;
+            }
+
+            if (!foundClsNames)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            URL jdkClsNames = ldr.getResource(JDK_CLS_NAMES_FILE);
+
+            if (jdkClsNames == null)
+                throw new IgniteException("Failed to load class names properties file packaged with ignite binaries " +
+                    "[file=" + JDK_CLS_NAMES_FILE + ", ldr=" + ldr + ']');
+
+            processResource(jdkClsNames);
+
+            checkHasClassName(GridDhtPartitionFullMap.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(GridDhtPartitionMap.class.getName(), ldr, CLS_NAMES_FILE);
+            checkHasClassName(HashMap.class.getName(), ldr, JDK_CLS_NAMES_FILE);
         }
         catch (IOException e) {
             throw new IllegalStateException("Failed to initialize marshaller context.", e);
@@ -60,6 +80,18 @@ public abstract class MarshallerContextAdapter implements MarshallerContext {
     }
 
     /**
+     * @param clsName Class name.
+     * @param ldr Class loader used to get properties file.
+     * @param fileName File name.
+     */
+    private void checkHasClassName(String clsName, ClassLoader ldr, String fileName) {
+        if (!map.containsKey(clsName.hashCode()))
+            throw new IgniteException("Failed to read class name from class names properties file. " +
+                "Make sure class names properties file packaged with ignite binaries is not corrupted " +
+                "[clsName=" + clsName + ", fileName=" + fileName + ", ldr=" + ldr + ']');
+    }
+
+    /**
      * @param url Resource URL.
      * @throws IOException In case of error.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 85939a6..e614408 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -59,7 +59,7 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
         ctx.cache().marshallerCache().context().continuousQueries().executeInternalQuery(
             new ContinuousQueryListener(log, workDir),
             null,
-            true,
+            ctx.cache().marshallerCache().context().affinityNode(),
             true
         );
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java b/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java
index ee32692..779b54d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/events/DiscoveryCustomEvent.java
@@ -22,19 +22,17 @@ import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
-import java.io.*;
-
 /**
  * Custom event.
  */
 public class DiscoveryCustomEvent extends DiscoveryEvent {
     /** */
     private static final long serialVersionUID = 0L;
-    
+
     /**
      * Built-in event type: custom event sent.
      * <br>
-     * Generated when someone invoke {@link GridDiscoveryManager#sendCustomEvent(Serializable)}.
+     * Generated when someone invoke {@link GridDiscoveryManager#sendCustomEvent(DiscoveryCustomMessage)}.
      * <p>
      *
      * @see DiscoveryCustomEvent
@@ -42,7 +40,7 @@ public class DiscoveryCustomEvent extends DiscoveryEvent {
     public static final int EVT_DISCOVERY_CUSTOM_EVT = 18;
 
     /** */
-    private Serializable data;
+    private DiscoveryCustomMessage customMsg;
 
     /** Affinity topology version. */
     private AffinityTopologyVersion affTopVer;
@@ -57,15 +55,15 @@ public class DiscoveryCustomEvent extends DiscoveryEvent {
     /**
      * @return Data.
      */
-    public Serializable data() {
-        return data;
+    public DiscoveryCustomMessage customMessage() {
+        return customMsg;
     }
 
     /**
-     * @param data New data.
+     * @param customMsg New customMessage.
      */
-    public void data(Serializable data) {
-        this.data = data;
+    public void customMessage(DiscoveryCustomMessage customMsg) {
+        this.customMsg = customMsg;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
index 11af716..6a6f22a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsMarshaller.java
@@ -73,6 +73,7 @@ public class IgfsMarshaller {
     }
 
     /**
+     * Serializes the message and sends it into the given output stream.
      * @param msg Message.
      * @param hdr Message header.
      * @param out Output.
@@ -119,6 +120,7 @@ public class IgfsMarshaller {
 
                     IgfsPathControlRequest req = (IgfsPathControlRequest)msg;
 
+                    U.writeString(out, req.userName());
                     writePath(out, req.path());
                     writePath(out, req.destinationPath());
                     out.writeBoolean(req.flag());
@@ -236,6 +238,7 @@ public class IgfsMarshaller {
                 case OPEN_CREATE: {
                     IgfsPathControlRequest req = new IgfsPathControlRequest();
 
+                    req.userName(U.readString(in));
                     req.path(readPath(in));
                     req.destinationPath(readPath(in));
                     req.flag(in.readBoolean());
@@ -298,8 +301,6 @@ public class IgfsMarshaller {
                 }
             }
 
-            assert msg != null;
-
             msg.command(cmd);
 
             return msg;
@@ -341,34 +342,4 @@ public class IgfsMarshaller {
 
         return null;
     }
-
-    /**
-     * Writes string to output.
-     *
-     * @param out Data output.
-     * @param str String.
-     * @throws IOException If write failed.
-     */
-    private void writeString(DataOutput out, @Nullable String str) throws IOException {
-        out.writeBoolean(str != null);
-
-        if (str != null)
-            out.writeUTF(str);
-    }
-
-    /**
-     * Reads string from input.
-     *
-     * @param in Data input.
-     * @return Read string.
-     * @throws IOException If read failed.
-     */
-    @Nullable private String readString(DataInput in) throws IOException {
-        boolean hasStr = in.readBoolean();
-
-        if (hasStr)
-            return in.readUTF();
-
-        return null;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
index 7ed1619..2f6e6e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/igfs/common/IgfsPathControlRequest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.igfs.common;
 
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -63,6 +64,9 @@ public class IgfsPathControlRequest extends IgfsMessage {
     /** Last modification time. */
     private long modificationTime;
 
+    /** The user name this control request is made on behalf of. */
+    private String userName;
+
     /**
      * @param path Path.
      */
@@ -235,4 +239,22 @@ public class IgfsPathControlRequest extends IgfsMessage {
     @Override public String toString() {
         return S.toString(IgfsPathControlRequest.class, this, "cmd", command());
     }
+
+    /**
+     * Getter for the user name.
+     * @return user name.
+     */
+    public final String userName() {
+        assert userName != null;
+
+        return userName;
+    }
+
+    /**
+     * Setter for the user name.
+     * @param userName the user name.
+     */
+    public final void userName(String userName) {
+        this.userName = IgfsUtils.fixUserName(userName);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index c93c059..bea4256 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -23,7 +23,7 @@ import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
-import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -31,7 +31,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.swapspace.*;
+
 import org.jetbrains.annotations.*;
 
 import javax.cache.expiry.*;
@@ -439,46 +439,10 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.cache().cache(cacheName).containsKey(key);
                     }
 
-                    @Override public void writeToSwap(String spaceName, Object key, @Nullable Object val,
-                        @Nullable ClassLoader ldr) {
-                        assert ctx.swap().enabled();
-
-                        try {
-                            ctx.swap().write(spaceName, key, val, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
-                    @SuppressWarnings({"unchecked"})
-                    @Nullable @Override public <T> T readFromSwap(String spaceName, SwapKey key,
-                        @Nullable ClassLoader ldr) {
-                        try {
-                            assert ctx.swap().enabled();
-
-                            return ctx.swap().readValue(spaceName, key, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public int partition(String cacheName, Object key) {
                         return ctx.cache().cache(cacheName).affinity().partition(key);
                     }
 
-                    @Override public void removeFromSwap(String spaceName, Object key, @Nullable ClassLoader ldr) {
-                        try {
-                            assert ctx.swap().enabled();
-
-                            ctx.swap().remove(spaceName, key, null, ldr);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public IgniteNodeValidationResult validateNode(ClusterNode node) {
                         for (GridComponent comp : ctx) {
                             IgniteNodeValidationResult err = comp.validateNode(node);
@@ -508,26 +472,6 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         }
                     }
 
-                    @SuppressWarnings("unchecked")
-                    @Nullable @Override public <V> V readValueFromOffheapAndSwap(@Nullable String spaceName,
-                        Object key, @Nullable ClassLoader ldr) {
-                        try {
-                            IgniteInternalCache<Object, V> cache = ctx.cache().cache(spaceName);
-
-                            GridCacheContext cctx = cache.context();
-
-                            if (cctx.isNear())
-                                cctx = cctx.near().dht().context();
-
-                            GridCacheSwapEntry e = cctx.swap().read(cctx.toCacheKeyObject(key), true, true);
-
-                            return e != null ? CU.<V>value(e.value(), cctx, true) : null;
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw U.convertException(e);
-                        }
-                    }
-
                     @Override public MessageFormatter messageFormatter() {
                         return ctx.io().formatter();
                     }
@@ -540,6 +484,14 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.discovery().tryFailNode(nodeId);
                     }
 
+                    @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+                        ctx.timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
+                    }
+
+                    @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+                        ctx.timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj));
+                    }
+
                     /**
                      * @param e Exception to handle.
                      * @return GridSpiException Converted exception.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
index 2e80b6f..ce2a36c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManager.java
@@ -56,11 +56,10 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
     private final GridMessageListener lsnr = new CheckpointRequestListener();
 
     /** */
-    private final ConcurrentMap<IgniteUuid, CheckpointSet> keyMap = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<IgniteUuid, CheckpointSet> keyMap;
 
     /** */
-    private final Collection<IgniteUuid> closedSess = new GridBoundedConcurrentLinkedHashSet<>(
-        MAX_CLOSED_SESS, MAX_CLOSED_SESS, 0.75f, 256, PER_SEGMENT_Q);
+    private final Collection<IgniteUuid> closedSess;
 
     /** Grid marshaller. */
     private final Marshaller marsh;
@@ -72,6 +71,21 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
         super(ctx, ctx.config().getCheckpointSpi());
 
         marsh = ctx.config().getMarshaller();
+
+        if (enabled()) {
+            keyMap = new ConcurrentHashMap8<>();
+
+            closedSess = new GridBoundedConcurrentLinkedHashSet<>(MAX_CLOSED_SESS,
+                MAX_CLOSED_SESS,
+                0.75f,
+                256,
+                PER_SEGMENT_Q);
+        }
+        else {
+            keyMap = null;
+
+            closedSess = null;
+        }
     }
 
     /** {@inheritDoc} */
@@ -112,7 +126,7 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @return Session IDs.
      */
     public Collection<IgniteUuid> sessionIds() {
-        return new ArrayList<>(keyMap.keySet());
+        return enabled() ? new ArrayList<>(keyMap.keySet()) : Collections.<IgniteUuid>emptyList();
     }
 
     /**
@@ -125,8 +139,17 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @return {@code true} if checkpoint has been actually saved, {@code false} otherwise.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public boolean storeCheckpoint(GridTaskSessionInternal ses, String key, Object state, ComputeTaskSessionScope scope,
-        long timeout, boolean override) throws IgniteCheckedException {
+    public boolean storeCheckpoint(GridTaskSessionInternal ses,
+        String key,
+        Object state,
+        ComputeTaskSessionScope scope,
+        long timeout,
+        boolean override)
+        throws IgniteCheckedException
+    {
+        if (!enabled())
+            return false;
+
         assert ses != null;
         assert key != null;
 
@@ -239,6 +262,9 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @return Whether or not checkpoint was removed.
      */
     public boolean removeCheckpoint(String key) {
+        if (!enabled())
+            return false;
+
         assert key != null;
 
         boolean rmv = false;
@@ -256,6 +282,9 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @return Whether or not checkpoint was removed.
      */
     public boolean removeCheckpoint(GridTaskSessionInternal ses, String key) {
+        if (!enabled())
+            return false;
+
         assert ses != null;
         assert key != null;
 
@@ -283,6 +312,9 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @Nullable public Serializable loadCheckpoint(GridTaskSessionInternal ses, String key) throws IgniteCheckedException {
+        if (!enabled())
+            return null;
+
         assert ses != null;
         assert key != null;
 
@@ -309,6 +341,9 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
      * @param cleanup Whether cleanup or not.
      */
     public void onSessionEnd(GridTaskSessionInternal ses, boolean cleanup) {
+        if (!enabled())
+            return;
+
         closedSess.add(ses.getId());
 
         // If on task node.
@@ -358,7 +393,7 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
     @Override public void printMemoryStats() {
         X.println(">>>");
         X.println(">>> Checkpoint manager memory stats [grid=" + ctx.gridName() + ']');
-        X.println(">>>  keyMap: " + keyMap.size());
+        X.println(">>>  keyMap: " + (keyMap != null ? keyMap.size() : 0));
     }
 
     /**
@@ -407,6 +442,9 @@ public class GridCheckpointManager extends GridManagerAdapter<CheckpointSpi> {
             if (log.isDebugEnabled())
                 log.debug("Received checkpoint request: " + req);
 
+            if (!enabled())
+                return;
+
             IgniteUuid sesId = req.getSessionId();
 
             if (closedSess.contains(sesId)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index c877d57..4382731 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -1211,6 +1211,11 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     public void addUserMessageListener(@Nullable final Object topic, @Nullable final IgniteBiPredicate<UUID, ?> p) {
         if (p != null) {
             try {
+                if (p instanceof GridLifecycleAwareMessageFilter)
+                    ((GridLifecycleAwareMessageFilter)p).initialize(ctx);
+                else
+                    ctx.resource().injectGeneric(p);
+
                 addMessageListener(TOPIC_COMM_USER,
                     new GridUserMessageListener(topic, (IgniteBiPredicate<UUID, Object>)p));
             }
@@ -1695,13 +1700,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             throws IgniteCheckedException {
             this.topic = topic;
             this.predLsnr = predLsnr;
-
-            if (predLsnr != null) {
-                if (predLsnr instanceof GridLifecycleAwareMessageFilter)
-                    ((GridLifecycleAwareMessageFilter)predLsnr).initialize(ctx);
-                else
-                    ctx.resource().injectGeneric(predLsnr);
-            }
         }
 
         /** {@inheritDoc} */
@@ -1724,69 +1722,84 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 return;
             }
 
-            Object msgBody = ioMsg.body();
-
-            assert msgBody != null || ioMsg.bodyBytes() != null;
+            busyLock.readLock();
 
             try {
-                byte[] msgTopicBytes = ioMsg.topicBytes();
-
-                Object msgTopic = ioMsg.topic();
-
-                GridDeployment dep = ioMsg.deployment();
-
-                if (dep == null && ctx.config().isPeerClassLoadingEnabled() &&
-                    ioMsg.deploymentClassName() != null) {
-                    dep = ctx.deploy().getGlobalDeployment(
-                        ioMsg.deploymentMode(),
-                        ioMsg.deploymentClassName(),
-                        ioMsg.deploymentClassName(),
-                        ioMsg.userVersion(),
-                        nodeId,
-                        ioMsg.classLoaderId(),
-                        ioMsg.loaderParticipants(),
-                        null);
-
-                    if (dep == null)
-                        throw new IgniteDeploymentCheckedException(
-                            "Failed to obtain deployment information for user message. " +
-                            "If you are using custom message or topic class, try implementing " +
-                            "GridPeerDeployAware interface. [msg=" + ioMsg + ']');
-
-                    ioMsg.deployment(dep); // Cache deployment.
+                if (stopping) {
+                    if (log.isDebugEnabled())
+                        log.debug("Received user message while stopping (will ignore) [nodeId=" +
+                            nodeId + ", msg=" + msg + ']');
+
+                    return;
                 }
 
-                // Unmarshall message topic if needed.
-                if (msgTopic == null && msgTopicBytes != null) {
-                    msgTopic = marsh.unmarshal(msgTopicBytes, dep != null ? dep.classLoader() : null);
+                Object msgBody = ioMsg.body();
 
-                    ioMsg.topic(msgTopic); // Save topic to avoid future unmarshallings.
-                }
+                assert msgBody != null || ioMsg.bodyBytes() != null;
 
-                if (!F.eq(topic, msgTopic))
-                    return;
+                try {
+                    byte[] msgTopicBytes = ioMsg.topicBytes();
+
+                    Object msgTopic = ioMsg.topic();
+
+                    GridDeployment dep = ioMsg.deployment();
+
+                    if (dep == null && ctx.config().isPeerClassLoadingEnabled() &&
+                        ioMsg.deploymentClassName() != null) {
+                        dep = ctx.deploy().getGlobalDeployment(
+                            ioMsg.deploymentMode(),
+                            ioMsg.deploymentClassName(),
+                            ioMsg.deploymentClassName(),
+                            ioMsg.userVersion(),
+                            nodeId,
+                            ioMsg.classLoaderId(),
+                            ioMsg.loaderParticipants(),
+                            null);
+
+                        if (dep == null)
+                            throw new IgniteDeploymentCheckedException(
+                                "Failed to obtain deployment information for user message. " +
+                                    "If you are using custom message or topic class, try implementing " +
+                                    "GridPeerDeployAware interface. [msg=" + ioMsg + ']');
+
+                        ioMsg.deployment(dep); // Cache deployment.
+                    }
 
-                if (msgBody == null) {
-                    msgBody = marsh.unmarshal(ioMsg.bodyBytes(), dep != null ? dep.classLoader() : null);
+                    // Unmarshall message topic if needed.
+                    if (msgTopic == null && msgTopicBytes != null) {
+                        msgTopic = marsh.unmarshal(msgTopicBytes, dep != null ? dep.classLoader() : null);
 
-                    ioMsg.body(msgBody); // Save body to avoid future unmarshallings.
-                }
+                        ioMsg.topic(msgTopic); // Save topic to avoid future unmarshallings.
+                    }
 
-                // Resource injection.
-                if (dep != null)
-                    ctx.resource().inject(dep, dep.deployedClass(ioMsg.deploymentClassName()), msgBody);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to unmarshal user message [node=" + nodeId + ", message=" +
-                    msg + ']', e);
-            }
+                    if (!F.eq(topic, msgTopic))
+                        return;
+
+                    if (msgBody == null) {
+                        msgBody = marsh.unmarshal(ioMsg.bodyBytes(), dep != null ? dep.classLoader() : null);
 
-            if (msgBody != null) {
-                if (predLsnr != null) {
-                    if (!predLsnr.apply(nodeId, msgBody))
-                        removeMessageListener(TOPIC_COMM_USER, this);
+                        ioMsg.body(msgBody); // Save body to avoid future unmarshallings.
+                    }
+
+                    // Resource injection.
+                    if (dep != null)
+                        ctx.resource().inject(dep, dep.deployedClass(ioMsg.deploymentClassName()), msgBody);
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to unmarshal user message [node=" + nodeId + ", message=" +
+                        msg + ']', e);
+                }
+
+                if (msgBody != null) {
+                    if (predLsnr != null) {
+                        if (!predLsnr.apply(nodeId, msgBody))
+                            removeMessageListener(TOPIC_COMM_USER, this);
+                    }
                 }
             }
+            finally {
+                busyLock.readUnlock();
+            }
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java
new file mode 100644
index 0000000..2005d4e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.discovery;
+
+import org.apache.ignite.cluster.*;
+
+/**
+ * Listener interface.
+ */
+public interface CustomEventListener<T extends DiscoveryCustomMessage> {
+    /**
+     * @param snd Sender.
+     * @param msg Message.
+     */
+    public void onCustomEvent(ClusterNode snd, T msg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
new file mode 100644
index 0000000..23f8bda
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomMessageWrapper.java
@@ -0,0 +1,63 @@
+/*
+ * 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.managers.discovery;
+
+import org.apache.ignite.spi.discovery.*;
+import org.jetbrains.annotations.*;
+
+/**
+ *
+ */
+class CustomMessageWrapper implements DiscoverySpiCustomMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final DiscoveryCustomMessage delegate;
+
+    /**
+     * @param delegate Delegate.
+     */
+    CustomMessageWrapper(DiscoveryCustomMessage delegate) {
+        this.delegate = delegate;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoverySpiCustomMessage ackMessage() {
+        DiscoveryCustomMessage res = delegate.ackMessage();
+
+        return res == null ? null : new CustomMessageWrapper(res);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return delegate.isMutable();
+    }
+
+    /**
+     * @return Delegate.
+     */
+    public DiscoveryCustomMessage delegate() {
+        return delegate;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return delegate.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
new file mode 100644
index 0000000..401486d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java
@@ -0,0 +1,54 @@
+/*
+ * 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.managers.discovery;
+
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ *
+ */
+public interface DiscoveryCustomMessage extends Serializable {
+    /**
+     * @return Unique custom message ID.
+     */
+    public IgniteUuid id();
+
+    /**
+     * Whether or not minor version of topology should be increased on message receive.
+     *
+     * @return {@code true} if minor topology version should be increased.
+     * @see AffinityTopologyVersion#minorTopVer
+     */
+    public boolean incrementMinorTopologyVersion();
+
+    /**
+     * Called when custom message has been handled by all nodes.
+     *
+     * @return Ack message or {@code null} if ack is not required.
+     */
+    @Nullable public DiscoveryCustomMessage ackMessage();
+
+    /**
+     * @return {@code true} if message can be modified during listener notification. Changes will be send to next nodes.
+     */
+    public boolean isMutable();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 0950774..71fbc61 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.jobmetrics.*;
 import org.apache.ignite.internal.processors.security.*;
+import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -165,10 +166,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     private final GridLocalMetrics metrics = createMetrics();
 
     /** Metrics update worker. */
-    private final MetricsUpdater metricsUpdater = new MetricsUpdater();
+    private GridTimeoutProcessor.CancelableTask metricsUpdateTask;
 
     /** Custom event listener. */
-    private GridPlainInClosure<Serializable> customEvtLsnr;
+    private ConcurrentMap<Class<?>, List<CustomEventListener<DiscoveryCustomMessage>>> customEvtLsnrs =
+        new ConcurrentHashMap8<>();
 
     /** Map of dynamic cache filters. */
     private Map<String, CachePredicate> registeredCaches = new HashMap<>();
@@ -176,6 +178,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
+    /** Received custom messages history. */
+    private final ArrayDeque<IgniteUuid> rcvdCustomMsgs = new ArrayDeque<>();
+
     /** @param ctx Context. */
     public GridDiscoveryManager(GridKernalContext ctx) {
         super(ctx, ctx.config().getDiscoverySpi());
@@ -214,6 +219,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param filter Cache filter.
+     * @param nearEnabled Near enabled flag.
      * @param loc {@code True} if cache is local.
      */
     public void setCacheFilter(
@@ -240,12 +246,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      *
      * @param cacheName Cache name.
      * @param clientNodeId Near node ID.
+     * @param nearEnabled Near enabled flag.
      */
     public void addClientNode(String cacheName, UUID clientNodeId, boolean nearEnabled) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        if (predicate != null)
-            predicate.addClientNode(clientNodeId, nearEnabled);
+        if (pred != null)
+            pred.addClientNode(clientNodeId, nearEnabled);
     }
 
     /**
@@ -279,17 +286,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
     }
 
-    /**
-     * @param evtType Event type.
-     * @return Next affinity topology version.
-     */
-    private AffinityTopologyVersion nextTopologyVersion(int evtType, long topVer) {
-        if (evtType == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT)
-            minorTopVer++;
-        else if (evtType != EVT_NODE_METRICS_UPDATED)
-            minorTopVer = 0;
-
-        return new AffinityTopologyVersion(topVer, minorTopVer);
+    /** {@inheritDoc} */
+    @Override protected void onKernalStart0() throws IgniteCheckedException {
+        if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
+            ctx.performance().add("Enable client mode for TcpDiscoverySpi " +
+                    "(set TcpDiscoverySpi.forceServerMode to false)");
     }
 
     /** {@inheritDoc} */
@@ -328,7 +329,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             checkSegmentOnStart();
         }
 
-        new IgniteThread(metricsUpdater).start();
+        metricsUpdateTask = ctx.timeout().schedule(new MetricsUpdater(), METRICS_UPDATE_FREQ, METRICS_UPDATE_FREQ);
 
         spi.setMetricsProvider(createMetricsProvider());
 
@@ -356,14 +357,41 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 ClusterNode node,
                 Collection<ClusterNode> topSnapshot,
                 Map<Long, Collection<ClusterNode>> snapshots,
-                @Nullable Serializable data
+                @Nullable DiscoverySpiCustomMessage spiCustomMsg
             ) {
+                DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
+                    : ((CustomMessageWrapper)spiCustomMsg).delegate();
+
+                if (skipMessage(type, customMsg))
+                    return;
+
                 final ClusterNode locNode = localNode();
 
                 if (snapshots != null)
                     topHist = snapshots;
 
-                AffinityTopologyVersion nextTopVer = nextTopologyVersion(type, topVer);
+                boolean verChanged;
+
+                if (type == EVT_NODE_METRICS_UPDATED)
+                    verChanged = false;
+                else if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+                    assert customMsg != null;
+
+                    if (customMsg.incrementMinorTopologyVersion()) {
+                        minorTopVer++;
+
+                        verChanged = true;
+                    }
+                    else
+                        verChanged = false;
+                }
+                else {
+                    minorTopVer = 0;
+
+                    verChanged = true;
+                }
+
+                AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
 
                 if (type == EVT_NODE_FAILED || type == EVT_NODE_LEFT) {
                     for (DiscoCache c : discoCacheHist.values())
@@ -373,19 +401,26 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 }
 
                 if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
-                    try {
-                        if (customEvtLsnr != null)
-                            customEvtLsnr.apply(data);
-                    }
-                    catch (Exception e) {
-                        U.error(log, "Failed to notify direct custom event listener: " + data, e);
+                    for (Class cls = customMsg.getClass(); cls != null; cls = cls.getSuperclass()) {
+                        List<CustomEventListener<DiscoveryCustomMessage>> list = customEvtLsnrs.get(cls);
+
+                        if (list != null) {
+                            for (CustomEventListener<DiscoveryCustomMessage> lsnr : list) {
+                                try {
+                                    lsnr.onCustomEvent(node, customMsg);
+                                }
+                                catch (Exception e) {
+                                    U.error(log, "Failed to notify direct custom event listener: " + customMsg, e);
+                                }
+                            }
+                        }
                     }
                 }
 
                 // Put topology snapshot into discovery history.
                 // There is no race possible between history maintenance and concurrent discovery
                 // event notifications, since SPI notifies manager about all events from this listener.
-                if (type != EVT_NODE_METRICS_UPDATED) {
+                if (verChanged) {
                     DiscoCache cache = new DiscoCache(locNode, F.view(topSnapshot, F.remoteNodes(locNode.id())));
 
                     discoCacheHist.put(nextTopVer, cache);
@@ -417,7 +452,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     return;
                 }
 
-                discoWrk.addEvent(type, nextTopVer, node, topSnapshot, data);
+                discoWrk.addEvent(type, nextTopVer, node, topSnapshot, customMsg);
             }
         });
 
@@ -486,10 +521,43 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
-     * @param customEvtLsnr Custom event listener.
+     * @param type Message type.
+     * @param customMsg Custom message.
+     * @return {@code True} if should not process message.
      */
-    public void setCustomEventListener(GridPlainInClosure<Serializable> customEvtLsnr) {
-        this.customEvtLsnr = customEvtLsnr;
+    private boolean skipMessage(int type, @Nullable DiscoveryCustomMessage customMsg) {
+        if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) {
+            assert customMsg != null && customMsg.id() != null : customMsg;
+
+            if (rcvdCustomMsgs.contains(customMsg.id())) {
+                if (log.isDebugEnabled())
+                    log.debug("Received duplicated custom message, will ignore [msg=" + customMsg + "]");
+
+                return true;
+            }
+
+            rcvdCustomMsgs.addLast(customMsg.id());
+
+            while (rcvdCustomMsgs.size() > DISCOVERY_HISTORY_SIZE)
+                rcvdCustomMsgs.pollFirst();
+        }
+
+        return false;
+    }
+
+    /**
+     * @param msgCls Message class.
+     * @param lsnr Custom event listener.
+     */
+    public <T extends DiscoveryCustomMessage> void setCustomEventListener(Class<T> msgCls, CustomEventListener<T> lsnr) {
+        List<CustomEventListener<DiscoveryCustomMessage>> list = customEvtLsnrs.get(msgCls);
+
+        if (list == null) {
+            list = F.addIfAbsent(customEvtLsnrs, msgCls,
+                new CopyOnWriteArrayList<CustomEventListener<DiscoveryCustomMessage>>());
+        }
+
+        list.add((CustomEventListener<DiscoveryCustomMessage>)lsnr);
     }
 
     /**
@@ -660,7 +728,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 Map<Integer, CacheMetrics> metrics = null;
 
                 for (GridCacheAdapter<?, ?> cache : caches) {
-                    if (cache.context().started() && cache.configuration().isStatisticsEnabled()) {
+                    if (cache.configuration().isStatisticsEnabled() &&
+                        cache.context().started() &&
+                        cache.context().affinity().affinityTopologyVersion().topologyVersion() > 0) {
                         if (metrics == null)
                             metrics = U.newHashMap(caches.size());
 
@@ -952,11 +1022,11 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         getSpi().setListener(null);
 
         // Stop discovery worker and metrics updater.
+        U.closeQuiet(metricsUpdateTask);
+
         U.cancel(discoWrk);
-        U.cancel(metricsUpdater);
 
         U.join(discoWrk, log);
-        U.join(metricsUpdater, log);
 
         // Stop SPI itself.
         stopSpi();
@@ -1218,13 +1288,23 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
-     * Gets alive remote nodes with at least one cache configured.
+     * Gets alive remote server nodes with at least one cache configured.
      *
      * @param topVer Topology version (maximum allowed node order).
      * @return Collection of alive cache nodes.
      */
-    public Collection<ClusterNode> aliveRemoteNodesWithCaches(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).aliveRemoteNodesWithCaches(topVer.topologyVersion());
+    public Collection<ClusterNode> aliveRemoteServerNodesWithCaches(AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(null, topVer).aliveRemoteServerNodesWithCaches(topVer.topologyVersion());
+    }
+
+    /**
+     * Gets alive server nodes with at least one cache configured.
+     *
+     * @param topVer Topology version (maximum allowed node order).
+     * @return Collection of alive cache nodes.
+     */
+    public Collection<ClusterNode> aliveServerNodesWithCaches(AffinityTopologyVersion topVer) {
+        return resolveDiscoCache(null, topVer).aliveServerNodesWithCaches(topVer.topologyVersion());
     }
 
     /**
@@ -1256,9 +1336,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node is a cache data node.
      */
     public boolean cacheAffinityNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.dataNode(node);
+        return pred != null && pred.dataNode(node);
     }
 
     /**
@@ -1267,9 +1347,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has near cache enabled.
      */
     public boolean cacheNearNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.nearNode(node);
+        return pred != null && pred.nearNode(node);
     }
 
     /**
@@ -1278,9 +1358,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return {@code True} if node has client cache (without near cache).
      */
     public boolean cacheClientNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.clientNode(node);
+        return pred != null && pred.clientNode(node);
     }
 
     /**
@@ -1289,9 +1369,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return If cache with the given name is accessible on the given node.
      */
     public boolean cacheNode(ClusterNode node, String cacheName) {
-        CachePredicate predicate = registeredCaches.get(cacheName);
+        CachePredicate pred = registeredCaches.get(cacheName);
 
-        return predicate != null && predicate.cacheNode(node);
+        return pred != null && pred.cacheNode(node);
     }
 
     /**
@@ -1384,10 +1464,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
-     * @param evt Event.
+     * @param msg Custom message.
      */
-    public void sendCustomEvent(Serializable evt) {
-        getSpi().sendCustomEvent(evt);
+    public void sendCustomEvent(DiscoveryCustomMessage msg) {
+        getSpi().sendCustomEvent(new CustomMessageWrapper(msg));
     }
 
     /**
@@ -1542,8 +1622,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** Worker for discovery events. */
     private class DiscoveryWorker extends GridWorker {
         /** Event queue. */
-        private final BlockingQueue<GridTuple5<Integer, AffinityTopologyVersion, ClusterNode, Collection<ClusterNode>, Serializable>> evts =
-            new LinkedBlockingQueue<>();
+        private final BlockingQueue<GridTuple5<Integer, AffinityTopologyVersion, ClusterNode, Collection<ClusterNode>,
+            DiscoveryCustomMessage>> evts = new LinkedBlockingQueue<>();
 
         /** Node segmented event fired flag. */
         private boolean nodeSegFired;
@@ -1609,9 +1689,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             AffinityTopologyVersion topVer,
             ClusterNode node,
             Collection<ClusterNode> topSnapshot,
-            @Nullable Serializable data
+            @Nullable DiscoveryCustomMessage data
         ) {
-            assert node != null;
+            assert node != null : data;
 
             evts.add(F.t(type, topVer, node, topSnapshot, data));
         }
@@ -1650,7 +1730,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         /** @throws InterruptedException If interrupted. */
         @SuppressWarnings("DuplicateCondition")
         private void body0() throws InterruptedException {
-            GridTuple5<Integer, AffinityTopologyVersion, ClusterNode, Collection<ClusterNode>, Serializable> evt = evts.take();
+            GridTuple5<Integer, AffinityTopologyVersion, ClusterNode, Collection<ClusterNode>,
+                DiscoveryCustomMessage> evt = evts.take();
 
             int type = evt.get1();
 
@@ -1768,7 +1849,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         customEvt.type(type);
                         customEvt.topologySnapshot(topVer.topologyVersion(), null);
                         customEvt.affinityTopologyVersion(topVer);
-                        customEvt.data(evt.get5());
+                        customEvt.customMessage(evt.get5());
 
                         ctx.event().record(customEvt);
                     }
@@ -1833,28 +1914,17 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      *
      */
-    private class MetricsUpdater extends GridWorker {
+    private class MetricsUpdater implements Runnable {
         /** */
         private long prevGcTime = -1;
 
         /** */
         private long prevCpuTime = -1;
 
-        /**
-         *
-         */
-        private MetricsUpdater() {
-            super(ctx.gridName(), "metrics-updater", GridDiscoveryManager.this.log);
-        }
-
         /** {@inheritDoc} */
-        @Override protected void body() throws IgniteInterruptedCheckedException {
-            while (!isCancelled()) {
-                U.sleep(METRICS_UPDATE_FREQ);
-
-                gcCpuLoad = getGcCpuLoad();
-                cpuLoad = getCpuLoad();
-            }
+        @Override public void run() {
+            gcCpuLoad = getGcCpuLoad();
+            cpuLoad = getCpuLoad();
         }
 
         /**
@@ -2065,9 +2135,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         private final Collection<ClusterNode> aliveNodesWithCaches;
 
         /**
-         * Cached alive remote nodes with caches.
+         * Cached alive server remote nodes with caches.
+         */
+        private final Collection<ClusterNode> aliveSrvNodesWithCaches;
+
+        /**
+         * Cached alive remote server nodes with caches.
          */
-        private final Collection<ClusterNode> aliveRmtNodesWithCaches;
+        private final Collection<ClusterNode> aliveRmtSrvNodesWithCaches;
 
         /**
          * @param loc Local node.
@@ -2088,21 +2163,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             all.addAll(rmtNodes);
 
+            Collections.sort(all, GridNodeOrderComparator.INSTANCE);
+
             allNodes = Collections.unmodifiableList(all);
 
-            Map<String, Collection<ClusterNode>> cacheMap =
-                new HashMap<>(allNodes.size(), 1.0f);
-            Map<String, Collection<ClusterNode>> rmtCacheMap =
-                new HashMap<>(allNodes.size(), 1.0f);
-            Map<String, Collection<ClusterNode>> dhtNodesMap =
-                new HashMap<>(allNodes.size(), 1.0f);
+            Map<String, Collection<ClusterNode>> cacheMap = new HashMap<>(allNodes.size(), 1.0f);
+            Map<String, Collection<ClusterNode>> rmtCacheMap = new HashMap<>(allNodes.size(), 1.0f);
+            Map<String, Collection<ClusterNode>> dhtNodesMap =new HashMap<>(allNodes.size(), 1.0f);
             Collection<ClusterNode> nodesWithCaches = new HashSet<>(allNodes.size());
             Collection<ClusterNode> rmtNodesWithCaches = new HashSet<>(allNodes.size());
 
             aliveCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
             aliveRmtCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
             aliveNodesWithCaches = new ConcurrentSkipListSet<>();
-            aliveRmtNodesWithCaches = new ConcurrentSkipListSet<>();
+            aliveSrvNodesWithCaches = new ConcurrentSkipListSet<>();
+            aliveRmtSrvNodesWithCaches = new ConcurrentSkipListSet<>();
             nodesByVer = new TreeMap<>();
 
             long maxOrder0 = 0;
@@ -2154,8 +2229,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     if (alive(node.id())) {
                         aliveNodesWithCaches.add(node);
 
-                        if (!loc.id().equals(node.id()))
-                            aliveRmtNodesWithCaches.add(node);
+                        if (!CU.clientNode(node)) {
+                            aliveSrvNodesWithCaches.add(node);
+
+                            if (!loc.id().equals(node.id()))
+                                aliveRmtSrvNodesWithCaches.add(node);
+                        }
                     }
                 }
 
@@ -2240,13 +2319,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * @return All nodes with at least one cache configured.
-         */
-        Collection<ClusterNode> allNodesWithCaches() {
-            return allNodesWithCaches;
-        }
-
-        /**
          * Gets collection of nodes which have version equal or greater than {@code ver}.
          *
          * @param ver Version to check.
@@ -2345,13 +2417,23 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * Gets all alive remote nodes with at least one cache configured.
+         * Gets all alive remote server nodes with at least one cache configured.
          *
          * @param topVer Topology version.
          * @return Collection of nodes.
          */
-        Collection<ClusterNode> aliveRemoteNodesWithCaches(final long topVer) {
-            return filter(topVer, aliveRmtNodesWithCaches);
+        Collection<ClusterNode> aliveRemoteServerNodesWithCaches(final long topVer) {
+            return filter(topVer, aliveRmtSrvNodesWithCaches);
+        }
+
+        /**
+         * Gets all alive server nodes with at least one cache configured.
+         *
+         * @param topVer Topology version.
+         * @return Collection of nodes.
+         */
+        Collection<ClusterNode> aliveServerNodesWithCaches(final long topVer) {
+            return filter(topVer, aliveSrvNodesWithCaches);
         }
 
         /**
@@ -2388,7 +2470,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             filterNodeMap(aliveRmtCacheNodes, leftNode);
 
             aliveNodesWithCaches.remove(leftNode);
-            aliveRmtNodesWithCaches.remove(leftNode);
+            aliveSrvNodesWithCaches.remove(leftNode);
+            aliveRmtSrvNodesWithCaches.remove(leftNode);
         }
 
         /**
@@ -2480,11 +2563,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         private boolean loc;
 
         /** Collection of client near nodes. */
-        private Map<UUID, Boolean> clientNodes;
+        private ConcurrentHashMap<UUID, Boolean> clientNodes;
 
         /**
          * @param cacheFilter Cache filter.
          * @param nearEnabled Near enabled flag.
+         * @param loc {@code True} if cache is local.
          */
         private CachePredicate(IgnitePredicate<ClusterNode> cacheFilter, boolean nearEnabled, boolean loc) {
             assert cacheFilter != null;
@@ -2498,9 +2582,10 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         /**
          * @param nodeId Near node ID to add.
+         * @param nearEnabled Near enabled flag.
          */
         public void addClientNode(UUID nodeId, boolean nearEnabled) {
-            clientNodes.put(nodeId, nearEnabled);
+            clientNodes.putIfAbsent(nodeId, nearEnabled);
         }
 
         /**
@@ -2515,7 +2600,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if this node is a data node for given cache.
          */
         public boolean dataNode(ClusterNode node) {
-            return !node.isDaemon() && cacheFilter.apply(node);
+            return !node.isDaemon() && CU.affinityNode(node, cacheFilter);
         }
 
         /**
@@ -2523,8 +2608,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @return {@code True} if cache is accessible on the given node.
          */
         public boolean cacheNode(ClusterNode node) {
-            return !node.isClient() && !node.isDaemon() &&
-                (cacheFilter.apply(node) || clientNodes.containsKey(node.id()));
+            return !node.isDaemon() && (CU.affinityNode(node, cacheFilter) || clientNodes.containsKey(node.id()));
         }
 
         /**
@@ -2535,8 +2619,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             if (node.isDaemon())
                 return false;
 
-            if (nearEnabled && cacheFilter.apply(node))
-                return true;
+            if (CU.affinityNode(node, cacheFilter))
+                return nearEnabled;
 
             Boolean near = clientNodes.get(node.id());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
index 9a81cd1..f1561bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/indexing/GridIndexingManager.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.indexing.*;
 
@@ -46,9 +45,6 @@ public class GridIndexingManager extends GridManagerAdapter<IndexingSpi> {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @Override public void start() throws IgniteCheckedException {
-        if (!enabled())
-            U.warn(log, "Indexing is disabled (to enable please configure GridIndexingSpi).");
-
         startSpi();
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
index e9df8b8..5373e46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java
@@ -68,6 +68,18 @@ class GridAffinityAssignment implements Serializable {
     }
 
     /**
+     * @param topVer Topology version.
+     * @param aff Assignment to copy from.
+     */
+    GridAffinityAssignment(AffinityTopologyVersion topVer, GridAffinityAssignment aff) {
+        this.topVer = topVer;
+
+        assignment = aff.assignment;
+        primary = aff.primary;
+        backup = aff.backup;
+    }
+
+    /**
      * @return Affinity assignment.
      */
     public List<List<ClusterNode>> assignment() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index eccd9f9..c46490e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -32,6 +32,8 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
+
 /**
  * Affinity cached function.
  */
@@ -221,6 +223,35 @@ public class GridAffinityAssignmentCache {
     }
 
     /**
+     * Copies previous affinity assignment when discovery event does not cause affinity assignment changes
+     * (e.g. client node joins on leaves).
+     *
+     * @param evt Event.
+     * @param topVer Topology version.
+     */
+    public void clientEventTopologyChange(DiscoveryEvent evt, AffinityTopologyVersion topVer) {
+        GridAffinityAssignment aff = head.get();
+
+        assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT  || aff.primaryPartitions(evt.eventNode().id()).isEmpty() : evt;
+        assert evt.type() == EVT_DISCOVERY_CUSTOM_EVT  || aff.backupPartitions(evt.eventNode().id()).isEmpty() : evt;
+
+        GridAffinityAssignment assignmentCpy = new GridAffinityAssignment(topVer, aff);
+
+        affCache.put(topVer, assignmentCpy);
+        head.set(assignmentCpy);
+
+        for (Map.Entry<AffinityTopologyVersion, AffinityReadyFuture> entry : readyFuts.entrySet()) {
+            if (entry.getKey().compareTo(topVer) <= 0) {
+                if (log.isDebugEnabled())
+                    log.debug("Completing topology ready future (use previous affinity) " +
+                        "[locNodeId=" + ctx.localNodeId() + ", futVer=" + entry.getKey() + ", topVer=" + topVer + ']');
+
+                entry.getValue().onDone(topVer);
+            }
+        }
+    }
+
+    /**
      * @return Last calculated affinity version.
      */
     public AffinityTopologyVersion lastVersion() {
@@ -422,6 +453,7 @@ public class GridAffinityAssignmentCache {
 
         /**
          *
+         * @param reqTopVer Required topology version.
          */
         private AffinityReadyFuture(AffinityTopologyVersion reqTopVer) {
             this.reqTopVer = reqTopVer;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index daa2bc2..aac63c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
@@ -164,14 +164,17 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
      *
      * @param cacheName Cache name.
      * @param key Key to map.
+     * @param topVer Topology version.
      * @return Affinity nodes, primary first.
      * @throws IgniteCheckedException If failed.
      */
-    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName, K key) throws IgniteCheckedException {
+    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName,
+        K key,
+        AffinityTopologyVersion topVer)
+        throws IgniteCheckedException
+    {
         A.notNull(key, "key");
 
-        AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
-
         AffinityInfo affInfo = affinityCache(cacheName, topVer);
 
         if (affInfo == null)
@@ -181,6 +184,20 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Map single key to primary and backup nodes.
+     *
+     * @param cacheName Cache name.
+     * @param key Key to map.
+     * @return Affinity nodes, primary first.
+     * @throws IgniteCheckedException If failed.
+     */
+    public <K> List<ClusterNode> mapKeyToPrimaryAndBackups(@Nullable String cacheName, K key)
+        throws IgniteCheckedException
+    {
+        return mapKeyToPrimaryAndBackups(cacheName, key, ctx.discovery().topologyVersionEx());
+    }
+
+    /**
      * Gets affinity key for cache key.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
index 5d6062e..7a3fbee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheEvictableEntryImpl.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -91,6 +92,36 @@ public class CacheEvictableEntryImpl<K, V> implements EvictableEntry<K, V> {
     }
 
     /** {@inheritDoc} */
+    public int size() {
+        try {
+            GridCacheContext<Object, Object> cctx = cached.context();
+
+            KeyCacheObject key = cached.key();
+
+            byte[] keyBytes = key.valueBytes(cctx.cacheObjectContext());
+
+            byte[] valBytes = null;
+
+            if (cctx.useOffheapEntry())
+                valBytes = cctx.offheap().get(cctx.swap().spaceName(), cached.partition(), key, keyBytes);
+            else {
+                CacheObject cacheObj = cached.valueBytes();
+
+                if (cacheObj != null)
+                    valBytes = cacheObj.valueBytes(cctx.cacheObjectContext());
+            }
+
+            return valBytes == null ? keyBytes.length : keyBytes.length + valBytes.length;
+        }
+        catch (GridCacheEntryRemovedException e) {
+            return 0;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public V getValue() {
         try {


[27/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
index bf8cf0d..221bc39 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicy.java
@@ -18,18 +18,28 @@
 package org.apache.ignite.cache.eviction.fifo;
 
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jsr166.*;
 import org.jsr166.ConcurrentLinkedDeque8.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
  * Eviction policy based on {@code First In First Out (FIFO)} algorithm and supports batch eviction.
  * <p>
- * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size.
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
  * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
  * <p>
  * This implementation is very efficient since it does not create any additional
@@ -41,11 +51,17 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /** Batch size. */
     private volatile int batchSize = 1;
 
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
+
     /** FIFO queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
         new ConcurrentLinkedDeque8<>();
@@ -63,7 +79,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public FifoEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -75,7 +91,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param batchSize Batch size.
      */
     public FifoEvictionPolicy(int max, int batchSize) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
         A.ensure(batchSize > 0, "batchSize > 0");
 
         this.max = max;
@@ -97,7 +113,7 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -119,6 +135,23 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         return queue.size();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemorySize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemorySize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemorySize() {
+        return memSize.longValue();
+    }
+
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -141,8 +174,11 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
         else {
             Node<EvictableEntry<K, V>> node = entry.removeMeta();
 
-            if (node != null)
+            if (node != null) {
                 queue.unlinkx(node);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
@@ -173,11 +209,18 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
 
@@ -189,38 +232,74 @@ public class FifoEvictionPolicy<K, V> implements EvictionPolicy<K, V>, FifoEvict
      * Shrinks FIFO queue to maximum allowed size.
      */
     private void shrink() {
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
+
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
+
+                    if (size == -1)
+                        break;
+
+                    i += size;
+                }
+        }
+
         int max = this.max;
 
-        int batchSize = this.batchSize;
+        if (max > 0) {
+            int startSize = queue.sizex();
+
+            // Shrink only if queue is full.
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && queue.sizex() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
 
-        int startSize = queue.sizex();
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        EvictableEntry<K, V> entry = queue.poll();
 
-        // Shrink only if queue is full.
-        if (startSize >= max + batchSize) {
-            for (int i = max; i < startSize && queue.sizex() > max; i++) {
-                EvictableEntry<K, V> entry = queue.poll();
+        if (entry == null)
+            return -1;
 
-                if (entry == null)
-                    break;
+        int size = 0;
 
-                Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+        Node<EvictableEntry<K, V>> meta = entry.removeMeta();
 
-                if (meta != null && !entry.evict())
-                    touch(entry);
-            }
+        if (meta != null) {
+            size = entry.size();
+
+            memSize.add(-size);
+
+            if (!entry.evict())
+                touch(entry);
         }
+
+        return size;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
         out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
         batchSize = in.readInt();
+        maxMemSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
index 63a413e..793aa66 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/fifo/FifoEvictionPolicyMBean.java
@@ -63,4 +63,26 @@ public interface FifoEvictionPolicyMBean {
      */
     @MXBeanDescription("Current FIFO queue size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemorySize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemorySize(long maxMemSize);
+
+    /**
+     * Gets current queue size in bytes.
+     *
+     * @return current queue size in bytes.
+     */
+    @MXBeanDescription("Current FIFO queue size in bytes.")
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
index 309d577..0be26c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicy.java
@@ -18,26 +18,48 @@
 package org.apache.ignite.cache.eviction.lru;
 
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+
 import org.jsr166.*;
 import org.jsr166.ConcurrentLinkedDeque8.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
- * Eviction policy based on {@code Least Recently Used (LRU)} algorithm. This
- * implementation is very efficient since it is lock-free and does not
- * create any additional table-like data structures. The {@code LRU} ordering
- * information is maintained by attaching ordering metadata to cache entries.
+ * Eviction policy based on {@code Least Recently Used (LRU)} algorithm and supports batch eviction.
+ * <p>
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
+ * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
+
+ * This implementation is very efficient since it is lock-free and does not create any additional table-like
+ * data structures. The {@code LRU} ordering information is maintained by attaching ordering metadata to cache entries.
  */
 public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictionPolicyMBean, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
+    private volatile int max = DFLT_CACHE_SIZE;
+
+    /** Batch size. */
+    private volatile int batchSize = 1;
+
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
 
     /** Queue. */
     private final ConcurrentLinkedDeque8<EvictableEntry<K, V>> queue =
@@ -56,7 +78,7 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     public LruEvictionPolicy(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -76,16 +98,45 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
 
     /** {@inheritDoc} */
+    @Override public int getBatchSize() {
+        return batchSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setBatchSize(int batchSize) {
+        A.ensure(batchSize > 0, "batchSize > 0");
+
+        this.batchSize = batchSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getCurrentSize() {
         return queue.size();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemorySize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemorySize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemorySize() {
+        return memSize.longValue();
+    }
+
     /**
      * Gets read-only view on internal {@code FIFO} queue in proper order.
      *
@@ -107,8 +158,11 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
         else {
             Node<EvictableEntry<K, V>> node = entry.removeMeta();
 
-            if (node != null)
+            if (node != null) {
                 queue.unlinkx(node);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
@@ -139,11 +193,18 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If node was unlinked by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(node))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
         else if (queue.unlinkx(node)) {
@@ -163,31 +224,73 @@ public class LruEvictionPolicy<K, V> implements EvictionPolicy<K, V>, LruEvictio
      * Shrinks queue to maximum allowed size.
      */
     private void shrink() {
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
+
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
+
+                    if (size == -1)
+                        break;
+
+                    i += size;
+                }
+        }
+
         int max = this.max;
 
-        int startSize = queue.sizex();
+        if (max > 0) {
+            int startSize = queue.sizex();
 
-        for (int i = 0; i < startSize && queue.sizex() > max; i++) {
-            EvictableEntry<K, V> entry = queue.poll();
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize))
+                for (int i = max; i < startSize && queue.sizex() > max; i++)
+                    if (shrink0() == -1)
+                        break;
+        }
+    }
 
-            if (entry == null)
-                break;
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        EvictableEntry<K, V> entry = queue.poll();
 
-            Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+        if (entry == null)
+            return -1;
 
-            if (meta != null && !entry.evict())
+        int size = 0;
+
+        Node<EvictableEntry<K, V>> meta = entry.removeMeta();
+
+        if (meta != null) {
+            size = entry.size();
+
+            memSize.add(-size);
+
+            if (!entry.evict())
                 touch(entry);
         }
+
+        return size;
     }
 
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
+        out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
     }
 
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
+        batchSize = in.readInt();
+        maxMemSize = in.readLong();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
index c243374..e17c057 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/lru/LruEvictionPolicyMBean.java
@@ -41,10 +41,48 @@ public interface LruEvictionPolicyMBean {
     public void setMaxSize(int max);
 
     /**
+     * Gets batch size.
+     *
+     * @return batch size.
+     */
+    @MXBeanDescription("Batch size.")
+    public int getBatchSize();
+
+    /**
+     * Sets batch size.
+     *
+     * @param batchSize Batch size.
+     */
+    @MXBeanDescription("Set batch size.")
+    public void setBatchSize(int batchSize);
+
+    /**
      * Gets current queue size.
      *
      * @return Current queue size.
      */
     @MXBeanDescription("Current queue size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemorySize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemorySize(long maxMemSize);
+
+    /**
+     * Gets current queue size in bytes.
+     *
+     * @return current queue size in bytes.
+     */
+    @MXBeanDescription("Current queue size in  bytes.")
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
index c88b31d..00a912f 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/random/RandomEvictionPolicy.java
@@ -18,20 +18,22 @@
 package org.apache.ignite.cache.eviction.random;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import javax.cache.*;
 import java.io.*;
 
+import static org.apache.ignite.configuration.CacheConfiguration.*;
+
 /**
  * Cache eviction policy which will select random cache entry for eviction if cache
  * size exceeds the {@link #getMaxSize()} parameter. This implementation is
  * extremely light weight, lock-free, and does not create any data structures to maintain
  * any order for eviction.
  * <p>
- * Random eviction will provide the best performance over any key set in which every
+ * Random eviction will provide the best performance over any key queue in which every
  * key has the same probability of being accessed.
  */
 public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomEvictionPolicyMBean, Externalizable {
@@ -39,7 +41,7 @@ public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomE
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max = CacheConfiguration.DFLT_CACHE_SIZE;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /**
      * Constructs random eviction policy with all defaults.
@@ -87,7 +89,7 @@ public class RandomEvictionPolicy<K, V> implements EvictionPolicy<K, V>, RandomE
 
         IgniteCache<K, V> cache = entry.unwrap(IgniteCache.class);
 
-        int size = cache.size();
+        int size = cache.localSize(CachePeekMode.ONHEAP);
 
         for (int i = max; i < size; i++) {
             Cache.Entry<K, V> e = cache.randomEntry();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
index 7965c97..b8b82fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicy.java
@@ -34,7 +34,15 @@ import static org.apache.ignite.configuration.CacheConfiguration.*;
 /**
  * Cache eviction policy which will select the minimum cache entry for eviction.
  * <p>
- * The eviction starts when the cache size becomes {@code batchSize} elements greater than the maximum size.
+ * The eviction starts in the following cases:
+ * <ul>
+ *     <li>The cache size becomes {@code batchSize} elements greater than the maximum size.</li>
+ *     <li>
+ *         The size of cache entries in bytes becomes greater than the maximum memory size.
+ *         The size of cache entry calculates as sum of key size and value size.
+ *     </li>
+ * </ul>
+ * <b>Note:</b>Batch eviction is enabled only if maximum memory limit isn't set ({@code maxMemSize == 0}).
  * {@code batchSize} elements will be evicted in this case. The default {@code batchSize} value is {@code 1}.
  * <p>
  * Entries comparison based on {@link Comparator} instance if provided.
@@ -48,18 +56,24 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     private static final long serialVersionUID = 0L;
 
     /** Maximum size. */
-    private volatile int max;
+    private volatile int max = DFLT_CACHE_SIZE;
 
     /** Batch size. */
     private volatile int batchSize = 1;
 
+    /** Max memory size. */
+    private volatile long maxMemSize;
+
+    /** Memory size. */
+    private final LongAdder8 memSize = new LongAdder8();
+
     /** Comparator. */
     private Comparator<Holder<K, V>> comp;
 
     /** Order. */
     private final AtomicLong orderCnt = new AtomicLong();
 
-    /** Backed sorted set. */
+    /** Backed sorted queue. */
     private final GridConcurrentSkipListSetEx<K, V> set;
 
     /**
@@ -96,7 +110,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      * @param comp Entries comparator.
      */
     public SortedEvictionPolicy(int max, int batchSize, @Nullable Comparator<EvictableEntry<K, V>> comp) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
         A.ensure(batchSize > 0, "batchSize > 0");
 
         this.max = max;
@@ -106,6 +120,16 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /**
+     * Constructs sorted eviction policy with given maximum size and given entry comparator.
+     *
+     * @param comp Entries comparator.
+     */
+    public SortedEvictionPolicy(@Nullable Comparator<EvictableEntry<K, V>> comp) {
+        this.comp = comp == null ? new DefaultHolderComparator<K, V>() : new HolderComparator<>(comp);
+        this.set = new GridConcurrentSkipListSetEx<>(this.comp);
+    }
+
+    /**
      * Gets maximum allowed size of cache before entry will start getting evicted.
      *
      * @return Maximum allowed size of cache before entry will start getting evicted.
@@ -120,7 +144,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
      * @param max Maximum allowed size of cache before entry will start getting evicted.
      */
     @Override public void setMaxSize(int max) {
-        A.ensure(max > 0, "max > 0");
+        A.ensure(max >= 0, "max >= 0");
 
         this.max = max;
     }
@@ -142,12 +166,29 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         return set.sizex();
     }
 
+    /** {@inheritDoc} */
+    @Override public long getMaxMemorySize() {
+        return maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setMaxMemorySize(long maxMemSize) {
+        A.ensure(maxMemSize >= 0, "maxMemSize >= 0");
+
+        this.maxMemSize = maxMemSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCurrentMemorySize() {
+        return memSize.longValue();
+    }
+
     /**
-     * Gets read-only view of backed set in proper order.
+     * Gets read-only view of backed queue in proper order.
      *
-     * @return Read-only view of backed set.
+     * @return Read-only view of backed queue.
      */
-    public Collection<EvictableEntry<K, V>> set() {
+    public Collection<EvictableEntry<K, V>> queue() {
         Set<EvictableEntry<K, V>> cp = new LinkedHashSet<>();
 
         for (Holder<K, V> holder : set)
@@ -168,19 +209,22 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
         else {
             Holder<K, V> holder = entry.removeMeta();
 
-            if (holder != null)
+            if (holder != null) {
                 removeHolder(holder);
+
+                memSize.add(-entry.size());
+            }
         }
     }
 
     /**
      * @param entry Entry to touch.
-     * @return {@code True} if backed set has been changed by this call.
+     * @return {@code True} if backed queue has been changed by this call.
      */
     private boolean touch(EvictableEntry<K, V> entry) {
         Holder<K, V> holder = entry.meta();
 
-        // Entry has not been add yet to backed set..
+        // Entry has not been add yet to backed queue..
         if (holder == null) {
             while (true) {
                 holder = new Holder<>(entry, orderCnt.incrementAndGet());
@@ -188,7 +232,7 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 set.add(holder);
 
                 if (entry.putMetaIfAbsent(holder) != null) {
-                    // Was concurrently added, need to remove it from set.
+                    // Was concurrently added, need to remove it from queue.
                     removeHolder(holder);
 
                     // Set has not been changed.
@@ -196,17 +240,24 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
                 }
                 else if (holder.order > 0) {
                     if (!entry.isCached()) {
-                        // Was concurrently evicted, need to remove it from set.
+                        // Was concurrently evicted, need to remove it from queue.
                         removeHolder(holder);
 
                         return false;
                     }
 
+                    memSize.add(entry.size());
+
                     return true;
                 }
                 // If holder was removed by concurrent shrink() call, we must repeat the whole cycle.
                 else if (!entry.removeMeta(holder))
                     return false;
+                else {
+                    memSize.add(-entry.size());
+
+                    return true;
+                }
             }
         }
 
@@ -215,34 +266,71 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     }
 
     /**
-     * Shrinks backed set to maximum allowed size.
+     * Shrinks backed queue to maximum allowed size.
      */
     private void shrink() {
-        int max = this.max;
+        long maxMem = this.maxMemSize;
+
+        if (maxMem > 0) {
+            long startMemSize = memSize.longValue();
 
-        int batchSize = this.batchSize;
+            if (startMemSize >= maxMem)
+                for (long i = maxMem; i < startMemSize && memSize.longValue() > maxMem;) {
+                    int size = shrink0();
 
-        int startSize = set.sizex();
+                    if (size == -1)
+                        break;
 
-        if (startSize >= max + batchSize) {
-            for (int i = max; i < startSize && set.sizex() > max; i++) {
-                Holder<K, V> h = set.pollFirst();
+                    i += size;
+                }
+        }
 
-                if (h == null)
-                    break;
+        int max = this.max;
 
-                EvictableEntry<K, V> entry = h.entry;
+        if (max > 0) {
+            int startSize = set.sizex();
 
-                if (h.order > 0 && entry.removeMeta(h) && !entry.evict())
-                    touch(entry);
+            if (startSize >= max + (maxMem > 0 ? 1 : this.batchSize)) {
+                for (int i = max; i < startSize && set.sizex() > max; i++) {
+                    if (shrink0() == -1)
+                        break;
+                }
             }
         }
     }
 
+    /**
+     * Tries to remove one item from queue.
+     *
+     * @return number of bytes that was free. {@code -1} if queue is empty.
+     */
+    private int shrink0() {
+        Holder<K, V> h = set.pollFirst();
+
+        if (h == null)
+            return -1;
+
+        int size = 0;
+
+        EvictableEntry<K, V> entry = h.entry;
+
+        if (h.order > 0 && entry.removeMeta(h)) {
+            size = entry.size();
+
+            memSize.add(-size);
+
+            if (!entry.evict())
+                touch(entry);
+        }
+
+        return size;
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         out.writeInt(max);
         out.writeInt(batchSize);
+        out.writeLong(maxMemSize);
         out.writeObject(comp);
     }
 
@@ -251,11 +339,12 @@ public class SortedEvictionPolicy<K, V> implements EvictionPolicy<K, V>, SortedE
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         max = in.readInt();
         batchSize = in.readInt();
+        maxMemSize = in.readLong();
         comp = (Comparator<Holder<K, V>>)in.readObject();
     }
 
     /**
-     * Removes holder from backed set and marks holder as removed.
+     * Removes holder from backed queue and marks holder as removed.
      *
      * @param holder Holder.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
index bc696ff..7283453 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/sorted/SortedEvictionPolicyMBean.java
@@ -63,4 +63,26 @@ public interface SortedEvictionPolicyMBean {
      */
     @MXBeanDescription("Current sorted key set size.")
     public int getCurrentSize();
+
+    /**
+     * Gets maximum allowed cache size in bytes.
+     *
+     * @return maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Maximum allowed cache size in bytes.")
+    public long getMaxMemorySize();
+
+    /**
+     * Sets maximum allowed cache size in bytes.
+     */
+    @MXBeanDescription("Set maximum allowed cache size in bytes.")
+    public void setMaxMemorySize(long maxMemSize);
+
+    /**
+     * Gets current sorted entries queue size in bytes.
+     *
+     * @return current sorted entries queue size in bytes.
+     */
+    @MXBeanDescription("Current sorted entries set size in bytes.")
+    public long getCurrentMemorySize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
index e66b32d..ef8fc49 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/QueryMetrics.java
@@ -20,9 +20,9 @@ package org.apache.ignite.cache.query;
 import org.apache.ignite.internal.processors.cache.query.*;
 
 /**
- * Cache query metrics used to obtain statistics on query. You can get metrics for
- * particular query via {@link CacheQuery#metrics()} method or accumulated metrics
- * for all queries via {@link GridCacheQueryManager#metrics()}.
+ * Cache query metrics used to obtain statistics on query. Metrics for particular query
+ * can be get via {@link CacheQuery#metrics()} method or aggregated metrics for all queries
+ * via {@link CacheQuery#metrics()}.
  */
 public interface QueryMetrics {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
index d018298..5bfdda1 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStore.java
@@ -94,6 +94,8 @@ public interface CacheStore<K, V> extends CacheLoader<K, V>, CacheWriter<K, V> {
      * @throws CacheWriterException If commit or rollback failed. Note that commit failure in some cases
      *      may bring cache transaction into {@link TransactionState#UNKNOWN} which will
      *      consequently cause all transacted entries to be invalidated.
+     * @deprecated Use {@link CacheStoreSessionListener} instead (refer to its JavaDoc for details).
      */
+    @Deprecated
     public void sessionEnd(boolean commit) throws CacheWriterException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
index 640d4a3..329e994 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSession.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache.store;
 
 import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -52,6 +53,27 @@ public interface CacheStoreSession {
     public boolean isWithinTransaction();
 
     /**
+     * Attaches the given object to this session.
+     * <p>
+     * An attached object may later be retrieved via the {@link #attachment()}
+     * method. Invoking this method causes any previous attachment to be
+     * discarded. To attach additional objects use {@link #properties()} map.
+     * <p>
+     * The current attachment may be discarded by attaching {@code null}.
+     *
+     * @param attachment The object to be attached (or {@code null} to discard current attachment).
+     * @return Previously attached object, if any.
+     */
+    @Nullable public <T> T attach(@Nullable Object attachment);
+
+    /**
+     * Retrieves the current attachment or {@code null} if there is no attachment.
+     *
+     * @return Currently attached object, if any.
+     */
+    @Nullable public <T> T attachment();
+
+    /**
      * Gets current session properties. You can add properties directly to the
      * returned map.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.java
new file mode 100644
index 0000000..1543bf9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheStoreSessionListener.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.cache.store;
+
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.configuration.*;
+
+import javax.cache.configuration.*;
+import javax.sql.*;
+
+/**
+ * Cache store session listener that allows to implement callbacks
+ * for session lifecycle.
+ * <p>
+ * The most common use case for session listeners is database
+ * connection and transaction management. Store can be invoked one
+ * or several times during one session, depending on whether it's
+ * executed within cache transaction or not. In any case, you have
+ * to create a connection when session is started and commit it or
+ * rollback when session is finished.
+ * <p>
+ * Cache store session listener allows to implement this and other
+ * scenarios providing to callback methods:
+ * <ul>
+ *     <li>
+ *         {@link #onSessionStart(CacheStoreSession)} - called
+ *         before any store operation within a session is invoked.
+ *     </li>
+ *     <li>
+ *         {@link #onSessionEnd(CacheStoreSession, boolean)} - called
+ *         after all operations within a session are invoked.
+ *     </li>
+ * </ul>
+ * <h2>Implementations</h2>
+ * Ignites provides several out-of-the-box implementations
+ * of session listener (refer to individual JavaDocs for more
+ * details):
+ * <ul>
+ *     <li>
+ *         {@link CacheJdbcStoreSessionListener} - JDBC-based session
+ *         listener. For each session it gets a new JDBC connection from
+ *         provided {@link DataSource} and commits (or rolls back) it
+ *         when session ends.
+ *     </li>
+ *     <li>
+ *         {@ignitelink org.apache.ignite.cache.store.spring.CacheSpringStoreSessionListener} -
+ *         session listener based on Spring transaction management.
+ *         It starts a new DB transaction for each session and commits
+ *         (or rolls back) it when session ends. If there is no ongoing
+ *         cache transaction, this listener is no-op.
+ *     </li>
+ *     <li>
+ *         {@ignitelink org.apache.ignite.cache.store.hibernate.CacheHibernateStoreSessionListener} -
+ *         Hibernate-based session listener. It creates a new Hibernate
+ *         session for each Ignite session. If there is an ongoing cache
+ *         transaction, a corresponding Hibernate transaction is created
+ *         as well.
+ *     </li>
+ * </ul>
+ * <h2>Configuration</h2>
+ * There are two ways to configure a session listener:
+ * <ul>
+ *     <li>
+ *         Provide a global listener for all caches via
+ *         {@link IgniteConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+ *         configuration property. This will we called for any store
+ *         session, not depending on what caches participate in
+ *         transaction.
+ *     </li>
+ *     <li>
+ *         Provide a listener for a particular cache via
+ *         {@link CacheConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+ *         configuration property. This will be called only if the
+ *         cache participates in transaction.
+ *     </li>
+ * </ul>
+ * For example, here is how global {@link CacheJdbcStoreSessionListener}
+ * can be configured in Spring XML configuration file:
+ * <pre name="code" class="xml">
+ * &lt;bean class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
+ *     ...
+ *
+ *     &lt;property name="CacheStoreSessionListenerFactories"&gt;
+ *         &lt;list&gt;
+ *             &lt;bean class="javax.cache.configuration.FactoryBuilder$SingletonFactory"&gt;
+ *                 &lt;constructor-arg&gt;
+ *                     &lt;bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcStoreSessionListener"&gt;
+ *                         &lt;!-- Inject external data source. --&gt;
+ *                         &lt;property name="dataSource" ref="jdbc-data-source"/&gt;
+ *                     &lt;/bean&gt;
+ *                 &lt;/constructor-arg&gt;
+ *             &lt;/bean&gt;
+ *         &lt;/list&gt;
+ *     &lt;/property&gt;
+ * &lt;/bean&gt;
+ * </pre>
+ */
+public interface CacheStoreSessionListener {
+    /**
+     * On session start callback.
+     * <p>
+     * Called before any store operation within a session is invoked.
+     *
+     * @param ses Current session.
+     */
+    public void onSessionStart(CacheStoreSession ses);
+
+    /**
+     * On session end callback.
+     * <p>
+     * Called after all operations within a session are invoked.
+     *
+     * @param ses Current session.
+     * @param commit {@code True} if persistence store transaction
+     *      should commit, {@code false} for rollback.
+     */
+    public void onSessionEnd(CacheStoreSession ses, boolean commit);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
new file mode 100644
index 0000000..a20e535
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListener.java
@@ -0,0 +1,141 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+
+/**
+ * Cache store session listener based on JDBC connection.
+ * <p>
+ * For each session this listener gets a new JDBC connection
+ * from provided {@link DataSource} and commits (or rolls
+ * back) it when session ends.
+ * <p>
+ * The connection is saved as a store session
+  * {@link CacheStoreSession#attachment() attachment}.
+ * The listener guarantees that the connection will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will be committed or rolled back only when
+ * session ends.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(Cache.Entry)}
+ * method can be implemented if {@link CacheJdbcStoreSessionListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get connection from the current session.
+ *         Connection conn = ses.attachment();
+ *
+ *         // Execute update SQL query.
+ *         try {
+ *             conn.createStatement().executeUpdate("...");
+ *         }
+ *         catch (SQLException e) {
+ *             throw new CacheWriterException("Failed to update the store.", e);
+ *         }
+ *     }
+ * }
+ * </pre>
+ * JDBC connection will be automatically created by the listener
+ * at the start of the session and closed when it ends.
+ */
+public class CacheJdbcStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Data source. */
+    private DataSource dataSrc;
+
+    /**
+     * Sets data source.
+     * <p>
+     * This is a required parameter. If data source is not set,
+     * exception will be thrown on startup.
+     *
+     * @param dataSrc Data source.
+     */
+    public void setDataSource(DataSource dataSrc) {
+        this.dataSrc = dataSrc;
+    }
+
+    /**
+     * Gets data source.
+     *
+     * @return Data source.
+     */
+    public DataSource getDataSource() {
+        return dataSrc;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void start() throws IgniteException {
+        if (dataSrc == null)
+            throw new IgniteException("Data source is required by " + getClass().getSimpleName() + '.');
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.attachment() == null) {
+            try {
+                Connection conn = dataSrc.getConnection();
+
+                conn.setAutoCommit(false);
+
+                ses.attach(conn);
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Connection conn = ses.attach(null);
+
+        if (conn != null) {
+            try {
+                if (commit)
+                    conn.commit();
+                else
+                    conn.rollback();
+            }
+            catch (SQLException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                U.closeQuiet(conn);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
index 9cb5d3d..85fd08a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.cluster;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -33,7 +35,7 @@ import java.util.*;
  * You can use cluster node attributes to provide static information about a node.
  * This information is initialized once within a cluster, during the node startup, and
  * remains the same throughout the lifetime of a node. Use
- * {@link org.apache.ignite.configuration.IgniteConfiguration#getUserAttributes()} method to initialize your custom
+ * {@link IgniteConfiguration#getUserAttributes()} method to initialize your custom
  * node attributes at startup. Here is an example of how to assign an attribute to a node at startup:
  * <pre name="code" class="xml">
  * &lt;bean class="org.apache.ignite.configuration.IgniteConfiguration">
@@ -114,7 +116,7 @@ public interface ClusterNode {
 
     /**
      * Gets a node attribute. Attributes are assigned to nodes at startup
-     * via {@link org.apache.ignite.configuration.IgniteConfiguration#getUserAttributes()} method.
+     * via {@link IgniteConfiguration#getUserAttributes()} method.
      * <p>
      * The system adds the following attributes automatically:
      * <ul>
@@ -149,7 +151,7 @@ public interface ClusterNode {
 
     /**
      * Gets all node attributes. Attributes are assigned to nodes at startup
-     * via {@link org.apache.ignite.configuration.IgniteConfiguration#getUserAttributes()} method.
+     * via {@link IgniteConfiguration#getUserAttributes()} method.
      * <p>
      * The system adds the following attributes automatically:
      * <ul>
@@ -167,7 +169,7 @@ public interface ClusterNode {
     /**
      * Gets collection of addresses this node is known by.
      * <p>
-     * If {@link org.apache.ignite.configuration.IgniteConfiguration#getLocalHost()} value isn't {@code null} node will try to use that
+     * If {@link IgniteConfiguration#getLocalHost()} value isn't {@code null} node will try to use that
      * address for all communications and returned collection will contain only that address.
      * If it is {@code null} then local wildcard address will be used, and Ignite
      * will make the best effort to supply all addresses of that node in returned collection.
@@ -179,12 +181,12 @@ public interface ClusterNode {
     /**
      * Gets collection of host names this node is known by.
      * <p>
-     * If {@link org.apache.ignite.configuration.IgniteConfiguration#getLocalHost()} value isn't {@code null} node will try to use
+     * If {@link IgniteConfiguration#getLocalHost()} value isn't {@code null} node will try to use
      * the host name of that resolved address for all communications and
      * returned collection will contain only that host name.
      * If that host name can not be resolved then ip address returned by method {@link #addresses()} is used.
      * <p>
-     * If {@link org.apache.ignite.configuration.IgniteConfiguration#getLocalHost()} value is {@code null} then local wildcard address will be used,
+     * If {@link IgniteConfiguration#getLocalHost()} value is {@code null} then local wildcard address will be used,
      * and this method returns host names of all addresses of that node.
      *
      * @return Collection of host names.
@@ -238,9 +240,17 @@ public interface ClusterNode {
     public boolean isDaemon();
 
     /**
-     * Tests whether or not this node is a client node.
+     * Tests whether or not this node is connected to cluster as a client.
+     * <p>
+     * Do not confuse client in terms of
+     * discovery {@link DiscoverySpi#isClientMode()} and client in terms of cache
+     * {@link IgniteConfiguration#isClientMode()}. Cache clients cannot carry data,
+     * while topology clients connect to topology in a different way.
      *
      * @return {@code True} if this node is a client node, {@code false} otherwise.
+     * @see IgniteConfiguration#isClientMode()
+     * @see Ignition#isClientMode()
+     * @see DiscoverySpi#isClientMode()
      */
     public boolean isClient();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 df6b2ee..1aa4fd6 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
@@ -25,7 +25,6 @@ import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.query.annotations.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;
@@ -145,9 +144,6 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Default value for 'readFromBackup' flag. */
     public static final boolean DFLT_READ_FROM_BACKUP = true;
 
-    /** Filter that accepts only server nodes. */
-    public static final IgnitePredicate<ClusterNode> SERVER_NODES = new IgniteServerNodePredicate();
-
     /** Filter that accepts all nodes. */
     public static final IgnitePredicate<ClusterNode> ALL_NODES = new IgniteAllNodesPredicate();
 
@@ -316,6 +312,9 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
     /** Cache topology validator. */
     private TopologyValidator topValidator;
 
+    /** Cache store session listeners. */
+    private Factory<? extends CacheStoreSessionListener>[] storeSesLsnrs;
+
     /** Empty constructor (all values are initialized to their defaults). */
     public CacheConfiguration() {
         /* No-op. */
@@ -389,6 +388,7 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         sqlOnheapRowCacheSize = cc.getSqlOnheapRowCacheSize();
         startSize = cc.getStartSize();
         storeFactory = cc.getCacheStoreFactory();
+        storeSesLsnrs = cc.getCacheStoreSessionListenerFactories();
         swapEnabled = cc.isSwapEnabled();
         tmLookupClsName = cc.getTransactionManagerLookupClassName();
         topValidator = cc.getTopologyValidator();
@@ -1664,7 +1664,18 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         A.ensure(indexedTypes == null || (indexedTypes.length & 1) == 0,
             "Number of indexed types is expected to be even. Refer to method javadoc for details.");
 
-        this.indexedTypes = indexedTypes;
+        if (indexedTypes != null) {
+            int len = indexedTypes.length;
+
+            Class<?>[] newIndexedTypes = new Class<?>[len];
+
+            for (int i = 0; i < len; i++)
+                newIndexedTypes[i] = U.box(indexedTypes[i]);
+
+            this.indexedTypes = newIndexedTypes;
+        }
+        else
+            this.indexedTypes = null;
 
         return this;
     }
@@ -1734,30 +1745,37 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(CacheConfiguration.class, this);
+    /**
+     * Gets cache store session listener factories.
+     *
+     * @return Cache store session listener factories.
+     * @see CacheStoreSessionListener
+     */
+    public Factory<? extends CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
+        return storeSesLsnrs;
     }
 
     /**
-     * Filter that accepts only server nodes.
+     * Cache store session listener factories.
+     * <p>
+     * These listeners override global listeners provided in
+     * {@link IgniteConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+     * configuration property.
+     *
+     * @param storeSesLsnrs Cache store session listener factories.
+     * @return {@code this} for chaining.
+     * @see CacheStoreSessionListener
      */
-    public static class IgniteServerNodePredicate implements IgnitePredicate<ClusterNode> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        @Override public boolean apply(ClusterNode n) {
-            Boolean attr = n.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
-
-            return attr != null && !attr;
-        }
+    public CacheConfiguration setCacheStoreSessionListenerFactories(
+        Factory<? extends CacheStoreSessionListener>... storeSesLsnrs) {
+        this.storeSesLsnrs = storeSesLsnrs;
 
-        @Override public boolean equals(Object obj) {
-            if (obj == null)
-                return false;
+        return this;
+    }
 
-            return obj.getClass().equals(this.getClass());
-        }
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheConfiguration.class, this);
     }
 
     /**
@@ -1767,10 +1785,12 @@ public class CacheConfiguration<K, V> extends MutableConfiguration<K, V> {
         /** */
         private static final long serialVersionUID = 0L;
 
+        /** {@inheritDoc} */
         @Override public boolean apply(ClusterNode clusterNode) {
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override public boolean equals(Object obj) {
             if (obj == null)
                 return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index ebe2b8e..2d36c7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.configuration;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.events.*;
@@ -52,6 +53,7 @@ import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
 
+import javax.cache.configuration.*;
 import javax.cache.event.*;
 import javax.cache.expiry.*;
 import javax.cache.integration.*;
@@ -334,9 +336,6 @@ public class IgniteConfiguration {
     /** Cache configurations. */
     private CacheConfiguration[] cacheCfg;
 
-    /** Client cache configurations. */
-    private NearCacheConfiguration[] nearCacheCfg;
-
     /** Client mode flag. */
     private Boolean clientMode;
 
@@ -398,6 +397,9 @@ public class IgniteConfiguration {
     /** User's class loader. */
     private ClassLoader classLdr;
 
+    /** Cache store session listeners. */
+    private Factory<CacheStoreSessionListener>[] storeSesLsnrs;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -478,6 +480,7 @@ public class IgniteConfiguration {
         segResolvers = cfg.getSegmentationResolvers();
         sndRetryCnt = cfg.getNetworkSendRetryCount();
         sndRetryDelay = cfg.getNetworkSendRetryDelay();
+        storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
         svcCfgs = cfg.getServiceConfiguration();
         sysPoolSize = cfg.getSystemThreadPoolSize();
         timeSrvPortBase = cfg.getTimeServerPortBase();
@@ -1823,9 +1826,11 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Gets client mode flag.
+     * Gets client mode flag. Client node cannot hold data in the caches. It's recommended to use
+     * {@link DiscoverySpi} in client mode if this property is {@code true}.
      *
      * @return Client mode flag.
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public Boolean isClientMode() {
         return clientMode;
@@ -2188,15 +2193,21 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Gets plugin configurations.
+     *
      * @return Plugin configurations.
+     * @see PluginProvider
      */
     public PluginConfiguration[] getPluginConfigurations() {
         return pluginCfgs;
     }
 
     /**
+     * Sets plugin configurations.
+     *
      * @param pluginCfgs Plugin configurations.
      * @return {@code this} for chaining.
+     * @see PluginProvider
      */
     public IgniteConfiguration setPluginConfigurations(PluginConfiguration... pluginCfgs) {
         this.pluginCfgs = pluginCfgs;
@@ -2242,6 +2253,35 @@ public class IgniteConfiguration {
         return classLdr;
     }
 
+    /**
+     * Gets cache store session listener factories.
+     *
+     * @return Cache store session listener factories.
+     * @see CacheStoreSessionListener
+     */
+    public Factory<CacheStoreSessionListener>[] getCacheStoreSessionListenerFactories() {
+        return storeSesLsnrs;
+    }
+
+    /**
+     * Cache store session listener factories.
+     * <p>
+     * These are global store session listeners, so they are applied to
+     * all caches. If you need to override listeners for a
+     * particular cache, use {@link CacheConfiguration#setCacheStoreSessionListenerFactories(Factory[])}
+     * configuration property.
+     *
+     * @param storeSesLsnrs Cache store session listener factories.
+     * @return {@code this} for chaining.
+     * @see CacheStoreSessionListener
+     */
+    public IgniteConfiguration setCacheStoreSessionListenerFactories(
+        Factory<CacheStoreSessionListener>... storeSesLsnrs) {
+        this.storeSesLsnrs = storeSesLsnrs;
+
+        return this;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteConfiguration.class, this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
new file mode 100644
index 0000000..5a65bdb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsUserContext.java
@@ -0,0 +1,119 @@
+/*
+ * 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.igfs;
+
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.concurrent.*;
+
+/**
+ * Provides ability to execute IGFS code in a context of a specific user.
+ */
+public abstract class IgfsUserContext {
+    /** Thread local to hold the current user context. */
+    private static final ThreadLocal<String> userStackThreadLocal = new ThreadLocal<>();
+
+    /**
+     * Executes given callable in the given user context.
+     * The main contract of this method is that {@link #currentUser()} method invoked
+     * inside closure always returns 'user' this callable executed with.
+     * @param user the user name to invoke closure on behalf of.
+     * @param clo the closure to execute
+     * @param <T> The type of closure result.
+     * @return the result of closure execution.
+     * @throws IllegalArgumentException if user name is null or empty String or if the closure is null.
+     */
+    public static <T> T doAs(String user, final IgniteOutClosure<T> clo) {
+        if (F.isEmpty(user))
+            throw new IllegalArgumentException("Failed to use null or empty user name.");
+
+        final String ctxUser = userStackThreadLocal.get();
+
+        if (F.eq(ctxUser, user))
+            return clo.apply(); // correct context is already there
+
+        userStackThreadLocal.set(user);
+
+        try {
+            return clo.apply();
+        }
+        finally {
+            userStackThreadLocal.set(ctxUser);
+        }
+    }
+
+    /**
+     * Same contract that {@link #doAs(String, IgniteOutClosure)} has, but accepts
+     * callable that throws checked Exception.
+     * The Exception is not ever wrapped anyhow.
+     * If your Callable throws Some specific checked Exceptions, the recommended usage pattern is:
+     * <pre name="code" class="java">
+     *  public Foo myOperation() throws MyCheckedException1, MyCheckedException2 {
+     *      try {
+     *          return IgfsUserContext.doAs(user, new Callable<Foo>() {
+     *              &#64;Override public Foo call() throws MyCheckedException1, MyCheckedException2 {
+     *                  return makeSomeFoo(); // do the job
+     *              }
+     *          });
+     *      }
+     *      catch (MyCheckedException1 | MyCheckedException2 | RuntimeException | Error e) {
+     *          throw e;
+     *      }
+     *      catch (Exception e) {
+     *          throw new AssertionError("Must never go there.");
+     *      }
+     *  }
+     * </pre>
+     * @param user the user name to invoke closure on behalf of.
+     * @param clbl the Callable to execute
+     * @param <T> The type of callable result.
+     * @return the result of closure execution.
+     * @throws IllegalArgumentException if user name is null or empty String or if the closure is null.
+     */
+    public static <T> T doAs(String user, final Callable<T> clbl) throws Exception {
+        if (F.isEmpty(user))
+            throw new IllegalArgumentException("Failed to use null or empty user name.");
+
+        final String ctxUser = userStackThreadLocal.get();
+
+        if (F.eq(ctxUser, user))
+            return clbl.call(); // correct context is already there
+
+        userStackThreadLocal.set(user);
+
+        try {
+            return clbl.call();
+        }
+        finally {
+            userStackThreadLocal.set(ctxUser);
+        }
+    }
+
+    /**
+     * Gets the current context user.
+     * If this method is invoked outside of any {@link #doAs(String, IgniteOutClosure)} on the call stack, it will
+     * return null. Otherwise it will return the user name set in the most lower
+     * {@link #doAs(String, IgniteOutClosure)} call on the call stack.
+     * @return The current user, may be null.
+     */
+    @Nullable public static String currentUser() {
+        return userStackThreadLocal.get();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
index 9026eac..cb69352 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/secondary/IgfsSecondaryFileSystem.java
@@ -198,4 +198,11 @@ public interface IgfsSecondaryFileSystem {
      * @return Map of properties.
      */
     public Map<String,String> properties();
+
+
+    /**
+     * Closes the secondary file system.
+     * @throws IgniteException in case of an error.
+     */
+    public void close() throws IgniteException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsSnapshot.java
index 4d5d146..6da45ea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterMetricsSnapshot.java
@@ -1247,6 +1247,20 @@ public class ClusterMetricsSnapshot implements ClusterMetrics {
     /**
      * Serializes node metrics into byte array.
      *
+     * @param metrics Node metrics to serialize.
+     * @return New offset.
+     */
+    public static byte[] serialize(ClusterMetrics metrics) {
+        byte[] buf = new byte[METRICS_SIZE];
+
+        serialize(buf, 0, metrics);
+
+        return buf;
+    }
+
+    /**
+     * Serializes node metrics into byte array.
+     *
      * @param data Byte array.
      * @param off Offset into byte array.
      * @param metrics Node metrics to serialize.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 505204d..f33fa39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.interop.*;
 import org.apache.ignite.internal.managers.deployment.*;
-import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.continuous.*;
@@ -131,41 +131,91 @@ class GridEventConsumeHandler implements GridContinuousHandler {
         final boolean loc = nodeId.equals(ctx.localNodeId());
 
         lsnr = new GridLocalEventListener() {
+            /** node ID, routine ID, event */
+            private final Queue<T3<UUID, UUID, Event>> notificationQueue = new LinkedList<>();
+
+            private boolean notificationInProgress;
+
             @Override public void onEvent(Event evt) {
-                if (filter == null || filter.apply(evt)) {
-                    if (loc) {
-                        if (!cb.apply(nodeId, evt))
-                            ctx.continuous().stopRoutine(routineId);
-                    }
-                    else {
-                        GridDiscoveryManager disco = ctx.discovery();
+                if (filter != null && !filter.apply(evt))
+                    return;
+
+                if (loc) {
+                    if (!cb.apply(nodeId, evt))
+                        ctx.continuous().stopRoutine(routineId);
+                }
+                else {
+                    if (ctx.discovery().node(nodeId) == null)
+                        return;
+
+                    synchronized (notificationQueue) {
+                        notificationQueue.add(new T3<>(nodeId, routineId, evt));
+
+                        if (!notificationInProgress) {
+                            ctx.getSystemExecutorService().submit(new Runnable() {
+                                @Override public void run() {
+                                    if (!ctx.continuous().lockStopping())
+                                        return;
 
-                        ClusterNode node = disco.node(nodeId);
+                                    try {
+                                        while (true) {
+                                            T3<UUID, UUID, Event> t3;
 
-                        if (node != null) {
-                            try {
-                                EventWrapper wrapper = new EventWrapper(evt);
+                                            synchronized (notificationQueue) {
+                                                t3 = notificationQueue.poll();
 
-                                if (evt instanceof CacheEvent) {
-                                    String cacheName = ((CacheEvent)evt).cacheName();
+                                                if (t3 == null) {
+                                                    notificationInProgress = false;
 
-                                    if (ctx.config().isPeerClassLoadingEnabled() && disco.cacheNode(node, cacheName)) {
-                                        wrapper.p2pMarshal(ctx.config().getMarshaller());
+                                                    return;
+                                                }
+                                            }
 
-                                        wrapper.cacheName = cacheName;
+                                            try {
+                                                Event evt = t3.get3();
 
-                                        GridCacheDeploymentManager depMgr =
-                                            ctx.cache().internalCache(cacheName).context().deploy();
+                                                EventWrapper wrapper = new EventWrapper(evt);
 
-                                        depMgr.prepare(wrapper);
+                                                if (evt instanceof CacheEvent) {
+                                                    String cacheName = ((CacheEvent)evt).cacheName();
+
+                                                    ClusterNode node = ctx.discovery().node(t3.get1());
+
+                                                    if (node == null)
+                                                        continue;
+
+                                                    if (ctx.config().isPeerClassLoadingEnabled()
+                                                        && ctx.discovery().cacheNode(node, cacheName)) {
+                                                        wrapper.p2pMarshal(ctx.config().getMarshaller());
+
+                                                        wrapper.cacheName = cacheName;
+
+                                                        GridCacheDeploymentManager depMgr = ctx.cache()
+                                                            .internalCache(cacheName).context().deploy();
+
+                                                        depMgr.prepare(wrapper);
+                                                    }
+                                                }
+
+                                                ctx.continuous().addNotification(t3.get1(), t3.get2(), wrapper, null, false,
+                                                    false);
+                                            }
+                                            catch (ClusterTopologyCheckedException ignored) {
+                                                // No-op.
+                                            }
+                                            catch (Throwable e) {
+                                                U.error(ctx.log(GridEventConsumeHandler.class),
+                                                    "Failed to send event notification to node: " + nodeId, e);
+                                            }
+                                        }
+                                    }
+                                    finally {
+                                        ctx.continuous().unlockStopping();
                                     }
                                 }
+                            });
 
-                                ctx.continuous().addNotification(nodeId, routineId, wrapper, null, false, false);
-                            }
-                            catch (IgniteCheckedException e) {
-                                U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, e);
-                            }
+                            notificationInProgress = true;
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index ad7d562..d6542f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -552,4 +552,9 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      * @return Marshaller context.
      */
     public MarshallerContextImpl marshallerContext();
+
+    /**
+     * @return {@code True} if local node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
+     */
+    public boolean clientNode();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 1ff483e..f921d49 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -894,6 +894,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public boolean clientNode() {
+        return cfg.isClientMode();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridKernalContextImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index c4b93b8..4f5e365 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.processors.datastructures.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.job.*;
 import org.apache.ignite.internal.processors.jobmetrics.*;
+import org.apache.ignite.internal.processors.nodevalidation.*;
 import org.apache.ignite.internal.processors.offheap.*;
 import org.apache.ignite.internal.processors.plugin.*;
 import org.apache.ignite.internal.processors.port.*;
@@ -56,7 +57,6 @@ import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.processors.segmentation.*;
 import org.apache.ignite.internal.processors.service.*;
 import org.apache.ignite.internal.processors.session.*;
-import org.apache.ignite.internal.processors.nodevalidation.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
@@ -169,11 +169,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
     /** */
     @GridToStringExclude
-    private Timer starveTimer;
+    private GridTimeoutProcessor.CancelableTask starveTask;
 
     /** */
     @GridToStringExclude
-    private Timer metricsLogTimer;
+    private GridTimeoutProcessor.CancelableTask metricsLogTask;
 
     /** Indicate error on grid stop. */
     @GridToStringExclude
@@ -867,13 +867,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         if (starveCheck) {
             final long interval = F.isEmpty(intervalStr) ? PERIODIC_STARVATION_CHECK_FREQ : Long.parseLong(intervalStr);
 
-            starveTimer = new Timer("ignite-starvation-checker");
-
-            starveTimer.scheduleAtFixedRate(new GridTimerTask() {
+            starveTask = ctx.timeout().schedule(new Runnable() {
                 /** Last completed task count. */
                 private long lastCompletedCnt;
 
-                @Override protected void safeRun() {
+                @Override public void run() {
                     if (!(execSvc instanceof ThreadPoolExecutor))
                         return;
 
@@ -896,13 +894,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         long metricsLogFreq = cfg.getMetricsLogFrequency();
 
         if (metricsLogFreq > 0) {
-            metricsLogTimer = new Timer("ignite-metrics-logger");
-
-            metricsLogTimer.scheduleAtFixedRate(new GridTimerTask() {
-                /** */
+            metricsLogTask = ctx.timeout().schedule(new Runnable() {
                 private final DecimalFormat dblFmt = new DecimalFormat("#.##");
 
-                @Override protected void safeRun() {
+                @Override public void run() {
                     if (log.isInfoEnabled()) {
                         ClusterMetrics m = cluster().localNode().metrics();
 
@@ -963,8 +958,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                             sysPoolQSize = exec.getQueue().size();
                         }
 
+                        String id = U.id8(localNode().id());
+
                         String msg = NL +
                             "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
+                            "    ^-- Node [id=" + id + ", name=" + name() + "]" + NL +
                             "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
                             "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
                                 dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
@@ -1165,6 +1163,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
         add(ATTR_CLIENT_MODE, cfg.isClientMode());
 
+        add(ATTR_CONSISTENCY_CHECK_SKIPPED, getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK));
+
         // Build a string from JVM arguments, because parameters with spaces are split.
         SB jvmArgs = new SB(512);
 
@@ -1550,7 +1550,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     ">>> Grid name: " + gridName + NL +
                     ">>> Local node [" +
                     "ID=" + locNode.id().toString().toUpperCase() +
-                    ", order=" + locNode.order() +
+                    ", order=" + locNode.order() + ", clientMode=" + ctx.clientNode() +
                     "]" + NL +
                     ">>> Local node addresses: " + U.addressesAsString(locNode) + NL +
                     ">>> Local ports: " + sb + NL;
@@ -1713,12 +1713,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             if (updateNtfTimer != null)
                 updateNtfTimer.cancel();
 
-            if (starveTimer != null)
-                starveTimer.cancel();
+            if (starveTask != null)
+                starveTask.close();
 
-            // Cancel metrics log timer.
-            if (metricsLogTimer != null)
-                metricsLogTimer.cancel();
+            if (metricsLogTask != null)
+                metricsLogTask.close();
 
             boolean interrupted = false;
 
@@ -2370,7 +2369,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         try {
             ctx.cache().dynamicStartCache(null, cacheName, nearCfg, true).get();
 
-            return ctx.cache().publicJCache(cacheName);
+            IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
+
+            checkNearCacheStarted(cache);
+
+            return cache;
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -2397,7 +2400,11 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     ctx.cache().dynamicStartCache(null, cacheName, nearCfg, false).get();
             }
 
-            return ctx.cache().publicJCache(cacheName);
+            IgniteCacheProxy<K, V> cache = ctx.cache().publicJCache(cacheName);
+
+            checkNearCacheStarted(cache);
+
+            return cache;
         }
         catch (IgniteCheckedException e) {
             throw CU.convertToCacheException(e);
@@ -2407,6 +2414,15 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param cache Cache.
+     */
+    private void checkNearCacheStarted(IgniteCacheProxy<?, ?> cache) {
+        if (!cache.context().isNear())
+            throw new IgniteException("Failed to start near cache " +
+                "(a cache with the same name without near cache is already started)");
+    }
+
     /** {@inheritDoc} */
     @Override public void destroyCache(String cacheName) {
         guard();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
index 98cc3a7..928db5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteNodeAttributes.java
@@ -126,9 +126,12 @@ public final class IgniteNodeAttributes {
     /** Security subject for authenticated node. */
     public static final String ATTR_SECURITY_SUBJECT = ATTR_PREFIX + ".security.subject";
 
-    /** Cache interceptors. */
+    /** Client mode flag. */
     public static final String ATTR_CLIENT_MODE = ATTR_PREFIX + ".cache.client";
 
+    /** Configuration consistency check disabled flag. */
+    public static final String ATTR_CONSISTENCY_CHECK_SKIPPED = ATTR_PREFIX + ".consistency.check.skipped";
+
     /**
      * Enforces singleton.
      */



[14/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index ed0e9dd..bbd98dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -18,46 +18,34 @@
 package org.apache.ignite.spi.discovery.tcp;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.events.*;
-import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.internal.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.discovery.tcp.messages.*;
 import org.jetbrains.annotations.*;
-import org.jsr166.*;
 
 import java.io.*;
 import java.net.*;
-import java.text.*;
 import java.util.*;
 import java.util.concurrent.*;
-
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.internal.IgniteNodeAttributes.*;
-import static org.apache.ignite.spi.IgnitePortProtocol.*;
-import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHeartbeatMessage.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage.*;
+import java.util.concurrent.atomic.*;
 
 /**
  * Discovery SPI implementation that uses TCP/IP for node discovery.
@@ -65,6 +53,14 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
  * Nodes are organized in ring. So almost all network exchange (except few cases) is
  * done across it.
  * <p>
+ * If node is configured as client node (see {@link IgniteConfiguration#clientMode})
+ * TcpDiscoverySpi starts in client mode as well. In this case node does not take its place in the ring,
+ * but it connects to random node in the ring (IP taken from IP finder configured) and
+ * use it as a router for discovery traffic.
+ * Therefore slow client node or its shutdown will not affect whole cluster. If TcpDiscoverySpi
+ * needs to be started in server mode regardless of {@link IgniteConfiguration#clientMode},
+ * {@link #forceSrvMode} should be set to true.
+ * <p>
  * At startup SPI tries to send messages to random IP taken from
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
  * and then this info goes to coordinator. When coordinator processes join request
@@ -105,6 +101,7 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
  * <li>Thread priority for threads started by SPI (see {@link #setThreadPriority(int)})</li>
  * <li>IP finder clean frequency (see {@link #setIpFinderCleanFrequency(long)})</li>
  * <li>Statistics print frequency (see {@link #setStatisticsPrintFrequency(long)}</li>
+ * <li>Force server mode (see {@link #setForceServerMode(boolean)}</li>
  * </ul>
  * <h2 class="header">Java Example</h2>
  * <pre name="code" class="java">
@@ -148,13 +145,43 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
 @IgniteSpiMultipleInstancesSupport(true)
 @DiscoverySpiOrderSupport(true)
 @DiscoverySpiHistorySupport(true)
-public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscoverySpiMBean {
+public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, TcpDiscoverySpiMBean {
+    /** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
+    public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
+
     /** Default local port range (value is <tt>100</tt>). */
     public static final int DFLT_PORT_RANGE = 100;
 
+    /** Default port to listen (value is <tt>47500</tt>). */
+    public static final int DFLT_PORT = 47500;
+
     /** Default timeout for joining topology (value is <tt>0</tt>). */
     public static final long DFLT_JOIN_TIMEOUT = 0;
 
+    /** Default network timeout in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_NETWORK_TIMEOUT = 5000;
+
+    /** Default value for thread priority (value is <tt>10</tt>). */
+    public static final int DFLT_THREAD_PRI = 10;
+
+    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 100;
+
+    /** Default size of topology snapshots history. */
+    public static final int DFLT_TOP_HISTORY_SIZE = 1000;
+
+    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT = 200;
+
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT = 50;
+
+    /** Default socket operations timeout in milliseconds (value is <tt>700ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT_CLIENT = 700;
+
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>700ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT_CLIENT = 700;
+
     /** Default reconnect attempts count (value is <tt>10</tt>). */
     public static final int DFLT_RECONNECT_CNT = 10;
 
@@ -173,133 +200,252 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
     /** Maximum ack timeout value for receiving message acknowledgement in milliseconds (value is <tt>600,000ms</tt>). */
     public static final long DFLT_MAX_ACK_TIMEOUT = 10 * 60 * 1000;
 
-    /** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
-    public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
+    /** Local address. */
+    protected String locAddr;
 
     /** Address resolver. */
     private AddressResolver addrRslvr;
 
+    /** IP finder. */
+    protected TcpDiscoveryIpFinder ipFinder;
+
+    /** Socket operations timeout. */
+    protected long sockTimeout; // Must be initialized in the constructor of child class.
+
+    /** Message acknowledgement timeout. */
+    protected long ackTimeout; // Must be initialized in the constructor of child class.
+
+    /** Network timeout. */
+    protected long netTimeout = DFLT_NETWORK_TIMEOUT;
+
+    /** Join timeout. */
+    @SuppressWarnings("RedundantFieldInitialization")
+    protected long joinTimeout = DFLT_JOIN_TIMEOUT;
+
+    /** Thread priority for all threads started by SPI. */
+    protected int threadPri = DFLT_THREAD_PRI;
+
+    /** Heartbeat messages issuing frequency. */
+    protected long hbFreq = DFLT_HEARTBEAT_FREQ;
+
+    /** Size of topology snapshots history. */
+    protected int topHistSize = DFLT_TOP_HISTORY_SIZE;
+
+    /** Grid discovery listener. */
+    protected volatile DiscoverySpiListener lsnr;
+
+    /** Data exchange. */
+    protected DiscoverySpiDataExchange exchange;
+
+    /** Metrics provider. */
+    protected DiscoveryMetricsProvider metricsProvider;
+
+    /** Local node attributes. */
+    protected Map<String, Object> locNodeAttrs;
+
+    /** Local node version. */
+    protected IgniteProductVersion locNodeVer;
+
+    /** Local node. */
+    protected TcpDiscoveryNode locNode;
+
+    /** Local host. */
+    protected InetAddress locHost;
+
+    /** Internal and external addresses of local node. */
+    protected Collection<InetSocketAddress> locNodeAddrs;
+
+    /** Start time of the very first grid node. */
+    protected volatile long gridStartTime;
+
+    /** Marshaller. */
+    protected final Marshaller marsh = new JdkMarshaller();
+
+    /** Statistics. */
+    protected final TcpDiscoveryStatistics stats = new TcpDiscoveryStatistics();
+
     /** Local port which node uses. */
-    private int locPort = DFLT_PORT;
+    protected int locPort = DFLT_PORT;
 
     /** Local port range. */
-    private int locPortRange = DFLT_PORT_RANGE;
+    protected int locPortRange = DFLT_PORT_RANGE;
+
+    /** Reconnect attempts count. */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+    protected int reconCnt = DFLT_RECONNECT_CNT;
 
     /** Statistics print frequency. */
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized", "RedundantFieldInitialization"})
-    private long statsPrintFreq = DFLT_STATS_PRINT_FREQ;
+    protected long statsPrintFreq = DFLT_STATS_PRINT_FREQ;
 
     /** Maximum message acknowledgement timeout. */
-    private long maxAckTimeout = DFLT_MAX_ACK_TIMEOUT;
-
-    /** Join timeout. */
-    @SuppressWarnings("RedundantFieldInitialization")
-    private long joinTimeout = DFLT_JOIN_TIMEOUT;
+    protected long maxAckTimeout = DFLT_MAX_ACK_TIMEOUT;
 
     /** Max heartbeats count node can miss without initiating status check. */
-    private int maxMissedHbs = DFLT_MAX_MISSED_HEARTBEATS;
+    protected int maxMissedHbs = DFLT_MAX_MISSED_HEARTBEATS;
 
     /** Max heartbeats count node can miss without failing client node. */
-    private int maxMissedClientHbs = DFLT_MAX_MISSED_CLIENT_HEARTBEATS;
+    protected int maxMissedClientHbs = DFLT_MAX_MISSED_CLIENT_HEARTBEATS;
 
     /** IP finder clean frequency. */
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    private long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
+    protected long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
 
-    /** Reconnect attempts count. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    private int reconCnt = DFLT_RECONNECT_CNT;
+    /** Node authenticator. */
+    protected DiscoverySpiNodeAuthenticator nodeAuth;
 
-    /** Nodes ring. */
+    /** Context initialization latch. */
     @GridToStringExclude
-    private final TcpDiscoveryNodesRing ring = new TcpDiscoveryNodesRing();
-
-    /** Topology snapshots history. */
-    private final SortedMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
-
-    /** Socket readers. */
-    private final Collection<SocketReader> readers = new LinkedList<>();
+    private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
 
-    /** TCP server for discovery SPI. */
-    private TcpServer tcpSrvr;
+    /** */
+    protected final CopyOnWriteArrayList<IgniteInClosure<TcpDiscoveryAbstractMessage>> sendMsgLsnrs =
+        new CopyOnWriteArrayList<>();
 
-    /** Message worker. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private RingMessageWorker msgWorker;
+    /** */
+    protected final CopyOnWriteArrayList<IgniteInClosure<Socket>> incomeConnLsnrs =
+        new CopyOnWriteArrayList<>();
 
-    /** Client message workers. */
-    private ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
+    /** Logger. */
+    @LoggerResource
+    protected IgniteLogger log;
 
-    /** Metrics sender. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private HeartbeatsSender hbsSnd;
+    /** */
+    protected TcpDiscoveryImpl impl;
 
-    /** Status checker. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private CheckStatusSender chkStatusSnd;
+    /** */
+    private boolean forceSrvMode;
 
-    /** IP finder cleaner. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private IpFinderCleaner ipFinderCleaner;
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+        return impl.getSpiState();
+    }
 
-    /** Statistics printer thread. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private StatisticsPrinter statsPrinter;
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return impl.getMessageWorkerQueueSize();
+    }
 
-    /** Failed nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID getCoordinator() {
+        return impl.getCoordinator();
+    }
 
-    /** Leaving nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return impl.getRemoteNodes();
+    }
 
-    /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */
-    private boolean ipFinderHasLocAddr;
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        return impl.getNode(nodeId);
+    }
 
-    /** Addresses that do not respond during join requests send (for resolving concurrent start). */
-    private final Collection<SocketAddress> noResAddrs = new GridConcurrentHashSet<>();
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        return impl.pingNode(nodeId);
+    }
 
-    /** Addresses that incoming join requests send were send from (for resolving concurrent start). */
-    private final Collection<SocketAddress> fromAddrs = new GridConcurrentHashSet<>();
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        impl.disconnect();
+    }
 
-    /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
-    private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
+    /** {@inheritDoc} */
+    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
+        nodeAuth = auth;
+    }
 
-    /** Context initialization latch. */
-    @GridToStringExclude
-    private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+        impl.sendCustomEvent(msg);
+    }
 
-    /** Node authenticator. */
-    private DiscoverySpiNodeAuthenticator nodeAuth;
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        impl.failNode(nodeId);
+    }
 
-    /** Mutex. */
-    private final Object mux = new Object();
+    /** {@inheritDoc} */
+    @Override public void dumpDebugInfo() {
+        impl.dumpDebugInfo(log);
+    }
 
-    /** Map with proceeding ping requests. */
-    private final ConcurrentMap<InetSocketAddress, IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
-        new ConcurrentHashMap8<>();
+    /** {@inheritDoc} */
+    @Override public boolean isClientMode() {
+        if (impl == null)
+            throw new IllegalStateException("TcpDiscoverySpi has not started");
 
-    /** Debug mode. */
-    private boolean debugMode;
+        return impl instanceof ClientImpl;
+    }
 
-    /** Debug messages history. */
-    private int debugMsgHist = 512;
+    /**
+     * If {@code true} TcpDiscoverySpi will started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}
+     *
+     * @return forceServerMode flag.
+     */
+    public boolean isForceServerMode() {
+        return forceSrvMode;
+    }
 
-    /** Received messages. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private ConcurrentLinkedDeque<String> debugLog;
+    /**
+     * Sets force server mode flag.
+     * <p>
+     * If {@code true} TcpDiscoverySpi is started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}.
+     *
+     * @param forceSrvMode forceServerMode flag.
+     * @return {@code this} for chaining.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setForceServerMode(boolean forceSrvMode) {
+        this.forceSrvMode = forceSrvMode;
 
-    /** */
-    private final CopyOnWriteArrayList<IgniteInClosure<TcpDiscoveryAbstractMessage>> sendMsgLsnrs =
-        new CopyOnWriteArrayList<>();
+        return this;
+    }
 
-    /** {@inheritDoc} */
+    /**
+     * Inject resources
+     *
+     * @param ignite Ignite.
+     */
     @IgniteInstanceResource
-    @Override public void injectResources(Ignite ignite) {
+    @Override protected void injectResources(Ignite ignite) {
         super.injectResources(ignite);
 
         // Inject resource.
-        if (ignite != null)
+        if (ignite != null) {
+            setLocalAddress(ignite.configuration().getLocalHost());
             setAddressResolver(ignite.configuration().getAddressResolver());
+        }
+    }
+
+    /**
+     * Sets local host IP address that discovery SPI uses.
+     * <p>
+     * If not provided, by default a first found non-loopback address
+     * will be used. If there is no non-loopback address available,
+     * then {@link InetAddress#getLocalHost()} will be used.
+     *
+     * @param locAddr IP address.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setLocalAddress(String locAddr) {
+        // Injection should not override value already set by Spring or user.
+        if (this.locAddr == null)
+            this.locAddr = locAddr;
+
+        return this;
+    }
+
+    /**
+     * Gets local address that was set to SPI with {@link #setLocalAddress(String)} method.
+     *
+     * @return local address.
+     */
+    public String getLocalAddress() {
+        return locAddr;
     }
 
     /**
@@ -340,8 +486,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @see #setAckTimeout(long)
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setReconnectCount(int reconCnt) {
+    public TcpDiscoverySpi setReconnectCount(int reconCnt) {
         this.reconCnt = reconCnt;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -358,36 +506,16 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * is reached, then the process of message sending is considered as failed.
      * <p>
      * If not specified, default is {@link #DFLT_MAX_ACK_TIMEOUT}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param maxAckTimeout Maximum acknowledgement timeout.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxAckTimeout(long maxAckTimeout) {
+    public TcpDiscoverySpi setMaxAckTimeout(long maxAckTimeout) {
         this.maxAckTimeout = maxAckTimeout;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getJoinTimeout() {
-        return joinTimeout;
-    }
 
-    /**
-     * Sets join timeout.
-     * <p>
-     * If non-shared IP finder is used and node fails to connect to
-     * any address from IP finder, node keeps trying to join within this
-     * timeout. If all addresses are still unresponsive, exception is thrown
-     * and node startup fails.
-     * <p>
-     * If not specified, default is {@link #DFLT_JOIN_TIMEOUT}.
-     *
-     * @param joinTimeout Join timeout ({@code 0} means wait forever).
-     *
-     * @see TcpDiscoveryIpFinder#isShared()
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setJoinTimeout(long joinTimeout) {
-        this.joinTimeout = joinTimeout;
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -401,12 +529,16 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * Sets local port to listen to.
      * <p>
      * If not specified, default is {@link #DFLT_PORT}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param locPort Local port to bind.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setLocalPort(int locPort) {
+    public TcpDiscoverySpi setLocalPort(int locPort) {
         this.locPort = locPort;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -420,12 +552,17 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * <tt>{@link #getLocalPort()} {@code + locPortRange}</tt>.
      * <p>
      * If not specified, default is {@link #DFLT_PORT_RANGE}.
+     * <p>
+     * Affected server nodes only.
+
      *
      * @param locPortRange Local port range to bind.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setLocalPortRange(int locPortRange) {
+    public TcpDiscoverySpi setLocalPortRange(int locPortRange) {
         this.locPortRange = locPortRange;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -437,12 +574,16 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * Sets max heartbeats count node can miss without initiating status check.
      * <p>
      * If not provided, default value is {@link #DFLT_MAX_MISSED_HEARTBEATS}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param maxMissedHbs Max missed heartbeats.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxMissedHeartbeats(int maxMissedHbs) {
+    public TcpDiscoverySpi setMaxMissedHeartbeats(int maxMissedHbs) {
         this.maxMissedHbs = maxMissedHbs;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -458,8 +599,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param maxMissedClientHbs Max missed client heartbeats.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxMissedClientHeartbeats(int maxMissedClientHbs) {
+    public TcpDiscoverySpi setMaxMissedClientHeartbeats(int maxMissedClientHbs) {
         this.maxMissedClientHbs = maxMissedClientHbs;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -479,8 +622,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param statsPrintFreq Statistics print frequency in milliseconds.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setStatisticsPrintFrequency(long statsPrintFreq) {
+    public TcpDiscoverySpi setStatisticsPrintFrequency(long statsPrintFreq) {
         this.statsPrintFreq = statsPrintFreq;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -492,211 +637,191 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * Sets IP finder clean frequency in milliseconds.
      * <p>
      * If not provided, default value is {@link #DFLT_IP_FINDER_CLEAN_FREQ}
+     * <p>
+     * Affected server nodes only.
      *
      * @param ipFinderCleanFreq IP finder clean frequency.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setIpFinderCleanFrequency(long ipFinderCleanFreq) {
+    public TcpDiscoverySpi setIpFinderCleanFrequency(long ipFinderCleanFreq) {
         this.ipFinderCleanFreq = ipFinderCleanFreq;
+
+        return this;
     }
 
     /**
-     * This method is intended for troubleshooting purposes only.
+     * Gets IP finder for IP addresses sharing and storing.
      *
-     * @param debugMode {code True} to start SPI in debug mode.
+     * @return IP finder for IP addresses sharing and storing.
      */
-    public void setDebugMode(boolean debugMode) {
-        this.debugMode = debugMode;
+    public TcpDiscoveryIpFinder getIpFinder() {
+        return ipFinder;
     }
 
     /**
-     * This method is intended for troubleshooting purposes only.
+     * Sets IP finder for IP addresses sharing and storing.
+     * <p>
+     * If not provided {@link org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder} will be used by default.
      *
-     * @param debugMsgHist Message history log size.
+     * @param ipFinder IP finder.
      */
-    public void setDebugMessageHistory(int debugMsgHist) {
-        this.debugMsgHist = debugMsgHist;
-    }
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setIpFinder(TcpDiscoveryIpFinder ipFinder) {
+        this.ipFinder = ipFinder;
 
-    /** {@inheritDoc} */
-    @Override public String getSpiState() {
-        synchronized (mux) {
-            return spiState.name();
-        }
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getSocketTimeout() {
-        return sockTimeout;
-    }
+    /**
+     * Sets socket operations timeout. This timeout is used to limit connection time and
+     * write-to-socket time.
+     * <p>
+     * Note that when running Ignite on Amazon EC2, socket timeout must be set to a value
+     * significantly greater than the default (e.g. to {@code 30000}).
+     * <p>
+     * If not specified, default is {@link #DFLT_SOCK_TIMEOUT} or {@link #DFLT_SOCK_TIMEOUT_CLIENT}.
+     *
+     * @param sockTimeout Socket connection timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setSocketTimeout(long sockTimeout) {
+        this.sockTimeout = sockTimeout;
 
-    /** {@inheritDoc} */
-    @Override public long getAckTimeout() {
-        return ackTimeout;
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getNetworkTimeout() {
-        return netTimeout;
-    }
+    /**
+     * Sets timeout for receiving acknowledgement for sent message.
+     * <p>
+     * If acknowledgement is not received within this timeout, sending is considered as failed
+     * and SPI tries to repeat message sending.
+     * <p>
+     * If not specified, default is {@link #DFLT_ACK_TIMEOUT} or {@link #DFLT_ACK_TIMEOUT_CLIENT}.
+     *
+     * @param ackTimeout Acknowledgement timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setAckTimeout(long ackTimeout) {
+        this.ackTimeout = ackTimeout;
 
-    /** {@inheritDoc} */
-    @Override public int getThreadPriority() {
-        return threadPri;
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getHeartbeatFrequency() {
-        return hbFreq;
-    }
+    /**
+     * Sets maximum network timeout to use for network operations.
+     * <p>
+     * If not specified, default is {@link #DFLT_NETWORK_TIMEOUT}.
+     *
+     * @param netTimeout Network timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setNetworkTimeout(long netTimeout) {
+        this.netTimeout = netTimeout;
 
-    /** {@inheritDoc} */
-    @Override public String getIpFinderFormatted() {
-        return ipFinder.toString();
+        return this;
     }
 
     /** {@inheritDoc} */
-    @Override public int getMessageWorkerQueueSize() {
-        return msgWorker.queueSize();
+    @Override public long getJoinTimeout() {
+        return joinTimeout;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getNodesJoined() {
-        return stats.joinedNodesCount();
-    }
+    /**
+     * Sets join timeout.
+     * <p>
+     * If non-shared IP finder is used and node fails to connect to
+     * any address from IP finder, node keeps trying to join within this
+     * timeout. If all addresses are still unresponsive, exception is thrown
+     * and node startup fails.
+     * <p>
+     * If not specified, default is {@link #DFLT_JOIN_TIMEOUT}.
+     *
+     * @param joinTimeout Join timeout ({@code 0} means wait forever).
+     *
+     * @see TcpDiscoveryIpFinder#isShared()
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setJoinTimeout(long joinTimeout) {
+        this.joinTimeout = joinTimeout;
 
-    /** {@inheritDoc} */
-    @Override public long getNodesLeft() {
-        return stats.leftNodesCount();
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getNodesFailed() {
-        return stats.failedNodesCount();
-    }
+    /**
+     * Sets thread priority. All threads within SPI will be started with it.
+     * <p>
+     * If not provided, default value is {@link #DFLT_THREAD_PRI}
+     *
+     * @param threadPri Thread priority.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setThreadPriority(int threadPri) {
+        this.threadPri = threadPri;
 
-    /** {@inheritDoc} */
-    @Override public long getPendingMessagesRegistered() {
-        return stats.pendingMessagesRegistered();
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getPendingMessagesDiscarded() {
-        return stats.pendingMessagesDiscarded();
-    }
+    /**
+     * Sets delay between issuing of heartbeat messages. SPI sends heartbeat messages
+     * in configurable time interval to other nodes to notify them about its state.
+     * <p>
+     * If not provided, default value is {@link #DFLT_HEARTBEAT_FREQ}.
+     *
+     * @param hbFreq Heartbeat frequency in milliseconds.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setHeartbeatFrequency(long hbFreq) {
+        this.hbFreq = hbFreq;
 
-    /** {@inheritDoc} */
-    @Override public long getAvgMessageProcessingTime() {
-        return stats.avgMessageProcessingTime();
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public long getMaxMessageProcessingTime() {
-        return stats.maxMessageProcessingTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalReceivedMessages() {
-        return stats.totalReceivedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Integer> getReceivedMessages() {
-        return stats.receivedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalProcessedMessages() {
-        return stats.totalProcessedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Integer> getProcessedMessages() {
-        return stats.processedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getCoordinatorSinceTimestamp() {
-        return stats.coordinatorSinceTimestamp();
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID getCoordinator() {
-        TcpDiscoveryNode crd = resolveCoordinator();
-
-        return crd != null ? crd.id() : null;
+    /**
+     * @return Size of topology snapshots history.
+     */
+    public long getTopHistorySize() {
+        return topHistSize;
     }
 
-    /** {@inheritDoc} */
-    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
-        assert nodeId != null;
-
-        UUID locNodeId0 = getLocalNodeId();
-
-        if (locNodeId0 != null && locNodeId0.equals(nodeId))
-            // Return local node directly.
-            return locNode;
+    /**
+     * Sets size of topology snapshots history. Specified size should be greater than or equal to default size
+     * {@link #DFLT_TOP_HISTORY_SIZE}.
+     *
+     * @param topHistSize Size of topology snapshots history.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setTopHistorySize(int topHistSize) {
+        if (topHistSize < DFLT_TOP_HISTORY_SIZE) {
+            U.warn(log, "Topology history size should be greater than or equal to default size. " +
+                "Specified size will not be set [curSize=" + this.topHistSize + ", specifiedSize=" + topHistSize +
+                ", defaultSize=" + DFLT_TOP_HISTORY_SIZE + ']');
 
-        TcpDiscoveryNode node = ring.node(nodeId);
+            return this;
+        }
 
-        if (node != null && !node.visible())
-            return null;
+        this.topHistSize = topHistSize;
 
-        return node;
+        return this;
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> getRemoteNodes() {
-        return F.upcast(ring.visibleRemoteNodes());
-    }
+    @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+        assert locNodeAttrs == null;
+        assert locNodeVer == null;
 
-    /** {@inheritDoc} */
-    @Override public Collection<Object> injectables() {
-        return F.<Object>asList(ipFinder);
-    }
+        if (log.isDebugEnabled()) {
+            log.debug("Node attributes to set: " + attrs);
+            log.debug("Node version to set: " + ver);
+        }
 
-    /** {@inheritDoc} */
-    @Override public void spiStart(String gridName) throws IgniteSpiException {
-        spiStart0(false);
+        locNodeAttrs = attrs;
+        locNodeVer = ver;
     }
 
     /**
-     * Starts or restarts SPI after stop (to reconnect).
-     *
-     * @param restart {@code True} if SPI is restarted after stop.
-     * @throws IgniteSpiException If failed.
+     * @param srvPort Server port.
      */
-    private void spiStart0(boolean restart) throws IgniteSpiException {
-        if (!restart)
-            // It is initial start.
-            onSpiStart();
-
-        synchronized (mux) {
-            spiState = DISCONNECTED;
-        }
-
-        if (debugMode) {
-            if (!log.isInfoEnabled())
-                throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
-                    "in debug mode.");
-
-            debugLog = new ConcurrentLinkedDeque<>();
-
-            U.quietAndWarn(log, "TCP discovery SPI is configured in debug mode.");
-        }
-
-        // Clear addresses collections.
-        fromAddrs.clear();
-        noResAddrs.clear();
-
-        sockTimeoutWorker = new SocketTimeoutWorker();
-        sockTimeoutWorker.start();
-
-        msgWorker = new RingMessageWorker();
-        msgWorker.start();
-
-        tcpSrvr = new TcpServer();
-
+    void initLocalNode(int srvPort, boolean addExtAddrAttr) {
         // Init local node.
         IgniteBiTuple<Collection<String>, Collection<String>> addrs;
 
@@ -711,4565 +836,975 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
             getLocalNodeId(),
             addrs.get1(),
             addrs.get2(),
-            tcpSrvr.port,
+            srvPort,
             metricsProvider,
             locNodeVer);
 
-        Collection<InetSocketAddress> extAddrs = addrRslvr == null ? null :
-            U.resolveAddresses(addrRslvr, F.flat(Arrays.asList(addrs.get1(), addrs.get2())),
-                locNode.discoveryPort());
-
-        if (extAddrs != null)
-            locNodeAttrs.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
-
-        locNode.setAttributes(locNodeAttrs);
-
-        locNode.local(true);
-
-        locNodeAddrs = getNodeAddresses(locNode);
-
-        if (log.isDebugEnabled())
-            log.debug("Local node initialized: " + locNode);
-
-        // Start TCP server thread after local node is initialized.
-        tcpSrvr.start();
-
-        ring.localNode(locNode);
-
-        if (ipFinder.isShared())
-            registerLocalNodeAddress();
-        else {
-            if (F.isEmpty(ipFinder.getRegisteredAddresses()))
-                throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
-                    "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
-                    "(specify list of IP addresses in configuration).");
-
-            ipFinderHasLocAddr = ipFinderHasLocalAddress();
-        }
-
-        if (statsPrintFreq > 0 && log.isInfoEnabled()) {
-            statsPrinter = new StatisticsPrinter();
-            statsPrinter.start();
-        }
-
-        stats.onJoinStarted();
-
-        joinTopology();
-
-        stats.onJoinFinished();
-
-        hbsSnd = new HeartbeatsSender();
-        hbsSnd.start();
-
-        chkStatusSnd = new CheckStatusSender();
-        chkStatusSnd.start();
-
-        if (ipFinder.isShared()) {
-            ipFinderCleaner = new IpFinderCleaner();
-            ipFinderCleaner.start();
-        }
-
-        if (log.isDebugEnabled() && !restart)
-            log.debug(startInfo());
-
-        if (restart)
-            getSpiContext().registerPort(tcpSrvr.port, TCP);
-    }
-
-    /**
-     * @throws IgniteSpiException If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    private void registerLocalNodeAddress() throws IgniteSpiException {
-        // Make sure address registration succeeded.
-        while (true) {
-            try {
-                ipFinder.initializeLocalAddresses(locNode.socketAddresses());
-
-                // Success.
-                break;
-            }
-            catch (IllegalStateException e) {
-                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
-                    locNode.socketAddresses(), e);
-            }
-            catch (IgniteSpiException e) {
-                LT.error(log, e, "Failed to register local node address in IP finder on start " +
-                    "(retrying every 2000 ms).");
-            }
-
-            try {
-                U.sleep(2000);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-        }
-    }
-
-    /**
-     * @throws IgniteSpiException If failed.
-     */
-    private void onSpiStart() throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(ipFinder != null, "ipFinder != null");
-        assertParameter(ipFinderCleanFreq > 0, "ipFinderCleanFreq > 0");
-        assertParameter(locPort > 1023, "localPort > 1023");
-        assertParameter(locPortRange >= 0, "localPortRange >= 0");
-        assertParameter(locPort + locPortRange <= 0xffff, "locPort + locPortRange <= 0xffff");
-        assertParameter(netTimeout > 0, "networkTimeout > 0");
-        assertParameter(sockTimeout > 0, "sockTimeout > 0");
-        assertParameter(ackTimeout > 0, "ackTimeout > 0");
-        assertParameter(maxAckTimeout > ackTimeout, "maxAckTimeout > ackTimeout");
-        assertParameter(reconCnt > 0, "reconnectCnt > 0");
-        assertParameter(hbFreq > 0, "heartbeatFreq > 0");
-        assertParameter(maxMissedHbs > 0, "maxMissedHeartbeats > 0");
-        assertParameter(maxMissedClientHbs > 0, "maxMissedClientHeartbeats > 0");
-        assertParameter(threadPri > 0, "threadPri > 0");
-        assertParameter(statsPrintFreq >= 0, "statsPrintFreq >= 0");
-
-        try {
-            locHost = U.resolveLocalHost(locAddr);
-        }
-        catch (IOException e) {
-            throw new IgniteSpiException("Unknown local address: " + locAddr, e);
-        }
-
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("localHost", locHost.getHostAddress()));
-            log.debug(configInfo("localPort", locPort));
-            log.debug(configInfo("localPortRange", locPortRange));
-            log.debug(configInfo("threadPri", threadPri));
-            log.debug(configInfo("networkTimeout", netTimeout));
-            log.debug(configInfo("sockTimeout", sockTimeout));
-            log.debug(configInfo("ackTimeout", ackTimeout));
-            log.debug(configInfo("maxAckTimeout", maxAckTimeout));
-            log.debug(configInfo("reconnectCount", reconCnt));
-            log.debug(configInfo("ipFinder", ipFinder));
-            log.debug(configInfo("ipFinderCleanFreq", ipFinderCleanFreq));
-            log.debug(configInfo("heartbeatFreq", hbFreq));
-            log.debug(configInfo("maxMissedHeartbeats", maxMissedHbs));
-            log.debug(configInfo("statsPrintFreq", statsPrintFreq));
-        }
-
-        // Warn on odd network timeout.
-        if (netTimeout < 3000)
-            U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
-
-        registerMBean(gridName, this, TcpDiscoverySpiMBean.class);
-
-        if (ipFinder instanceof TcpDiscoveryMulticastIpFinder) {
-            TcpDiscoveryMulticastIpFinder mcastIpFinder = ((TcpDiscoveryMulticastIpFinder)ipFinder);
-
-            if (mcastIpFinder.getLocalAddress() == null)
-                mcastIpFinder.setLocalAddress(locAddr);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        super.onContextInitialized0(spiCtx);
-
-        ctxInitLatch.countDown();
-
-        spiCtx.registerPort(tcpSrvr.port, TCP);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSpiContext getSpiContext() {
-        if (ctxInitLatch.getCount() > 0) {
-            if (log.isDebugEnabled())
-                log.debug("Waiting for context initialization.");
-
-            try {
-                U.await(ctxInitLatch);
-
-                if (log.isDebugEnabled())
-                    log.debug("Context has been initialized.");
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                U.warn(log, "Thread has been interrupted while waiting for SPI context initialization.", e);
-            }
-        }
-
-        return super.getSpiContext();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        spiStop0(false);
-    }
-
-    /**
-     * Stops SPI finally or stops SPI for restart.
-     *
-     * @param disconnect {@code True} if SPI is being disconnected.
-     * @throws IgniteSpiException If failed.
-     */
-    private void spiStop0(boolean disconnect) throws IgniteSpiException {
-        if (ctxInitLatch.getCount() > 0)
-            // Safety.
-            ctxInitLatch.countDown();
-
-        if (log.isDebugEnabled()) {
-            if (disconnect)
-                log.debug("Disconnecting SPI.");
-            else
-                log.debug("Preparing to start local node stop procedure.");
-        }
-
-        if (disconnect) {
-            synchronized (mux) {
-                spiState = DISCONNECTING;
-            }
-        }
-
-        if (msgWorker != null && msgWorker.isAlive() && !disconnect) {
-            // Send node left message only if it is final stop.
-            msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(getLocalNodeId()));
-
-            synchronized (mux) {
-                long threshold = U.currentTimeMillis() + netTimeout;
-
-                long timeout = netTimeout;
-
-                while (spiState != LEFT && timeout > 0) {
-                    try {
-                        mux.wait(timeout);
+        if (addExtAddrAttr) {
+            Collection<InetSocketAddress> extAddrs = addrRslvr == null ? null :
+                U.resolveAddresses(addrRslvr, F.flat(Arrays.asList(addrs.get1(), addrs.get2())),
+                    locNode.discoveryPort());
 
-                        timeout = threshold - U.currentTimeMillis();
-                    }
-                    catch (InterruptedException ignored) {
-                        Thread.currentThread().interrupt();
+            locNodeAddrs = new LinkedHashSet<>();
+            locNodeAddrs.addAll(locNode.socketAddresses());
 
-                        break;
-                    }
-                }
+            if (extAddrs != null) {
+                locNodeAttrs.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
 
-                if (spiState == LEFT) {
-                    if (log.isDebugEnabled())
-                        log.debug("Verification for local node leave has been received from coordinator" +
-                            " (continuing stop procedure).");
-                }
-                else if (log.isInfoEnabled()) {
-                    log.info("No verification for local node leave has been received from coordinator" +
-                        " (will stop node anyway).");
-                }
+                locNodeAddrs.addAll(extAddrs);
             }
         }
 
-        U.interrupt(tcpSrvr);
-        U.join(tcpSrvr, log);
-
-        Collection<SocketReader> tmp;
-
-        synchronized (mux) {
-            tmp = U.arrayList(readers);
-        }
-
-        U.interrupt(tmp);
-        U.joinThreads(tmp, log);
-
-        U.interrupt(hbsSnd);
-        U.join(hbsSnd, log);
-
-        U.interrupt(chkStatusSnd);
-        U.join(chkStatusSnd, log);
-
-        U.interrupt(ipFinderCleaner);
-        U.join(ipFinderCleaner, log);
-
-        U.interrupt(msgWorker);
-        U.join(msgWorker, log);
-
-        U.interrupt(sockTimeoutWorker);
-        U.join(sockTimeoutWorker, log);
-
-        U.interrupt(statsPrinter);
-        U.join(statsPrinter, log);
-
-        if (ipFinder != null)
-            ipFinder.close();
-
-        Collection<TcpDiscoveryNode> rmts = null;
-
-        if (!disconnect) {
-            // This is final stop.
-            unregisterMBean();
-
-            if (log.isDebugEnabled())
-                log.debug(stopInfo());
-        }
-        else {
-            getSpiContext().deregisterPorts();
-
-            rmts = ring.visibleRemoteNodes();
-        }
-
-        long topVer = ring.topologyVersion();
-
-        ring.clear();
-
-        if (rmts != null && !rmts.isEmpty()) {
-            // This is restart/disconnection and remote nodes are not empty.
-            // We need to fire FAIL event for each.
-            DiscoverySpiListener lsnr = this.lsnr;
-
-            if (lsnr != null) {
-                Set<ClusterNode> processed = new HashSet<>();
-
-                for (TcpDiscoveryNode n : rmts) {
-                    assert n.visible();
-
-                    processed.add(n);
-
-                    List<ClusterNode> top = U.arrayList(rmts, F.notIn(processed));
-
-                    topVer++;
-
-                    Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer,
-                        Collections.unmodifiableList(top));
-
-                    lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, hist, null);
-                }
-            }
-        }
-
-        printStatistics();
-
-        stats.clear();
-
-        synchronized (mux) {
-            // Clear stored data.
-            leavingNodes.clear();
-            failedNodes.clear();
-
-            spiState = DISCONNECTED;
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        super.onContextDestroyed0();
-
-        if (ctxInitLatch.getCount() > 0)
-            // Safety.
-            ctxInitLatch.countDown();
-
-        getSpiContext().deregisterPorts();
-    }
-
-    /**
-     * @throws IgniteSpiException If any error occurs.
-     * @return {@code true} if IP finder contains local address.
-     */
-    private boolean ipFinderHasLocalAddress() throws IgniteSpiException {
-        for (InetSocketAddress locAddr : locNodeAddrs) {
-            for (InetSocketAddress addr : registeredAddresses())
-                try {
-                    int port = addr.getPort();
-
-                    InetSocketAddress resolved = addr.isUnresolved() ?
-                        new InetSocketAddress(InetAddress.getByName(addr.getHostName()), port) :
-                        new InetSocketAddress(addr.getAddress(), port);
-
-                    if (resolved.equals(locAddr))
-                        return true;
-                }
-                catch (UnknownHostException e) {
-                    onException(e.getMessage(), e);
-                }
-        }
-
-        return false;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean pingNode(UUID nodeId) {
-        assert nodeId != null;
-
-        if (log.isDebugEnabled())
-            log.debug("Ping node. NodeId: [" + nodeId + "].");
-
-        if (nodeId == getLocalNodeId())
-            return true;
-
-        TcpDiscoveryNode node = ring.node(nodeId);
-
-        if (node == null || !node.visible())
-            return false;
-
-        boolean res = pingNode(node);
-
-        if (!res && !node.isClient()) {
-            LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
-
-            msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
-        }
-
-        return res;
-    }
-
-    /**
-     * Pings the remote node to see if it's alive.
-     *
-     * @param node Node.
-     * @return {@code True} if ping succeeds.
-     */
-    private boolean pingNode(TcpDiscoveryNode node) {
-        assert node != null;
-
-        if (node.id().equals(getLocalNodeId()))
-            return true;
-
-        UUID clientNodeId = null;
-
-        if (node.isClient()) {
-            clientNodeId = node.id();
-
-            node = ring.node(node.clientRouterNodeId());
-
-            if (node == null || !node.visible())
-                return false;
-        }
-
-        for (InetSocketAddress addr : getNodeAddresses(node, U.sameMacs(locNode, node))) {
-            try {
-                // ID returned by the node should be the same as ID of the parameter for ping to succeed.
-                IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
-
-                return node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
-            }
-            catch (IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']');
-
-                onException("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']', e);
-                // continue;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Pings the remote node by its address to see if it's alive.
-     *
-     * @param addr Address of the node.
-     * @return ID of the remote node if node alive.
-     * @throws IgniteSpiException If an error occurs.
-     */
-    private IgniteBiTuple<UUID, Boolean> pingNode(InetSocketAddress addr, @Nullable UUID clientNodeId)
-        throws IgniteCheckedException {
-        assert addr != null;
-
-        UUID locNodeId = getLocalNodeId();
-
-        if (F.contains(locNodeAddrs, addr))
-            return F.t(getLocalNodeId(), false);
-
-        GridFutureAdapter<IgniteBiTuple<UUID, Boolean>> fut = new GridFutureAdapter<>();
-
-        IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
-
-        if (oldFut != null)
-            return oldFut.get();
-        else {
-            Collection<Throwable> errs = null;
-
-            try {
-                Socket sock = null;
-
-                for (int i = 0; i < reconCnt; i++) {
-                    try {
-                        if (addr.isUnresolved())
-                            addr = new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort());
-
-                        long tstamp = U.currentTimeMillis();
-
-                        sock = openSocket(addr);
-
-                        writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
-
-                        TcpDiscoveryPingResponse res = readMessage(sock, null, netTimeout);
-
-                        if (locNodeId.equals(res.creatorNodeId())) {
-                            if (log.isDebugEnabled())
-                                log.debug("Ping response from local node: " + res);
-
-                            break;
-                        }
-
-                        stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
-
-                        IgniteBiTuple<UUID, Boolean> t = F.t(res.creatorNodeId(), res.clientExists());
-
-                        fut.onDone(t);
-
-                        return t;
-                    }
-                    catch (IOException | IgniteCheckedException e) {
-                        if (errs == null)
-                            errs = new ArrayList<>();
-
-                        errs.add(e);
-                    }
-                    finally {
-                        U.closeQuiet(sock);
-                    }
-                }
-            }
-            catch (Throwable t) {
-                fut.onDone(t);
-
-                if (t instanceof Error)
-                    throw t;
-
-                throw U.cast(t);
-            }
-            finally {
-                if (!fut.isDone())
-                    fut.onDone(U.exceptionWithSuppressed("Failed to ping node by address: " + addr, errs));
-
-                boolean b = pingMap.remove(addr, fut);
-
-                assert b;
-            }
-
-            return fut.get();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disconnect() throws IgniteSpiException {
-        spiStop0(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator nodeAuth) {
-        this.nodeAuth = nodeAuth;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendCustomEvent(Serializable evt) {
-        try {
-            byte[] msgBytes;
-
-            msgBytes = marsh.marshal(evt);
-
-            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, msgBytes));
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        ClusterNode node = ring.node(nodeId);
-
-        if (node != null) {
-            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
-                node.id(), node.order());
-
-            msgWorker.addMessage(msg);
-        }
-    }
-
-    /**
-     * Tries to join this node to topology.
-     *
-     * @throws IgniteSpiException If any error occurs.
-     */
-    private void joinTopology() throws IgniteSpiException {
-        synchronized (mux) {
-            assert spiState == CONNECTING || spiState == DISCONNECTED;
-
-            spiState = CONNECTING;
-        }
-
-        SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes()
-            .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-        // Marshal credentials for backward compatibility and security.
-        marshalCredentials(locNode);
-
-        while (true) {
-            if (!sendJoinRequestMessage()) {
-                if (log.isDebugEnabled())
-                    log.debug("Join request message has not been sent (local node is the first in the topology).");
-
-                if (nodeAuth != null) {
-                    // Authenticate local node.
-                    try {
-                        SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
-
-                        if (subj == null)
-                            throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
-
-                        Map<String, Object> attrs = new HashMap<>(locNode.attributes());
-
-                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                            ignite.configuration().getMarshaller().marshal(subj));
-                        attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-                        locNode.setAttributes(attrs);
-                    }
-                    catch (IgniteException | IgniteCheckedException e) {
-                        throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
-                    }
-                }
-
-                locNode.order(1);
-                locNode.internalOrder(1);
-
-                gridStartTime = U.currentTimeMillis();
-
-                locNode.visible(true);
-
-                ring.clear();
-
-                ring.topologyVersion(1);
-
-                synchronized (mux) {
-                    topHist.clear();
-
-                    spiState = CONNECTED;
-
-                    mux.notifyAll();
-                }
-
-                notifyDiscovery(EVT_NODE_JOINED, 1, locNode);
-
-                break;
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("Join request message has been sent (waiting for coordinator response).");
-
-            synchronized (mux) {
-                long threshold = U.currentTimeMillis() + netTimeout;
-
-                long timeout = netTimeout;
-
-                while (spiState == CONNECTING && timeout > 0) {
-                    try {
-                        mux.wait(timeout);
-
-                        timeout = threshold - U.currentTimeMillis();
-                    }
-                    catch (InterruptedException ignored) {
-                        Thread.currentThread().interrupt();
-
-                        throw new IgniteSpiException("Thread has been interrupted.");
-                    }
-                }
-
-                if (spiState == CONNECTED)
-                    break;
-                else if (spiState == DUPLICATE_ID)
-                    throw duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
-                else if (spiState == AUTH_FAILED)
-                    throw authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
-                else if (spiState == CHECK_FAILED)
-                    throw checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
-                else if (spiState == LOOPBACK_PROBLEM) {
-                    TcpDiscoveryLoopbackProblemMessage msg = (TcpDiscoveryLoopbackProblemMessage)joinRes.get();
-
-                    boolean locHostLoopback = locHost.isLoopbackAddress();
-
-                    String firstNode = locHostLoopback ? "local" : "remote";
-
-                    String secondNode = locHostLoopback ? "remote" : "local";
-
-                    throw new IgniteSpiException("Failed to add node to topology because " + firstNode +
-                        " node is configured to use loopback address, but " + secondNode + " node is not " +
-                        "(consider changing 'localAddress' configuration parameter) " +
-                        "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" +
-                        U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
-                }
-                else
-                    LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
-                        "Check remote nodes logs for possible error messages. " +
-                        "Note that large topology may require significant time to start. " +
-                        "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
-                        "if getting this message on the starting nodes [networkTimeout=" + netTimeout + ']');
-            }
-        }
-
-        assert locNode.order() != 0;
-        assert locNode.internalOrder() != 0;
-
-        if (log.isDebugEnabled())
-            log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder());
-    }
-
-    /**
-     * Tries to send join request message to a random node presenting in topology.
-     * Address is provided by {@link TcpDiscoveryIpFinder} and message is
-     * sent to first node connection succeeded to.
-     *
-     * @return {@code true} if send succeeded.
-     * @throws IgniteSpiException If any error occurs.
-     */
-    @SuppressWarnings({"BusyWait"})
-    private boolean sendJoinRequestMessage() throws IgniteSpiException {
-        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
-            collectExchangeData(getLocalNodeId()));
-
-        // Time when it has been detected, that addresses from IP finder do not respond.
-        long noResStart = 0;
-
-        while (true) {
-            Collection<InetSocketAddress> addrs = resolvedAddresses();
-
-            if (F.isEmpty(addrs))
-                return false;
-
-            boolean retry = false;
-            Collection<Exception> errs = new ArrayList<>();
-
-            try (SocketMultiConnector multiConnector = new SocketMultiConnector(addrs, 2)) {
-                GridTuple3<InetSocketAddress, Socket, Exception> tuple;
-
-                while ((tuple = multiConnector.next()) != null) {
-                    InetSocketAddress addr = tuple.get1();
-                    Socket sock = tuple.get2();
-                    Exception ex = tuple.get3();
-
-                    if (ex == null) {
-                        assert sock != null;
-
-                        try {
-                            Integer res = sendMessageDirectly(joinReq, addr, sock);
-
-                            assert res != null;
-
-                            noResAddrs.remove(addr);
-
-                            // Address is responsive, reset period start.
-                            noResStart = 0;
-
-                            switch (res) {
-                                case RES_WAIT:
-                                    // Concurrent startup, try sending join request again or wait if no success.
-                                    retry = true;
-
-                                    break;
-                                case RES_OK:
-                                    if (log.isDebugEnabled())
-                                        log.debug("Join request message has been sent to address [addr=" + addr +
-                                            ", req=" + joinReq + ']');
-
-                                    // Join request sending succeeded, wait for response from topology.
-                                    return true;
-
-                                default:
-                                    // Concurrent startup, try next node.
-                                    if (res == RES_CONTINUE_JOIN) {
-                                        if (!fromAddrs.contains(addr))
-                                            retry = true;
-                                    }
-                                    else {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Unexpected response to join request: " + res);
-
-                                        retry = true;
-                                    }
-
-                                    break;
-                            }
-                        }
-                        catch (IgniteSpiException e) {
-                            e.printStackTrace();
-
-                            ex = e;
-                        }
-                    }
-
-                    if (ex != null) {
-                        errs.add(ex);
-
-                        if (log.isDebugEnabled()) {
-                            IOException ioe = X.cause(ex, IOException.class);
-
-                            log.debug("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']');
-
-                            onException("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe);
-                        }
-
-                        noResAddrs.add(addr);
-                    }
-                }
-            }
-
-            if (retry) {
-                if (log.isDebugEnabled())
-                    log.debug("Concurrent discovery SPI start has been detected (local node should wait).");
-
-                try {
-                    U.sleep(2000);
-                }
-                catch (IgniteInterruptedCheckedException e) {
-                    throw new IgniteSpiException("Thread has been interrupted.", e);
-                }
-            }
-            else if (!ipFinder.isShared() && !ipFinderHasLocAddr) {
-                IgniteCheckedException e = null;
-
-                if (!errs.isEmpty()) {
-                    e = new IgniteCheckedException("Multiple connection attempts failed.");
-
-                    for (Exception err : errs)
-                        e.addSuppressed(err);
-                }
-
-                if (e != null && X.hasCause(e, ConnectException.class))
-                    LT.warn(log, null, "Failed to connect to any address from IP finder " +
-                        "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
-                        addrs);
-
-                if (joinTimeout > 0) {
-                    if (noResStart == 0)
-                        noResStart = U.currentTimeMillis();
-                    else if (U.currentTimeMillis() - noResStart > joinTimeout)
-                        throw new IgniteSpiException(
-                            "Failed to connect to any address from IP finder within join timeout " +
-                                "(make sure IP finder addresses are correct, and operating system firewalls are disabled " +
-                                "on all host machines, or consider increasing 'joinTimeout' configuration property): " +
-                                addrs, e);
-                }
-
-                try {
-                    U.sleep(2000);
-                }
-                catch (IgniteInterruptedCheckedException ex) {
-                    throw new IgniteSpiException("Thread has been interrupted.", ex);
-                }
-            }
-            else
-                break;
-        }
-
-        return false;
-    }
-
-    /**
-     * Establishes connection to an address, sends message and returns the response (if any).
-     *
-     * @param msg Message to send.
-     * @param addr Address to send message to.
-     * @return Response read from the recipient or {@code null} if no response is supposed.
-     * @throws IgniteSpiException If an error occurs.
-     */
-    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, Socket sock)
-        throws IgniteSpiException {
-        assert msg != null;
-        assert addr != null;
-
-        Collection<Throwable> errs = null;
-
-        long ackTimeout0 = ackTimeout;
-
-        int connectAttempts = 1;
-
-        boolean joinReqSent = false;
-
-        UUID locNodeId = getLocalNodeId();
-
-        for (int i = 0; i < reconCnt; i++) {
-            // Need to set to false on each new iteration,
-            // since remote node may leave in the middle of the first iteration.
-            joinReqSent = false;
-
-            boolean openSock = false;
-
-            try {
-                long tstamp = U.currentTimeMillis();
-
-                if (sock == null)
-                    sock = openSocket(addr);
-
-                openSock = true;
-
-                // Handshake.
-                writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
-
-                TcpDiscoveryHandshakeResponse res = readMessage(sock, null, ackTimeout0);
-
-                if (locNodeId.equals(res.creatorNodeId())) {
-                    if (log.isDebugEnabled())
-                        log.debug("Handshake response from local node: " + res);
-
-                    break;
-                }
-
-                stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
-
-                // Send message.
-                tstamp = U.currentTimeMillis();
-
-                writeToSocket(sock, msg);
-
-                stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
-
-                if (debugMode)
-                    debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
-                        ", rmtNodeId=" + res.creatorNodeId() + ']');
-
-                if (log.isDebugEnabled())
-                    log.debug("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
-                        ", rmtNodeId=" + res.creatorNodeId() + ']');
-
-                // Connection has been established, but
-                // join request may not be unmarshalled on remote host.
-                // E.g. due to class not found issue.
-                joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
-
-                return readReceipt(sock, ackTimeout0);
-            }
-            catch (ClassCastException e) {
-                // This issue is rarely reproducible on AmazonEC2, but never
-                // on dedicated machines.
-                if (log.isDebugEnabled())
-                    U.error(log, "Class cast exception on direct send: " + addr, e);
-
-                onException("Class cast exception on direct send: " + addr, e);
-
-                if (errs == null)
-                    errs = new ArrayList<>();
-
-                errs.add(e);
-            }
-            catch (IOException | IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.error("Exception on direct send: " + e.getMessage(), e);
-
-                onException("Exception on direct send: " + e.getMessage(), e);
-
-                if (errs == null)
-                    errs = new ArrayList<>();
-
-                errs.add(e);
-
-                if (!openSock) {
-                    // Reconnect for the second time, if connection is not established.
-                    if (connectAttempts < 2) {
-                        connectAttempts++;
-
-                        continue;
-                    }
-
-                    break; // Don't retry if we can not establish connection.
-                }
-
-                if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
-                    ackTimeout0 *= 2;
-
-                    if (!checkAckTimeout(ackTimeout0))
-                        break;
-                }
-            }
-            finally {
-                U.closeQuiet(sock);
-
-                sock = null;
-            }
-        }
-
-        if (joinReqSent) {
-            if (log.isDebugEnabled())
-                log.debug("Join request has been sent, but receipt has not been read (returning RES_WAIT).");
-
-            // Topology will not include this node,
-            // however, warning on timed out join will be output.
-            return RES_OK;
-        }
-
-        throw new IgniteSpiException(
-            "Failed to send message to address [addr=" + addr + ", msg=" + msg + ']',
-            U.exceptionWithSuppressed("Failed to send message to address " +
-                "[addr=" + addr + ", msg=" + msg + ']', errs));
-    }
-
-    /**
-     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
-     *
-     * @param node Node to marshall credentials for.
-     * @throws IgniteSpiException If marshalling failed.
-     */
-    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
-        try {
-            // Use security-unsafe getter.
-            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
-
-            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
-                marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
-
-            node.setAttributes(attrs);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
-        }
-    }
-
-    /**
-     * Unmarshalls credentials with discovery SPI marshaller (will not replace attribute value).
-     *
-     * @param node Node to unmarshall credentials for.
-     * @return Security credentials.
-     * @throws IgniteSpiException If unmarshal fails.
-     */
-    private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
-        try {
-            byte[] credBytes = (byte[])node.getAttributes().get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-            if (credBytes == null)
-                return null;
-
-            return marsh.unmarshal(credBytes, null);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e);
-        }
-    }
-
-    /**
-     * @param ackTimeout Acknowledgement timeout.
-     * @return {@code True} if acknowledgement timeout is less or equal to
-     * maximum acknowledgement timeout, {@code false} otherwise.
-     */
-    private boolean checkAckTimeout(long ackTimeout) {
-        if (ackTimeout > maxAckTimeout) {
-            LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
-                "(consider increasing 'maxAckTimeout' configuration property) " +
-                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + maxAckTimeout + ']');
-
-            return false;
-        }
-
-        return true;
-    }
-
-    /**
-     * Notify external listener on discovery event.
-     *
-     * @param type Discovery event type. See {@link DiscoveryEvent} for more details.
-     * @param topVer Topology version.
-     * @param node Remote node this event is connected with.
-     */
-    private void notifyDiscovery(int type, long topVer, TcpDiscoveryNode node) {
-        assert type > 0;
-        assert node != null;
-
-        DiscoverySpiListener lsnr = this.lsnr;
-
-        TcpDiscoverySpiState spiState = spiStateCopy();
-
-        if (lsnr != null && node.visible() && (spiState == CONNECTED || spiState == DISCONNECTING)) {
-            if (log.isDebugEnabled())
-                log.debug("Discovery notification [node=" + node + ", spiState=" + spiState +
-                    ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
-
-            Collection<ClusterNode> top = F.<TcpDiscoveryNode, ClusterNode>upcast(ring.visibleNodes());
-
-            Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer, top);
-
-            lsnr.onDiscovery(type, topVer, node, top, hist, null);
-        }
-        else if (log.isDebugEnabled())
-            log.debug("Skipped discovery notification [node=" + node + ", spiState=" + spiState +
-                ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
-    }
-
-    /**
-     * Update topology history with new topology snapshots.
-     *
-     * @param topVer Topology version.
-     * @param top Topology snapshot.
-     * @return Copy of updated topology history.
-     */
-    @Nullable private Map<Long, Collection<ClusterNode>> updateTopologyHistory(long topVer, Collection<ClusterNode> top) {
-        synchronized (mux) {
-            if (topHist.containsKey(topVer))
-                return null;
-
-            topHist.put(topVer, top);
-
-            while (topHist.size() > topHistSize)
-                topHist.remove(topHist.firstKey());
-
-            if (log.isDebugEnabled())
-                log.debug("Added topology snapshot to history, topVer=" + topVer + ", historySize=" + topHist.size());
-
-            return new TreeMap<>(topHist);
-        }
-    }
-
-    /**
-     * @param msg Error message.
-     * @param e Exception.
-     */
-    private void onException(String msg, Exception e){
-        getExceptionRegistry().onException(msg, e);
-    }
-
-    /**
-     * @param node Node.
-     * @return {@link LinkedHashSet} of internal and external addresses of provided node.
-     *      Internal addresses placed before external addresses.
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    private LinkedHashSet<InetSocketAddress> getNodeAddresses(TcpDiscoveryNode node) {
-        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(node.socketAddresses());
-
-        Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
-
-        if (extAddrs != null)
-            res.addAll(extAddrs);
-
-        return res;
-    }
-
-    /**
-     * @param node Node.
-     * @param sameHost Same host flag.
-     * @return {@link LinkedHashSet} of internal and external addresses of provided node.
-     *      Internal addresses placed before external addresses.
-     *      Internal addresses will be sorted with {@code inetAddressesComparator(sameHost)}.
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    private LinkedHashSet<InetSocketAddress> getNodeAddresses(TcpDiscoveryNode node, boolean sameHost) {
-        List<InetSocketAddress> addrs = U.arrayList(node.socketAddresses());
-
-        Collections.sort(addrs, U.inetAddressesComparator(sameHost));
-
-        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(addrs);
-
-        Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
-
-        if (extAddrs != null)
-            res.addAll(extAddrs);
-
-        return res;
-    }
-
-    /**
-     * Checks whether local node is coordinator. Nodes that are leaving or failed
-     * (but are still in topology) are removed from search.
-     *
-     * @return {@code true} if local node is coordinator.
-     */
-    private boolean isLocalNodeCoordinator() {
-        synchronized (mux) {
-            boolean crd = spiState == CONNECTED && locNode.equals(resolveCoordinator());
-
-            if (crd)
-                stats.onBecomingCoordinator();
-
-            return crd;
-        }
-    }
-
-    /**
-     * @return Spi state copy.
-     */
-    private TcpDiscoverySpiState spiStateCopy() {
-        TcpDiscoverySpiState state;
-
-        synchronized (mux) {
-            state = spiState;
-        }
-
-        return state;
-    }
-
-    /**
-     * Resolves coordinator. Nodes that are leaving or failed (but are still in
-     * topology) are removed from search.
-     *
-     * @return Coordinator node or {@code null} if there are no coordinator
-     * (i.e. local node is the last one and is currently stopping).
-     */
-    @Nullable private TcpDiscoveryNode resolveCoordinator() {
-        return resolveCoordinator(null);
-    }
-
-    /**
-     * Resolves coordinator. Nodes that are leaving or failed (but are still in
-     * topology) are removed from search as well as provided filter.
-     *
-     * @param filter Nodes to exclude when resolving coordinator (optional).
-     * @return Coordinator node or {@code null} if there are no coordinator
-     * (i.e. local node is the last one and is currently stopping).
-     */
-    @Nullable private TcpDiscoveryNode resolveCoordinator(
-        @Nullable Collection<TcpDiscoveryNode> filter) {
-        synchronized (mux) {
-            Collection<TcpDiscoveryNode> excluded = F.concat(false, failedNodes, leavingNodes);
-
-            if (!F.isEmpty(filter))
-                excluded = F.concat(false, excluded, filter);
-
-            return ring.coordinator(excluded);
-        }
-    }
-
-    /**
-     * Prints SPI statistics.
-     */
-    private void printStatistics() {
-        if (log.isInfoEnabled() && statsPrintFreq > 0) {
-            int failedNodesSize;
-            int leavingNodesSize;
-
-            synchronized (mux) {
-                failedNodesSize = failedNodes.size();
-                leavingNodesSize = leavingNodes.size();
-            }
-
-            Runtime runtime = Runtime.getRuntime();
-
-            TcpDiscoveryNode coord = resolveCoordinator();
-
-            log.info("Discovery SPI statistics [statistics=" + stats + ", spiState=" + spiStateCopy() +
-                ", coord=" + coord +
-                ", topSize=" + ring.allNodes().size() +
-                ", leavingNodesSize=" + leavingNodesSize + ", failedNodesSize=" + failedNodesSize +
-                ", msgWorker.queue.size=" + (msgWorker != null ? msgWorker.queueSize() : "N/A") +
-                ", lastUpdate=" + (locNode != null ? U.format(locNode.lastUpdateTime()) : "N/A") +
-                ", heapFree=" + runtime.freeMemory() / (1024 * 1024) +
-                "M, heapTotal=" + runtime.maxMemory() / (1024 * 1024) + "M]");
-        }
-    }
-
-    /**
-     * @param msg Message to prepare.
-     * @param destNodeId Destination node ID.
-     * @param msgs Messages to include.
-     * @param discardMsgId Discarded message ID.
-     */
-    private void prepareNodeAddedMessage(TcpDiscoveryAbstractMessage msg, UUID destNodeId,
-        @Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardMsgId) {
-        assert destNodeId != null;
-
-        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
-            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
-
-            TcpDiscoveryNode node = nodeAddedMsg.node();
-
-            if (node.id().equals(destNodeId)) {
-                Collection<TcpDiscoveryNode> allNodes = ring.allNodes();
-                Collection<TcpDiscoveryNode> topToSend = new ArrayList<>(allNodes.size());
-
-                for (TcpDiscoveryNode n0 : allNodes) {
-                    assert n0.internalOrder() != 0 : n0;
-
-                    // Skip next node and nodes added after next
-                    // in case this message is resent due to failures/leaves.
-                    // There will be separate messages for nodes with greater
-                    // internal order.
-                    if (n0.internalOrder() < nodeAddedMsg.node().internalOrder())
-                        topToSend.add(n0);
-                }
-
-                nodeAddedMsg.topology(topToSend);
-                nodeAddedMsg.messages(msgs, discardMsgId);
-
-                Map<Long, Collection<ClusterNode>> hist;
-
-                synchronized (mux) {
-                    hist = new TreeMap<>(topHist);
-                }
-
-                nodeAddedMsg.topologyHistory(hist);
-            }
-        }
-    }
-
-    /**
-     * @param msg Message to clear.
-     */
-    private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) {
-        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
-            // Nullify topology before registration.
-            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
-
-            nodeAddedMsg.topology(null);
-            nodeAddedMsg.topologyHistory(null);
-            nodeAddedMsg.messages(null, null);
-        }
-    }
-
-    /**
-     * <strong>FOR TEST ONLY!!!</strong>
-     * <p>
-     * Simulates this node failure by stopping service threads. So, node will become
-     * unresponsive.
-     * <p>
-     * This method is intended for test purposes only.
-     */
-    void simulateNodeFailure() {
-        U.warn(log, "Simulating node failure: " + getLocalNodeId());
-
-        U.interrupt(tcpSrvr);
-        U.join(tcpSrvr, log);
-
-        U.interrupt(hbsSnd);
-        U.join(hbsSnd, log);
-
-        U.interrupt(chkStatusSnd);
-        U.join(chkStatusSnd, log);
-
-        U.interrupt(ipFinderCleaner);
-        U.join(ipFinderCleaner, log);
-
-        Collection<SocketReader> tmp;
-
-        synchronized (mux) {
-            tmp = U.arrayList(readers);
-        }
-
-        U.interrupt(tmp);
-        U.joinThreads(tmp, log);
-
-        U.interrupt(msgWorker);
-        U.join(msgWorker, log);
-
-        U.interrupt(statsPrinter);
-        U.join(statsPrinter, log);
-    }
-
-    /**
-     * <strong>FOR TEST ONLY!!!</strong>
-     * <p>
-     * Simulates situation when next node is still alive but is bypassed
-     * since it has been excluded from the ring, possibly, due to short time
-     * network problems.
-     * <p>
-     * This method is intended for test purposes only.
-     */
-    void forceNextNodeFailure() {
-        U.warn(log, "Next node will be forcibly failed (if any).");
-
-        TcpDiscoveryNode next;
-
-        synchronized (mux) {
-            next = ring.nextNode(failedNodes);
-        }
-
-        if (next != null)
-            msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(getLocalNodeId(), next.id(),
-                next.internalOrder()));
-    }
-
-    /**
-     * <strong>FOR TEST ONLY!!!</strong>
-     */
-    public void addSendMessageListener(IgniteInClosure<TcpDiscoveryAbstractMessage> msg) {
-        sendMsgLsnrs.add(msg);
-    }
-
-    /**
-     * <strong>FOR TEST ONLY!!!</strong>
-     */
-    public void removeSendMessageListener(IgniteInClosure<TcpDiscoveryAbstractMessage> msg) {
-        sendMsgLsnrs.remove(msg);
-    }
-
-    /**
-     * <strong>FOR TEST ONLY!!!</strong>
-     * <p>
-     * This method is intended for test purposes only.
-     *
-     * @return Nodes ring.
-     */
-    TcpDiscoveryNodesRing ring() {
-        return ring;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void dumpDebugInfo() {
-        dumpDebugInfo(log);
-    }
-
-    /**
-     * @param log Logger.
-     */
-    public void dumpDebugInfo(IgniteLogger log) {
-        if (!debugMode) {
-            U.quietAndWarn(log, "Failed to dump debug info (discovery SPI was not configured " +
-                "in debug mode, consider setting 'debugMode' configuration property to 'true').");
-
-            return;
-        }
-
-        assert log.isInfoEnabled();
-
-        synchronized (mux) {
-            StringBuilder b = new StringBuilder(U.nl());
-
-            b.append(">>>").append(U.nl());
-            b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl());
-            b.append(">>>").append(U.nl());
-
-            b.append("Local node ID: ").append(getLocalNodeId()).append(U.nl()).append(U.nl());
-            b.append("Local node: ").append(locNode).append(U.nl()).append(U.nl());
-            b.append("SPI state: ").append(spiState).append(U.nl()).append(U.nl());
-
-            b.append("Internal threads: ").append(U.nl());
-
-            b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
-            b.append("    Check status sender: ").append(threadStatus(chkStatusSnd)).append(U.nl());
-            b.append("    HB sender: ").append(threadStatus(hbsSnd)).append(U.nl());
-            b.append("    Socket timeout worker: ").append(threadStatus(sockTimeoutWorker)).append(U.nl());
-            b.append("    IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl());
-            b.append("    Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl());
-
-            b.append(U.nl());
-
-            b.append("Socket readers: ").append(U.nl());
-
-            for (SocketReader rdr : readers)
-                b.append("    ").append(rdr).append(U.nl());
-
-            b.append(U.nl());
-
-            b.append("In-memory log messages: ").append(U.nl());
-
-            for (String msg : debugLog)
-                b.append("    ").append(msg).append(U.nl());
-
-            b.append(U.nl());
-
-            b.append("Leaving nodes: ").append(U.nl());
-
-            for (TcpDiscoveryNode node : leavingNodes)
-                b.append("    ").append(node.id()).append(U.nl());
-
-            b.append(U.nl());
-
-            b.append("Failed nodes: ").append(U.nl());
-
-            for (TcpDiscoveryNode node : failedNodes)
-                b.append("    ").append(node.id()).append(U.nl());
-
-            b.append(U.nl());
-
-            b.append("Stats: ").append(stats).append(U.nl());
-
-            U.quietAndInfo(log, b.toString());
-        }
-    }
-
-    /**
-     * @param msg Message.
-     */
-    private void debugLog(String msg) {
-        assert debugMode;
-
-        String msg0 = new SimpleDateFormat("[HH:mm:ss,SSS]").format(new Date(System.currentTimeMillis())) +
-            '[' + Thread.currentThread().getName() + "][" + getLocalNodeId() +
-            "-" + locNode.internalOrder() + "] " +
-            msg;
-
-        debugLog.add(msg0);
-
-        int delta = debugLog.size() - debugMsgHist;
-
-        for (int i = 0; i < delta && debugLog.size() > debugMsgHist; i++)
-            debugLog.poll();
-    }
-
-    /**
-     * @param msg Message.
-     * @return {@code True} if recordable in debug mode.
-     */
-    private boolean recordable(TcpDiscoveryAbstractMessage msg) {
-        return !(msg instanceof TcpDiscoveryHeartbeatMessage) &&
-            !(msg instanceof TcpDiscoveryStatusCheckMessage) &&
-            !(msg instanceof TcpDiscoveryDiscardMessage);
-    }
-
-    /**
-     * @param t Thread.
-     * @return Status as string.
-     */
-    private String threadStatus(Thread t) {
-        if (t == null)
-            return "N/A";
-
-        return t.isAlive() ? "alive" : "dead";
-    }
-
-    /**
-     * Checks if two given {@link SecurityPermissionSet} 

<TRUNCATED>


[13/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
deleted file mode 100644
index 802da02..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
+++ /dev/null
@@ -1,1160 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spi.discovery.tcp;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.jdk.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.*;
-import org.apache.ignite.spi.discovery.tcp.internal.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.messages.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.*;
-
-/**
- * Base class for TCP discovery SPIs.
- */
-abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements DiscoverySpi {
-    /** Default port to listen (value is <tt>47500</tt>). */
-    public static final int DFLT_PORT = 47500;
-
-    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT = 200;
-
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT = 50;
-
-    /** Default network timeout in milliseconds (value is <tt>5000ms</tt>). */
-    public static final long DFLT_NETWORK_TIMEOUT = 5000;
-
-    /** Default value for thread priority (value is <tt>10</tt>). */
-    public static final int DFLT_THREAD_PRI = 10;
-
-    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 100;
-
-    /** Default size of topology snapshots history. */
-    public static final int DFLT_TOP_HISTORY_SIZE = 1000;
-
-    /** Response OK. */
-    protected static final int RES_OK = 1;
-
-    /** Response CONTINUE JOIN. */
-    protected static final int RES_CONTINUE_JOIN = 100;
-
-    /** Response WAIT. */
-    protected static final int RES_WAIT = 200;
-
-    /** Local address. */
-    protected String locAddr;
-
-    /** IP finder. */
-    protected TcpDiscoveryIpFinder ipFinder;
-
-    /** Socket operations timeout. */
-    protected long sockTimeout = DFLT_SOCK_TIMEOUT;
-
-    /** Message acknowledgement timeout. */
-    protected long ackTimeout = DFLT_ACK_TIMEOUT;
-
-    /** Network timeout. */
-    protected long netTimeout = DFLT_NETWORK_TIMEOUT;
-
-    /** Thread priority for all threads started by SPI. */
-    protected int threadPri = DFLT_THREAD_PRI;
-
-    /** Heartbeat messages issuing frequency. */
-    protected long hbFreq = DFLT_HEARTBEAT_FREQ;
-
-    /** Size of topology snapshots history. */
-    protected int topHistSize = DFLT_TOP_HISTORY_SIZE;
-
-    /** Grid discovery listener. */
-    protected volatile DiscoverySpiListener lsnr;
-
-    /** Data exchange. */
-    protected DiscoverySpiDataExchange exchange;
-
-    /** Metrics provider. */
-    protected DiscoveryMetricsProvider metricsProvider;
-
-    /** Local node attributes. */
-    protected Map<String, Object> locNodeAttrs;
-
-    /** Local node version. */
-    protected IgniteProductVersion locNodeVer;
-
-    /** Local node. */
-    protected TcpDiscoveryNode locNode;
-
-    /** Local host. */
-    protected InetAddress locHost;
-
-    /** Internal and external addresses of local node. */
-    protected Collection<InetSocketAddress> locNodeAddrs;
-
-    /** Socket timeout worker. */
-    protected SocketTimeoutWorker sockTimeoutWorker;
-
-    /** Discovery state. */
-    protected TcpDiscoverySpiState spiState = DISCONNECTED;
-
-    /** Start time of the very first grid node. */
-    protected volatile long gridStartTime;
-
-    /** Marshaller. */
-    protected final Marshaller marsh = new JdkMarshaller();
-
-    /** Statistics. */
-    protected final TcpDiscoveryStatistics stats = new TcpDiscoveryStatistics();
-
-    /** Logger. */
-    @LoggerResource
-    protected IgniteLogger log;
-
-    /**
-     * Inject resources
-     *
-     * @param ignite Ignite.
-     */
-    @IgniteInstanceResource
-    @Override protected void injectResources(Ignite ignite) {
-        super.injectResources(ignite);
-
-        // Inject resource.
-        if (ignite != null)
-            setLocalAddress(ignite.configuration().getLocalHost());
-    }
-
-    /**
-     * Sets local host IP address that discovery SPI uses.
-     * <p>
-     * If not provided, by default a first found non-loopback address
-     * will be used. If there is no non-loopback address available,
-     * then {@link InetAddress#getLocalHost()} will be used.
-     *
-     * @param locAddr IP address.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setLocalAddress(String locAddr) {
-        // Injection should not override value already set by Spring or user.
-        if (this.locAddr == null)
-            this.locAddr = locAddr;
-    }
-
-    /**
-     * Gets local address that was set to SPI with {@link #setLocalAddress(String)} method.
-     *
-     * @return local address.
-     */
-    public String getLocalAddress() {
-        return locAddr;
-    }
-
-    /**
-     * Gets IP finder for IP addresses sharing and storing.
-     *
-     * @return IP finder for IP addresses sharing and storing.
-     */
-    public TcpDiscoveryIpFinder getIpFinder() {
-        return ipFinder;
-    }
-
-    /**
-     * Sets IP finder for IP addresses sharing and storing.
-     * <p>
-     * If not provided {@link org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder} will be used by default.
-     *
-     * @param ipFinder IP finder.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setIpFinder(TcpDiscoveryIpFinder ipFinder) {
-        this.ipFinder = ipFinder;
-    }
-
-    /**
-     * Sets socket operations timeout. This timeout is used to limit connection time and
-     * write-to-socket time.
-     * <p>
-     * Note that when running Ignite on Amazon EC2, socket timeout must be set to a value
-     * significantly greater than the default (e.g. to {@code 30000}).
-     * <p>
-     * If not specified, default is {@link #DFLT_SOCK_TIMEOUT}.
-     *
-     * @param sockTimeout Socket connection timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setSocketTimeout(long sockTimeout) {
-        this.sockTimeout = sockTimeout;
-    }
-
-    /**
-     * Sets timeout for receiving acknowledgement for sent message.
-     * <p>
-     * If acknowledgement is not received within this timeout, sending is considered as failed
-     * and SPI tries to repeat message sending.
-     * <p>
-     * If not specified, default is {@link #DFLT_ACK_TIMEOUT}.
-     *
-     * @param ackTimeout Acknowledgement timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setAckTimeout(long ackTimeout) {
-        this.ackTimeout = ackTimeout;
-    }
-
-    /**
-     * Sets maximum network timeout to use for network operations.
-     * <p>
-     * If not specified, default is {@link #DFLT_NETWORK_TIMEOUT}.
-     *
-     * @param netTimeout Network timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setNetworkTimeout(long netTimeout) {
-        this.netTimeout = netTimeout;
-    }
-
-    /**
-     * Sets thread priority. All threads within SPI will be started with it.
-     * <p>
-     * If not provided, default value is {@link #DFLT_THREAD_PRI}
-     *
-     * @param threadPri Thread priority.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setThreadPriority(int threadPri) {
-        this.threadPri = threadPri;
-    }
-
-    /**
-     * Sets delay between issuing of heartbeat messages. SPI sends heartbeat messages
-     * in configurable time interval to other nodes to notify them about its state.
-     * <p>
-     * If not provided, default value is {@link #DFLT_HEARTBEAT_FREQ}.
-     *
-     * @param hbFreq Heartbeat frequency in milliseconds.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setHeartbeatFrequency(long hbFreq) {
-        this.hbFreq = hbFreq;
-    }
-
-    /**
-     * @return Size of topology snapshots history.
-     */
-    public long getTopHistorySize() {
-        return topHistSize;
-    }
-
-    /**
-     * Sets size of topology snapshots history. Specified size should be greater than or equal to default size
-     * {@link #DFLT_TOP_HISTORY_SIZE}.
-     *
-     * @param topHistSize Size of topology snapshots history.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setTopHistorySize(int topHistSize) {
-        if (topHistSize < DFLT_TOP_HISTORY_SIZE) {
-            U.warn(log, "Topology history size should be greater than or equal to default size. " +
-                "Specified size will not be set [curSize=" + this.topHistSize + ", specifiedSize=" + topHistSize +
-                ", defaultSize=" + DFLT_TOP_HISTORY_SIZE + ']');
-
-            return;
-        }
-
-        this.topHistSize = topHistSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
-        assert locNodeAttrs == null;
-        assert locNodeVer == null;
-
-        if (log.isDebugEnabled()) {
-            log.debug("Node attributes to set: " + attrs);
-            log.debug("Node version to set: " + ver);
-        }
-
-        locNodeAttrs = attrs;
-        locNodeVer = ver;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        super.onContextInitialized0(spiCtx);
-
-        ipFinder.onSpiContextInitialized(spiCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        super.onContextDestroyed0();
-
-        if (ipFinder != null)
-            ipFinder.onSpiContextDestroyed();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getLocalNode() {
-        return locNode;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
-        this.lsnr = lsnr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDataExchange(DiscoverySpiDataExchange exchange) {
-        this.exchange = exchange;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
-        this.metricsProvider = metricsProvider;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getGridStartTime() {
-        assert gridStartTime != 0;
-
-        return gridStartTime;
-    }
-
-    /**
-     * @param sockAddr Remote address.
-     * @return Opened socket.
-     * @throws IOException If failed.
-     */
-    protected Socket openSocket(InetSocketAddress sockAddr) throws IOException {
-        assert sockAddr != null;
-
-        InetSocketAddress resolved = sockAddr.isUnresolved() ?
-            new InetSocketAddress(InetAddress.getByName(sockAddr.getHostName()), sockAddr.getPort()) : sockAddr;
-
-        InetAddress addr = resolved.getAddress();
-
-        assert addr != null;
-
-        Socket sock = new Socket();
-
-        sock.bind(new InetSocketAddress(locHost, 0));
-
-        sock.setTcpNoDelay(true);
-
-        sock.connect(resolved, (int)sockTimeout);
-
-        writeToSocket(sock, U.IGNITE_HEADER);
-
-        return sock;
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param data Raw data to write.
-     * @throws IOException If IO failed or write timed out.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, byte[] data) throws IOException {
-        assert sock != null;
-        assert data != null;
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        IOException err = null;
-
-        try {
-            OutputStream out = sock.getOutputStream();
-
-            out.write(data);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param msg Message.
-     * @throws IOException If IO failed or write timed out.
-     * @throws IgniteCheckedException If marshalling failed.
-     */
-    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg) throws IOException, IgniteCheckedException {
-        writeToSocket(sock, msg, new GridByteArrayOutputStream(8 * 1024)); // 8K.
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param msg Message.
-     * @param bout Byte array output stream.
-     * @throws IOException If IO failed or write timed out.
-     * @throws IgniteCheckedException If marshalling failed.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, GridByteArrayOutputStream bout)
-        throws IOException, IgniteCheckedException {
-        assert sock != null;
-        assert msg != null;
-        assert bout != null;
-
-        // Marshall message first to perform only write after.
-        marsh.marshal(msg, bout);
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        IOException err = null;
-
-        try {
-            OutputStream out = sock.getOutputStream();
-
-            bout.writeTo(out);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Writes response to the socket.
-     *
-     * @param sock Socket.
-     * @param res Integer response.
-     * @throws IOException If IO failed or write timed out.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, int res) throws IOException {
-        assert sock != null;
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        OutputStream out = sock.getOutputStream();
-
-        IOException err = null;
-
-        try {
-            out.write(res);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Reads message from the socket limiting read time.
-     *
-     * @param sock Socket.
-     * @param in Input stream (in case socket stream was wrapped).
-     * @param timeout Socket timeout for this operation.
-     * @return Message.
-     * @throws IOException If IO failed or read timed out.
-     * @throws IgniteCheckedException If unmarshalling failed.
-     */
-    protected <T> T readMessage(Socket sock, @Nullable InputStream in, long timeout) throws IOException, IgniteCheckedException {
-        assert sock != null;
-
-        int oldTimeout = sock.getSoTimeout();
-
-        try {
-            sock.setSoTimeout((int)timeout);
-
-            return marsh.unmarshal(in == null ? sock.getInputStream() : in, U.gridClassLoader());
-        }
-        catch (IOException | IgniteCheckedException e) {
-            if (X.hasCause(e, SocketTimeoutException.class))
-                LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " +
-                    "in long GC pauses on remote node. Current timeout: " + timeout + '.');
-
-            throw e;
-        }
-        finally {
-            // Quietly restore timeout.
-            try {
-                sock.setSoTimeout(oldTimeout);
-            }
-            catch (SocketException ignored) {
-                // No-op.
-            }
-        }
-    }
-
-    /**
-     * Reads message delivery receipt from the socket.
-     *
-     * @param sock Socket.
-     * @param timeout Socket timeout for this operation.
-     * @return Receipt.
-     * @throws IOException If IO failed or read timed out.
-     */
-    protected int readReceipt(Socket sock, long timeout) throws IOException {
-        assert sock != null;
-
-        int oldTimeout = sock.getSoTimeout();
-
-        try {
-            sock.setSoTimeout((int)timeout);
-
-            int res = sock.getInputStream().read();
-
-            if (res == -1)
-                throw new EOFException();
-
-            return res;
-        }
-        catch (SocketTimeoutException e) {
-            LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " +
-                "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " +
-                "configuration property). Will retry to send message with increased timeout. " +
-                "Current timeout: " + timeout + '.');
-
-            stats.onAckTimeout();
-
-            throw e;
-        }
-        finally {
-            // Quietly restore timeout.
-            try {
-                sock.setSoTimeout(oldTimeout);
-            }
-            catch (SocketException ignored) {
-                // No-op.
-            }
-        }
-    }
-
-    /**
-     * Resolves addresses registered in the IP finder, removes duplicates and local host
-     * address and returns the collection of.
-     *
-     * @return Resolved addresses without duplicates and local address (potentially
-     *      empty but never null).
-     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
-     */
-    protected Collection<InetSocketAddress> resolvedAddresses() throws IgniteSpiException {
-        List<InetSocketAddress> res = new ArrayList<>();
-
-        Collection<InetSocketAddress> addrs;
-
-        // Get consistent addresses collection.
-        while (true) {
-            try {
-                addrs = registeredAddresses();
-
-                break;
-            }
-            catch (IgniteSpiException e) {
-                LT.error(log, e, "Failed to get registered addresses from IP finder on start " +
-                    "(retrying every 2000 ms).");
-            }
-
-            try {
-                U.sleep(2000);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-        }
-
-        for (InetSocketAddress addr : addrs) {
-            assert addr != null;
-
-            try {
-                InetSocketAddress resolved = addr.isUnresolved() ?
-                    new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort()) : addr;
-
-                if (locNodeAddrs == null || !locNodeAddrs.contains(resolved))
-                    res.add(resolved);
-            }
-            catch (UnknownHostException ignored) {
-                LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr);
-
-                // Add address in any case.
-                res.add(addr);
-            }
-        }
-
-        if (!res.isEmpty())
-            Collections.shuffle(res);
-
-        return res;
-    }
-
-    /**
-     * Gets addresses registered in the IP finder, initializes addresses having no
-     * port (or 0 port) with {@link #DFLT_PORT}.
-     *
-     * @return Registered addresses.
-     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
-     */
-    protected Collection<InetSocketAddress> registeredAddresses() throws IgniteSpiException {
-        Collection<InetSocketAddress> res = new ArrayList<>();
-
-        for (InetSocketAddress addr : ipFinder.getRegisteredAddresses()) {
-            if (addr.getPort() == 0) {
-                // TcpDiscoveryNode.discoveryPort() returns an correct port for a server node and 0 for client node.
-                int port = locNode.discoveryPort() != 0 ? locNode.discoveryPort() : DFLT_PORT;
-
-                addr = addr.isUnresolved() ? new InetSocketAddress(addr.getHostName(), port) :
-                    new InetSocketAddress(addr.getAddress(), port);
-            }
-
-            res.add(addr);
-        }
-
-        return res;
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException duplicateIdError(TcpDiscoveryDuplicateIdMessage msg) {
-        assert msg != null;
-
-        return new IgniteSpiException("Local node has the same ID as existing node in topology " +
-            "(fix configuration and restart local node) [localNode=" + locNode +
-            ", existingNode=" + msg.node() + ']');
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException authenticationFailedError(TcpDiscoveryAuthFailedMessage msg) {
-        assert msg != null;
-
-        return new IgniteSpiException(new IgniteAuthenticationException("Authentication failed [nodeId=" +
-            msg.creatorNodeId() + ", addr=" + msg.address().getHostAddress() + ']'));
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException checkFailedError(TcpDiscoveryCheckFailedMessage msg) {
-        assert msg != null;
-
-        return versionCheckFailed(msg) ? new IgniteSpiVersionCheckException(msg.error()) :
-            new IgniteSpiException(msg.error());
-    }
-
-    /**
-     * @param msg Message.
-     * @return Whether delivery of the message is ensured.
-     */
-    protected boolean ensured(TcpDiscoveryAbstractMessage msg) {
-        return U.getAnnotation(msg.getClass(), TcpDiscoveryEnsureDelivery.class) != null;
-    }
-
-    /**
-     * @param msg Failed message.
-     * @return {@code True} if specified failed message relates to version incompatibility, {@code false} otherwise.
-     * @deprecated Parsing of error message was used for preserving backward compatibility. We should remove it
-     *      and create separate message for failed version check with next major release.
-     */
-    @Deprecated
-    private static boolean versionCheckFailed(TcpDiscoveryCheckFailedMessage msg) {
-        return msg.error().contains("versions are not compatible");
-    }
-
-    /**
-     * @param joiningNodeID Joining node ID.
-     * @param nodeId Remote node ID for which data is provided.
-     * @param data Collection of marshalled discovery data objects from different components.
-     * @param clsLdr Class loader for discovery data unmarshalling.
-     */
-    protected void onExchange(UUID joiningNodeID,
-        UUID nodeId,
-        Map<Integer, byte[]> data,
-        ClassLoader clsLdr)
-    {
-        Map<Integer, Serializable> data0 = U.newHashMap(data.size());
-
-        for (Map.Entry<Integer, byte[]> entry : data.entrySet()) {
-            try {
-                Serializable compData = marsh.unmarshal(entry.getValue(), clsLdr);
-
-                data0.put(entry.getKey(), compData);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to unmarshal discovery data for component: "  + entry.getKey(), e);
-            }
-        }
-
-        exchange.onExchange(joiningNodeID, nodeId, data0);
-    }
-
-    /**
-     * Handles sockets timeouts.
-     */
-    protected class SocketTimeoutWorker extends IgniteSpiThread {
-        /** Time-based sorted set for timeout objects. */
-        private final GridConcurrentSkipListSet<SocketTimeoutObject> timeoutObjs =
-            new GridConcurrentSkipListSet<>(new Comparator<SocketTimeoutObject>() {
-                @Override public int compare(SocketTimeoutObject o1, SocketTimeoutObject o2) {
-                    long time1 = o1.endTime();
-                    long time2 = o2.endTime();
-
-                    long id1 = o1.id();
-                    long id2 = o2.id();
-
-                    return time1 < time2 ? -1 : time1 > time2 ? 1 :
-                        id1 < id2 ? -1 : id1 > id2 ? 1 : 0;
-                }
-            });
-
-        /** Mutex. */
-        private final Object mux0 = new Object();
-
-        /**
-         *
-         */
-        SocketTimeoutWorker() {
-            super(gridName, "tcp-disco-sock-timeout-worker", log);
-
-            setPriority(threadPri);
-        }
-
-        /**
-         * @param timeoutObj Timeout object to add.
-         */
-        @SuppressWarnings({"NakedNotify"})
-        public void addTimeoutObject(SocketTimeoutObject timeoutObj) {
-            assert timeoutObj != null && timeoutObj.endTime() > 0 && timeoutObj.endTime() != Long.MAX_VALUE;
-
-            timeoutObjs.add(timeoutObj);
-
-            if (timeoutObjs.firstx() == timeoutObj) {
-                synchronized (mux0) {
-                    mux0.notifyAll();
-                }
-            }
-        }
-
-        /**
-         * @param timeoutObj Timeout object to remove.
-         */
-        public void removeTimeoutObject(SocketTimeoutObject timeoutObj) {
-            assert timeoutObj != null;
-
-            timeoutObjs.remove(timeoutObj);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            if (log.isDebugEnabled())
-                log.debug("Socket timeout worker has been started.");
-
-            while (!isInterrupted()) {
-                long now = U.currentTimeMillis();
-
-                for (Iterator<SocketTimeoutObject> iter = timeoutObjs.iterator(); iter.hasNext(); ) {
-                    SocketTimeoutObject timeoutObj = iter.next();
-
-                    if (timeoutObj.endTime() <= now) {
-                        iter.remove();
-
-                        if (timeoutObj.onTimeout()) {
-                            LT.warn(log, null, "Socket write has timed out (consider increasing " +
-                                "'sockTimeout' configuration property) [sockTimeout=" + sockTimeout + ']');
-
-                            stats.onSocketTimeout();
-                        }
-                    }
-                    else
-                        break;
-                }
-
-                synchronized (mux0) {
-                    while (true) {
-                        // Access of the first element must be inside of
-                        // synchronization block, so we don't miss out
-                        // on thread notification events sent from
-                        // 'addTimeoutObject(..)' method.
-                        SocketTimeoutObject first = timeoutObjs.firstx();
-
-                        if (first != null) {
-                            long waitTime = first.endTime() - U.currentTimeMillis();
-
-                            if (waitTime > 0)
-                                mux0.wait(waitTime);
-                            else
-                                break;
-                        }
-                        else
-                            mux0.wait(5000);
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Socket timeout object.
-     */
-    private static class SocketTimeoutObject {
-        /** */
-        private static final AtomicLong idGen = new AtomicLong();
-
-        /** */
-        private final long id = idGen.incrementAndGet();
-
-        /** */
-        private final Socket sock;
-
-        /** */
-        private final long endTime;
-
-        /** */
-        private final AtomicBoolean done = new AtomicBoolean();
-
-        /**
-         * @param sock Socket.
-         * @param endTime End time.
-         */
-        SocketTimeoutObject(Socket sock, long endTime) {
-            assert sock != null;
-            assert endTime > 0;
-
-            this.sock = sock;
-            this.endTime = endTime;
-        }
-
-        /**
-         * @return {@code True} if object has not yet been processed.
-         */
-        boolean cancel() {
-            return done.compareAndSet(false, true);
-        }
-
-        /**
-         * @return {@code True} if object has not yet been canceled.
-         */
-        boolean onTimeout() {
-            if (done.compareAndSet(false, true)) {
-                // Close socket - timeout occurred.
-                U.closeQuiet(sock);
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
-         * @return End time.
-         */
-        long endTime() {
-            return endTime;
-        }
-
-        /**
-         * @return ID.
-         */
-        long id() {
-            return id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(SocketTimeoutObject.class, this);
-        }
-    }
-
-    /**
-     * Base class for message workers.
-     */
-    protected abstract class MessageWorkerAdapter extends IgniteSpiThread {
-        /** Pre-allocated output stream (100K). */
-        private final GridByteArrayOutputStream bout = new GridByteArrayOutputStream(100 * 1024);
-
-        /** Message queue. */
-        private final BlockingDeque<TcpDiscoveryAbstractMessage> queue = new LinkedBlockingDeque<>();
-
-        /** Backed interrupted flag. */
-        private volatile boolean interrupted;
-
-        /**
-         * @param name Thread name.
-         */
-        protected MessageWorkerAdapter(String name) {
-            super(gridName, name, log);
-
-            setPriority(threadPri);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            if (log.isDebugEnabled())
-                log.debug("Message worker started [locNodeId=" + getLocalNodeId() + ']');
-
-            while (!isInterrupted()) {
-                TcpDiscoveryAbstractMessage msg = queue.poll(2000, TimeUnit.MILLISECONDS);
-
-                if (msg == null)
-                    continue;
-
-                processMessage(msg);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void interrupt() {
-            interrupted = true;
-
-            super.interrupt();
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean isInterrupted() {
-            return interrupted || super.isInterrupted();
-        }
-
-        /**
-         * @return Current queue size.
-         */
-        int queueSize() {
-            return queue.size();
-        }
-
-        /**
-         * Adds message to queue.
-         *
-         * @param msg Message to add.
-         */
-        void addMessage(TcpDiscoveryAbstractMessage msg) {
-            assert msg != null;
-
-            if (msg instanceof TcpDiscoveryHeartbeatMessage)
-                queue.addFirst(msg);
-            else
-                queue.add(msg);
-
-            if (log.isDebugEnabled())
-                log.debug("Message has been added to queue: " + msg);
-        }
-
-        /**
-         * @param msg Message.
-         */
-        protected abstract void processMessage(TcpDiscoveryAbstractMessage msg);
-
-        /**
-         * @param sock Socket.
-         * @param msg Message.
-         * @throws IOException If IO failed.
-         * @throws IgniteCheckedException If marshalling failed.
-         */
-        protected final void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg)
-            throws IOException, IgniteCheckedException {
-            bout.reset();
-
-            TcpDiscoverySpiAdapter.this.writeToSocket(sock, msg, bout);
-        }
-    }
-
-    /**
-     *
-     */
-    protected class SocketMultiConnector implements AutoCloseable {
-        /** */
-        private int connInProgress;
-
-        /** */
-        private final ExecutorService executor;
-
-        /** */
-        private final CompletionService<GridTuple3<InetSocketAddress, Socket, Exception>> completionSrvc;
-
-        /**
-         * @param addrs Addresses.
-         * @param retryCnt Retry count.
-         */
-        public SocketMultiConnector(Collection<InetSocketAddress> addrs, final int retryCnt) {
-            connInProgress = addrs.size();
-
-            executor = Executors.newFixedThreadPool(Math.min(1, addrs.size()));
-
-            completionSrvc = new ExecutorCompletionService<>(executor);
-
-            for (final InetSocketAddress addr : addrs) {
-                completionSrvc.submit(new Callable<GridTuple3<InetSocketAddress, Socket, Exception>>() {
-                    @Override public GridTuple3<InetSocketAddress, Socket, Exception> call() {
-                        Exception ex = null;
-                        Socket sock = null;
-
-                        for (int i = 0; i < retryCnt; i++) {
-                            if (Thread.currentThread().isInterrupted())
-                                return null; // Executor is shutdown.
-
-                            try {
-                                sock = openSocket(addr);
-
-                                break;
-                            }
-                            catch (Exception e) {
-                                ex = e;
-                            }
-                        }
-
-                        return new GridTuple3<>(addr, sock, ex);
-                    }
-                });
-            }
-        }
-
-        /**
-         *
-         */
-        @Nullable public GridTuple3<InetSocketAddress, Socket, Exception> next() {
-            if (connInProgress == 0)
-                return null;
-
-            try {
-                Future<GridTuple3<InetSocketAddress, Socket, Exception>> fut = completionSrvc.take();
-
-                connInProgress--;
-
-                return fut.get();
-            }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-            catch (ExecutionException e) {
-                throw new IgniteSpiException(e);
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override public void close() {
-            List<Runnable> unstartedTasks = executor.shutdownNow();
-
-            connInProgress -= unstartedTasks.size();
-
-            if (connInProgress > 0) {
-                Thread thread = new Thread(new Runnable() {
-                    @Override public void run() {
-                        try {
-                            executor.awaitTermination(5, TimeUnit.MINUTES);
-
-                            Future<GridTuple3<InetSocketAddress, Socket, Exception>> fut;
-
-                            while ((fut = completionSrvc.poll()) != null) {
-                                try {
-                                    GridTuple3<InetSocketAddress, Socket, Exception> tuple3 = fut.get();
-
-                                    if (tuple3 != null)
-                                        IgniteUtils.closeQuiet(tuple3.get2());
-                                }
-                                catch (ExecutionException ignore) {
-
-                                }
-                            }
-                        }
-                        catch (InterruptedException e) {
-                            Thread.currentThread().interrupt();
-
-                            throw new RuntimeException(e);
-                        }
-                    }
-                });
-
-                thread.setDaemon(true);
-
-                thread.start();
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
index df9d0f4..f338fab 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
@@ -272,4 +272,13 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Dump debug info.")
     public void dumpDebugInfo();
+
+    /**
+     * Whether or not discovery is started in client mode.
+     *
+     * @return {@code true} if node is in client mode.
+     * @throws IllegalStateException If discovery SPI is not started.
+     */
+    @MXBeanDescription("Client mode.")
+    public boolean isClientMode() throws IllegalStateException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index bb8f051..cc61c9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -448,11 +448,8 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
 
         ClusterMetrics metrics = this.metrics;
 
-        if (metrics != null) {
-            mtr = new byte[ClusterMetricsSnapshot.METRICS_SIZE];
-
-            ClusterMetricsSnapshot.serialize(mtr, 0, metrics);
-        }
+        if (metrics != null)
+            mtr = ClusterMetricsSnapshot.serialize(metrics);
 
         U.writeByteArray(out, mtr);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
index e866504..e9eaa1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNodesRing.java
@@ -32,7 +32,7 @@ import java.util.concurrent.locks.*;
  */
 public class TcpDiscoveryNodesRing {
     /** Visible nodes filter. */
-    private static final IgnitePredicate<TcpDiscoveryNode> VISIBLE_NODES = new P1<TcpDiscoveryNode>() {
+    public static final IgnitePredicate<TcpDiscoveryNode> VISIBLE_NODES = new P1<TcpDiscoveryNode>() {
         @Override public boolean apply(TcpDiscoveryNode node) {
             return node.visible();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
index 51ad7b4..95758e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/TcpDiscoveryIpFinder.java
@@ -31,7 +31,7 @@ public interface TcpDiscoveryIpFinder {
      * method is completed, SPI context can be stored for future access.
      *
      * @param spiCtx Spi context.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void onSpiContextInitialized(IgniteSpiContext spiCtx) throws IgniteSpiException;
 
@@ -46,7 +46,7 @@ public interface TcpDiscoveryIpFinder {
      * Initializes addresses discovery SPI binds to.
      *
      * @param addrs Addresses discovery SPI binds to.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void initializeLocalAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 
@@ -54,7 +54,7 @@ public interface TcpDiscoveryIpFinder {
      * Gets all addresses registered in this finder.
      *
      * @return All known addresses, potentially empty, but never {@code null}.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public Collection<InetSocketAddress> getRegisteredAddresses() throws IgniteSpiException;
 
@@ -76,7 +76,7 @@ public interface TcpDiscoveryIpFinder {
      * is already registered.
      *
      * @param addrs Addresses to register. Not {@code null} and not empty.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void registerAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 
@@ -87,7 +87,7 @@ public interface TcpDiscoveryIpFinder {
      * registered quietly (just no-op).
      *
      * @param addrs Addresses to unregister. Not {@code null} and not empty.
-     * @throws org.apache.ignite.spi.IgniteSpiException In case of error.
+     * @throws IgniteSpiException In case of error.
      */
     public void unregisterAddresses(Collection<InetSocketAddress> addrs) throws IgniteSpiException;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 6cf06ab..a992620 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -26,6 +26,8 @@ import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.jetbrains.annotations.*;
 
@@ -254,6 +256,20 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                 "(it is recommended in production to specify at least one address in " +
                 "TcpDiscoveryMulticastIpFinder.getAddresses() configuration property)");
 
+        boolean clientMode;
+
+        if (ignite != null) { // Can be null if used in tests without starting Ignite.
+            DiscoverySpi discoSpi = ignite.configuration().getDiscoverySpi();
+
+            if (!(discoSpi instanceof TcpDiscoverySpi))
+                throw new IgniteSpiException("TcpDiscoveryMulticastIpFinder should be used with " +
+                    "TcpDiscoverySpi: " + discoSpi);
+
+            clientMode = ((TcpDiscoverySpi)discoSpi).isClientMode();
+        }
+        else
+            clientMode = false;
+
         InetAddress mcastAddr;
 
         try {
@@ -296,7 +312,8 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
             if (!addr.isLoopbackAddress()) {
                 try {
-                    addrSnds.add(new AddressSender(mcastAddr, addr, addrs));
+                    if (!clientMode)
+                        addrSnds.add(new AddressSender(mcastAddr, addr, addrs));
 
                     reqItfs.add(addr);
                 }
@@ -309,20 +326,24 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
             }
         }
 
-        if (addrSnds.isEmpty()) {
-            try {
-                // Create non-bound socket if local host is loopback or failed to create sockets explicitly
-                // bound to interfaces.
-                addrSnds.add(new AddressSender(mcastAddr, null, addrs));
-            }
-            catch (IOException e) {
-                throw new IgniteSpiException("Failed to create multicast socket [mcastAddr=" + mcastAddr +
-                    ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ']', e);
+        if (!clientMode) {
+            if (addrSnds.isEmpty()) {
+                try {
+                    // Create non-bound socket if local host is loopback or failed to create sockets explicitly
+                    // bound to interfaces.
+                    addrSnds.add(new AddressSender(mcastAddr, null, addrs));
+                }
+                catch (IOException e) {
+                    throw new IgniteSpiException("Failed to create multicast socket [mcastAddr=" + mcastAddr +
+                        ", mcastGrp=" + mcastGrp + ", mcastPort=" + mcastPort + ']', e);
+                }
             }
-        }
 
-        for (AddressSender addrSnd :addrSnds)
-            addrSnd.start();
+            for (AddressSender addrSnd : addrSnds)
+                addrSnd.start();
+        }
+        else
+            assert addrSnds.isEmpty() : addrSnds;
 
         Collection<InetSocketAddress> ret;
 
@@ -495,11 +516,13 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
     /** {@inheritDoc} */
     @Override public void close() {
-        for (AddressSender addrSnd : addrSnds)
-            U.interrupt(addrSnd);
+        if (addrSnds != null) {
+            for (AddressSender addrSnd : addrSnds)
+                U.interrupt(addrSnd);
 
-        for (AddressSender addrSnd : addrSnds)
-            U.join(addrSnd, log);
+            for (AddressSender addrSnd : addrSnds)
+                U.join(addrSnd, log);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
index 1a00359..145b518 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
@@ -52,9 +52,6 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
     /** Topology version. */
     private long topVer;
 
-    /** Destination client node ID. */
-    private UUID destClientNodeId;
-
     /** Flags. */
     @GridToStringExclude
     private int flags;
@@ -178,20 +175,6 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
     }
 
     /**
-     * @return Destination client node ID.
-     */
-    public UUID destinationClientNodeId() {
-        return destClientNodeId;
-    }
-
-    /**
-     * @param destClientNodeId Destination client node ID.
-     */
-    public void destinationClientNodeId(UUID destClientNodeId) {
-        this.destClientNodeId = destClientNodeId;
-    }
-
-    /**
      * @return Pending message index.
      */
     public short pendingIndex() {
@@ -232,6 +215,13 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
             flags &= ~mask;
     }
 
+    /**
+     * @return {@code true} if message must be added to head of queue.
+     */
+    public boolean highPriority() {
+        return false;
+    }
+
     /** {@inheritDoc} */
     @Override public final boolean equals(Object obj) {
         if (this == obj)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java
new file mode 100644
index 0000000..95ac340
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientHeartbeatMessage.java
@@ -0,0 +1,67 @@
+/*
+ * 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.spi.discovery.tcp.messages;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+
+/**
+ * Heartbeat message.
+ * <p>
+ * Client sends his hearbeats in this message.
+ */
+public class TcpDiscoveryClientHeartbeatMessage extends TcpDiscoveryAbstractMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final byte[] metrics;
+
+    /**
+     * Constructor.
+     *
+     * @param creatorNodeId Creator node.
+     */
+    public TcpDiscoveryClientHeartbeatMessage(UUID creatorNodeId, ClusterMetrics metrics) {
+        super(creatorNodeId);
+
+        this.metrics = ClusterMetricsSnapshot.serialize(metrics);
+    }
+
+    /**
+     * Gets metrics map.
+     *
+     * @return Metrics map.
+     */
+    public ClusterMetrics metrics() {
+        return ClusterMetricsSnapshot.deserialize(metrics, 0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean highPriority() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryClientHeartbeatMessage.class, this, "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingRequest.java
new file mode 100644
index 0000000..f9f164d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingRequest.java
@@ -0,0 +1,56 @@
+/*
+ * 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.spi.discovery.tcp.messages;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Ping request.
+ */
+public class TcpDiscoveryClientPingRequest extends TcpDiscoveryAbstractMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Pinged client node ID. */
+    private final UUID nodeToPing;
+
+    /**
+     * @param creatorNodeId Creator node ID.
+     * @param nodeToPing Pinged client node ID.
+     */
+    public TcpDiscoveryClientPingRequest(UUID creatorNodeId, @Nullable UUID nodeToPing) {
+        super(creatorNodeId);
+
+        this.nodeToPing = nodeToPing;
+    }
+
+    /**
+     * @return Pinged client node ID.
+     */
+    @Nullable public UUID nodeToPing() {
+        return nodeToPing;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryClientPingRequest.class, this, "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingResponse.java
new file mode 100644
index 0000000..26a2b00
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientPingResponse.java
@@ -0,0 +1,67 @@
+/*
+ * 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.spi.discovery.tcp.messages;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Ping request.
+ */
+public class TcpDiscoveryClientPingResponse extends TcpDiscoveryAbstractMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Pinged client node ID. */
+    private final UUID nodeToPing;
+
+    /** */
+    private final boolean res;
+
+    /**
+     * @param creatorNodeId Creator node ID.
+     * @param nodeToPing Pinged client node ID.
+     */
+    public TcpDiscoveryClientPingResponse(UUID creatorNodeId, @Nullable UUID nodeToPing, boolean res) {
+        super(creatorNodeId);
+
+        this.nodeToPing = nodeToPing;
+        this.res = res;
+    }
+
+    /**
+     * @return Pinged client node ID.
+     */
+    @Nullable public UUID nodeToPing() {
+        return nodeToPing;
+    }
+
+    /**
+     * @return Result of ping.
+     */
+    public boolean result() {
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryClientPingResponse.class, this, "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
index 4e42f2d..0739c1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryCustomEventMessage.java
@@ -18,29 +18,34 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.spi.discovery.*;
+import org.jetbrains.annotations.*;
 
-import java.io.*;
 import java.util.*;
 
 /**
  * Wrapped for custom message.
  */
+@TcpDiscoveryRedirectToClient
 @TcpDiscoveryEnsureDelivery
 public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
     /** */
-    private transient Serializable msg;
+    private transient volatile DiscoverySpiCustomMessage msg;
 
     /** */
-    private final byte[] msgBytes;
+    private byte[] msgBytes;
 
     /**
      * @param creatorNodeId Creator node id.
+     * @param msg Message.
      * @param msgBytes Serialized message.
      */
-    public TcpDiscoveryCustomEventMessage(UUID creatorNodeId, Serializable msg, byte[] msgBytes) {
+    public TcpDiscoveryCustomEventMessage(UUID creatorNodeId, @Nullable DiscoverySpiCustomMessage msg,
+        @NotNull byte[] msgBytes) {
         super(creatorNodeId);
 
         this.msg = msg;
@@ -48,17 +53,33 @@ public class TcpDiscoveryCustomEventMessage extends TcpDiscoveryAbstractMessage
     }
 
     /**
-     * @return Message.
+     * @return Serialized message.
      */
-    public Serializable message() {
-        return msg;
+    public byte[] messageBytes() {
+        return msgBytes;
     }
 
     /**
-     * @return Serialized message.
+     * @param msg Message.
+     * @param msgBytes Serialized message.
      */
-    public byte[] messageBytes() {
-        return msgBytes;
+    public void message(@Nullable DiscoverySpiCustomMessage msg, @NotNull byte[] msgBytes) {
+        this.msg = msg;
+        this.msgBytes = msgBytes;
+    }
+
+    /**
+     * @return Deserialized message,
+     * @throws java.lang.Throwable if unmarshal failed.
+     */
+    @Nullable public DiscoverySpiCustomMessage message(@NotNull Marshaller marsh) throws Throwable {
+        if (msg == null) {
+            msg = marsh.unmarshal(msgBytes, U.gridClassLoader());
+
+            assert msg != null;
+        }
+
+        return msg;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java
index bafde9f..f721401 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHeartbeatMessage.java
@@ -58,13 +58,6 @@ public class TcpDiscoveryHeartbeatMessage extends TcpDiscoveryAbstractMessage {
     private final Map<UUID, Map<Integer, CacheMetrics>> cacheMetrics = new HashMap<>();
 
     /**
-     * Public default no-arg constructor for {@link Externalizable} interface.
-     */
-    public TcpDiscoveryHeartbeatMessage() {
-        // No-op.
-    }
-
-    /**
      * Constructor.
      *
      * @param creatorNodeId Creator node.
@@ -211,22 +204,13 @@ public class TcpDiscoveryHeartbeatMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public String toString() {
-        return S.toString(TcpDiscoveryHeartbeatMessage.class, this, "super", super.toString());
+    @Override public boolean highPriority() {
+        return true;
     }
 
-    /**
-     * @param metrics Metrics.
-     * @return Serialized metrics.
-     */
-    private static byte[] serializeMetrics(ClusterMetrics metrics) {
-        assert metrics != null;
-
-        byte[] buf = new byte[ClusterMetricsSnapshot.METRICS_SIZE];
-
-        ClusterMetricsSnapshot.serialize(buf, 0, metrics);
-
-        return buf;
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryHeartbeatMessage.class, this, "super", super.toString());
     }
 
     /**
@@ -273,7 +257,7 @@ public class TcpDiscoveryHeartbeatMessage extends TcpDiscoveryAbstractMessage {
         public MetricsSet(ClusterMetrics metrics) {
             assert metrics != null;
 
-            this.metrics = serializeMetrics(metrics);
+            this.metrics = ClusterMetricsSnapshot.serialize(metrics);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
index 5a71eb3..1d974e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddFinishedMessage.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
+import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -34,6 +36,17 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
     private final UUID nodeId;
 
     /**
+     * Client node can not get discovery data from TcpDiscoveryNodeAddedMessage, we have to pass discovery data in
+     * TcpDiscoveryNodeAddFinishedMessage
+     */
+    @GridToStringExclude
+    private Map<UUID, Map<Integer, byte[]>> clientDiscoData;
+
+    /** */
+    @GridToStringExclude
+    private Map<String, Object> clientNodeAttrs;
+
+    /**
      * Constructor.
      *
      * @param creatorNodeId ID of the creator node (coordinator).
@@ -54,6 +67,36 @@ public class TcpDiscoveryNodeAddFinishedMessage extends TcpDiscoveryAbstractMess
         return nodeId;
     }
 
+    /**
+     * @return Discovery data for joined client.
+     */
+    public Map<UUID, Map<Integer, byte[]>> clientDiscoData() {
+        return clientDiscoData;
+    }
+
+    /**
+     * @param clientDiscoData Discovery data for joined client.
+     */
+    public void clientDiscoData(@Nullable Map<UUID, Map<Integer, byte[]>> clientDiscoData) {
+        this.clientDiscoData = clientDiscoData;
+
+        assert clientDiscoData == null || !clientDiscoData.containsKey(nodeId);
+    }
+
+    /**
+     * @return Client node attributes.
+     */
+    public Map<String, Object> clientNodeAttributes() {
+        return clientNodeAttrs;
+    }
+
+    /**
+     * @param clientNodeAttrs New client node attributes.
+     */
+    public void clientNodeAttributes(Map<String, Object> clientNodeAttrs) {
+        this.clientNodeAttrs = clientNodeAttrs;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TcpDiscoveryNodeAddFinishedMessage.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
index a9303f3..2a14158 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeAddedMessage.java
@@ -148,7 +148,7 @@ public class TcpDiscoveryNodeAddedMessage extends TcpDiscoveryAbstractMessage {
      *
      * @return Map with topology snapshots history.
      */
-    @Nullable public Map<Long, Collection<ClusterNode>> topologyHistory() {
+    public Map<Long, Collection<ClusterNode>> topologyHistory() {
         return topHist;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingRequest.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingRequest.java
index de5b0a7..f17c91b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingRequest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
+import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -47,4 +48,9 @@ public class TcpDiscoveryPingRequest extends TcpDiscoveryAbstractMessage {
     @Nullable public UUID clientNodeId() {
         return clientNodeId;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryPingRequest.class, this, "super", super.toString());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingResponse.java
index 6396764..02b2d48 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryPingResponse.java
@@ -17,7 +17,8 @@
 
 package org.apache.ignite.spi.discovery.tcp.messages;
 
-import java.io.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
 import java.util.*;
 
 /**
@@ -31,13 +32,6 @@ public class TcpDiscoveryPingResponse extends TcpDiscoveryAbstractMessage {
     private boolean clientExists;
 
     /**
-     * For {@link Externalizable}.
-     */
-    public TcpDiscoveryPingResponse() {
-        // No-op.
-    }
-
-    /**
      * @param creatorNodeId Creator node ID.
      */
     public TcpDiscoveryPingResponse(UUID creatorNodeId) {
@@ -57,4 +51,9 @@ public class TcpDiscoveryPingResponse extends TcpDiscoveryAbstractMessage {
     public boolean clientExists() {
         return clientExists;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryPingResponse.class, this, "super", super.toString());
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
index e7db285..7a88426 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -387,15 +387,13 @@ public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi,
 
         Space space = space(spaceName, false);
 
-        if (space == null)
-            return;
-
-        byte[] val = space.remove(key, c != null);
+        byte[] val = space == null ? null : space.remove(key, c != null);
 
         if (c != null)
             c.apply(val);
 
-        notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
+        if (space != null)
+             notifyListener(EVT_SWAP_SPACE_DATA_REMOVED, spaceName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
new file mode 100644
index 0000000..467349f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/IgniteClientNodeAffinityTest.java
@@ -0,0 +1,182 @@
+/*
+ * 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.cache.affinity;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.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.junits.common.*;
+
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientNodeAffinityTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODE_CNT = 4;
+
+    /** */
+    private static final String CACHE1 = "cache1";
+
+    /** */
+    private static final String CACHE2 = "cache2";
+
+    /** */
+    private static final String CACHE3 = "cache3";
+
+    /** */
+    private static final String CACHE4 = "cache4";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (gridName.equals(getTestGridName(NODE_CNT - 1)))
+            cfg.setClientMode(true);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setBackups(1);
+        ccfg1.setName(CACHE1);
+        ccfg1.setAffinity(new RendezvousAffinityFunction());
+        ccfg1.setNodeFilter(new TestNodesFilter());
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setBackups(1);
+        ccfg2.setName(CACHE2);
+        ccfg2.setAffinity(new RendezvousAffinityFunction());
+
+        CacheConfiguration ccfg3 = new CacheConfiguration();
+
+        ccfg3.setBackups(1);
+        ccfg3.setName(CACHE3);
+        ccfg3.setAffinity(new FairAffinityFunction());
+        ccfg3.setNodeFilter(new TestNodesFilter());
+
+        CacheConfiguration ccfg4 = new CacheConfiguration();
+
+        ccfg4.setCacheMode(REPLICATED);
+        ccfg4.setName(CACHE4);
+        ccfg4.setNodeFilter(new TestNodesFilter());
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2, ccfg3, ccfg4);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODE_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeNotInAffinity() throws Exception {
+        checkCache(CACHE1, 2);
+
+        checkCache(CACHE2, 2);
+
+        checkCache(CACHE3, 2);
+
+        checkCache(CACHE4, 3);
+
+        Ignite client = ignite(NODE_CNT - 1);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setBackups(0);
+
+        ccfg.setNodeFilter(new TestNodesFilter());
+
+        try (IgniteCache<Integer, Integer> cache = client.createCache(ccfg)) {
+            checkCache(null, 1);
+        }
+
+        try (IgniteCache<Integer, Integer> cache = client.createCache(ccfg, new NearCacheConfiguration())) {
+            checkCache(null, 1);
+        }
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param expNodes Expected number of nodes per partition.
+     */
+    private void checkCache(String cacheName, int expNodes) {
+        log.info("Test cache: " + cacheName);
+
+        Ignite client = ignite(NODE_CNT - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        ClusterNode clientNode = client.cluster().localNode();
+
+        for (int i = 0; i < NODE_CNT; i++) {
+            Ignite ignite = ignite(i);
+
+            Affinity<Integer> aff = ignite.affinity(cacheName);
+
+            for (int part = 0; part < aff.partitions(); part++) {
+                Collection<ClusterNode> nodes = aff.mapPartitionToPrimaryAndBackups(part);
+
+                assertEquals(expNodes, nodes.size());
+
+                assertFalse(nodes.contains(clientNode));
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestNodesFilter implements IgnitePredicate<ClusterNode> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            Boolean attr = clusterNode.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
+
+            assertNotNull(attr);
+
+            assertFalse(attr);
+
+            return true;
+        }
+    }
+}


[25/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 560de97..74ba100 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -19,12 +19,15 @@ 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.internal.processors.cache.store.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.util.concurrent.atomic.*;
 
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.*;
+
 /**
  * Adapter for cache metrics.
  */
@@ -63,7 +66,7 @@ public class CacheMetricsImpl implements CacheMetrics {
     private AtomicLong getTimeNanos = new AtomicLong();
 
     /** Remove time taken nanos. */
-    private AtomicLong removeTimeNanos = new AtomicLong();
+    private AtomicLong rmvTimeNanos = new AtomicLong();
 
     /** Commit transaction time taken nanos. */
     private AtomicLong commitTimeNanos = new AtomicLong();
@@ -71,6 +74,39 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** Commit transaction time taken nanos. */
     private AtomicLong rollbackTimeNanos = new AtomicLong();
 
+    /** Number of reads from off-heap memory. */
+    private AtomicLong offHeapGets = new AtomicLong();
+
+    /** Number of writes to off-heap memory. */
+    private AtomicLong offHeapPuts = new AtomicLong();
+
+    /** Number of removed entries from off-heap memory. */
+    private AtomicLong offHeapRemoves = new AtomicLong();
+
+    /** Number of evictions from off-heap memory. */
+    private AtomicLong offHeapEvicts = new AtomicLong();
+
+    /** Number of off-heap hits. */
+    private AtomicLong offHeapHits = new AtomicLong();
+
+    /** Number of off-heap misses. */
+    private AtomicLong offHeapMisses = new AtomicLong();
+
+    /** Number of reads from swap. */
+    private AtomicLong swapGets = new AtomicLong();
+
+    /** Number of writes to swap. */
+    private AtomicLong swapPuts = new AtomicLong();
+
+    /** Number of removed entries from swap. */
+    private AtomicLong swapRemoves = new AtomicLong();
+
+    /** Number of swap hits. */
+    private AtomicLong swapHits = new AtomicLong();
+
+    /** Number of swap misses. */
+    private AtomicLong swapMisses = new AtomicLong();
+
     /** Cache metrics. */
     @GridToStringExclude
     private transient CacheMetricsImpl delegate;
@@ -118,7 +154,9 @@ public class CacheMetricsImpl implements CacheMetrics {
     /** {@inheritDoc} */
     @Override public long getOverflowSize() {
         try {
-            return cctx.cache().overflowSize();
+            GridCacheAdapter<?, ?> cache = cctx.cache();
+
+            return cache != null ? cache.overflowSize() : -1;
         }
         catch (IgniteCheckedException ignored) {
             return -1;
@@ -126,35 +164,192 @@ public class CacheMetricsImpl implements CacheMetrics {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        long hits0 = offHeapHits.get();
+        long gets0 = offHeapGets.get();
+
+        if (hits0 == 0)
+            return 0;
+
+        return (float) hits0 / gets0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        long misses0 = offHeapMisses.get();
+        long reads0 = offHeapGets.get();
+
+        if (misses0 == 0)
+            return 0;
+
+        return (float) misses0 / reads0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
-        return cctx.cache().offHeapEntriesCount();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.offHeapEntriesCount() : -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(true, false, cctx.affinity().affinityTopologyVersion());
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        try {
+            return cctx.swap().offheapEntriesCount(false, true, cctx.affinity().affinityTopologyVersion());
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
     }
 
     /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
-        return cctx.cache().offHeapAllocatedSize();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.offHeapAllocatedSize() : -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cctx.config().getOffHeapMaxMemory();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return swapGets.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return swapPuts.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return swapRemoves.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return swapHits.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return swapMisses.get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        try {
+            return cctx.cache().swapKeys();
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        try {
+            return cctx.cache().swapSize();
+        }
+        catch (IgniteCheckedException e) {
+            return 0;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        long hits0 = swapHits.get();
+        long gets0 = swapGets.get();
+
+        if (hits0 == 0)
+            return 0;
+
+        return (float) hits0 / gets0 * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        long misses0 = swapMisses.get();
+        long reads0 = swapGets.get();
+
+        if (misses0 == 0)
+            return 0;
+
+        return (float) misses0 / reads0 * 100.0f;
     }
 
     /** {@inheritDoc} */
     @Override public int getSize() {
-        return cctx.cache().size();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache != null ? cache.size() : 0;
     }
 
     /** {@inheritDoc} */
     @Override public int getKeySize() {
-        return cctx.cache().size();
+        return getSize();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isEmpty() {
-        return cctx.cache().isEmpty();
+        GridCacheAdapter<?, ?> cache = cctx.cache();
+
+        return cache == null || cache.isEmpty();
     }
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        return cctx.isNear() ?
-                dhtCtx != null ? dhtCtx.evicts().evictQueueSize() : -1
-                : cctx.evicts().evictQueueSize();
+        GridCacheContext<?, ?> ctx = cctx.isNear() ? dhtCtx : cctx;
+
+        if (ctx == null)
+            return -1;
+
+        GridCacheEvictionManager evictMgr = ctx.evicts();
+
+        return evictMgr != null ? evictMgr.evictQueueSize() : -1;
     }
 
     /** {@inheritDoc} */
@@ -317,11 +512,24 @@ public class CacheMetricsImpl implements CacheMetrics {
         txCommits.set(0);
         txRollbacks.set(0);
         putTimeNanos.set(0);
-        removeTimeNanos.set(0);
+        rmvTimeNanos.set(0);
         getTimeNanos.set(0);
         commitTimeNanos.set(0);
         rollbackTimeNanos.set(0);
 
+        offHeapGets.set(0);
+        offHeapPuts.set(0);
+        offHeapRemoves.set(0);
+        offHeapHits.set(0);
+        offHeapMisses.set(0);
+        offHeapEvicts.set(0);
+
+        swapGets.set(0);
+        swapPuts.set(0);
+        swapRemoves.set(0);
+        swapHits.set(0);
+        swapMisses.set(0);
+
         if (delegate != null)
             delegate.clear();
     }
@@ -402,7 +610,7 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        long timeNanos = removeTimeNanos.get();
+        long timeNanos = rmvTimeNanos.get();
         long removesCnt = rmCnt.get();
 
         if (timeNanos == 0 || removesCnt == 0)
@@ -483,7 +691,6 @@ public class CacheMetricsImpl implements CacheMetrics {
             delegate.onTxRollback(duration);
     }
 
-
     /**
      * Increments the get time accumulator.
      *
@@ -514,7 +721,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
 
         if (delegate != null)
             delegate.addRemoveTimeNanos(duration);
@@ -526,7 +733,7 @@ public class CacheMetricsImpl implements CacheMetrics {
      * @param duration the time taken in nanoseconds.
      */
     public void addRemoveAndGetTimeNanos(long duration) {
-        removeTimeNanos.addAndGet(duration);
+        rmvTimeNanos.addAndGet(duration);
         getTimeNanos.addAndGet(duration);
 
         if (delegate != null)
@@ -548,37 +755,153 @@ public class CacheMetricsImpl implements CacheMetrics {
 
     /** {@inheritDoc} */
     @Override public String getKeyType() {
-        return cctx.config().getKeyType().getName();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null ? ccfg.getKeyType().getName() : null;
     }
 
     /** {@inheritDoc} */
     @Override public String getValueType() {
-        return cctx.config().getValueType().getName();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null ? ccfg.getValueType().getName() : null;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isReadThrough() {
-        return cctx.config().isReadThrough();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isReadThrough();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isWriteThrough() {
-        return cctx.config().isWriteThrough();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isWriteThrough();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStoreByValue() {
-        return cctx.config().isStoreByValue();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isStoreByValue();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStatisticsEnabled() {
-        return cctx.config().isStatisticsEnabled();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isStatisticsEnabled();
     }
 
     /** {@inheritDoc} */
     @Override public boolean isManagementEnabled() {
-        return cctx.config().isManagementEnabled();
+        CacheConfiguration ccfg = cctx.config();
+
+        return ccfg != null && ccfg.isManagementEnabled();
+    }
+
+    /**
+     * Off-heap read callback.
+     *
+     * @param hit Hit or miss flag.
+     */
+    public void onOffHeapRead(boolean hit) {
+        offHeapGets.incrementAndGet();
+
+        if (hit)
+            offHeapHits.incrementAndGet();
+        else
+            offHeapMisses.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRead(hit);
+    }
+
+    /**
+     * Off-heap write callback.
+     */
+    public void onOffHeapWrite() {
+        offHeapPuts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapWrite();
+    }
+
+    /**
+     * Off-heap remove callback.
+     */
+    public void onOffHeapRemove() {
+        offHeapRemoves.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
+    /**
+     * Off-heap evict callback.
+     */
+    public void onOffHeapEvict() {
+        offHeapEvicts.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onOffHeapRemove();
+    }
+
+    /**
+     * Swap read callback.
+     *
+     * @param hit Hit or miss flag.
+     */
+    public void onSwapRead(boolean hit) {
+        swapGets.incrementAndGet();
+
+        if (hit)
+            swapHits.incrementAndGet();
+        else
+            swapMisses.incrementAndGet();
+
+        if (delegate != null)
+            delegate.onSwapRead(hit);
+    }
+
+    /**
+     * Swap write callback.
+     */
+    public void onSwapWrite() {
+        onSwapWrite(1);
+    }
+
+    /**
+     * Swap write callback.
+     *
+     * @param cnt Amount of entries.
+     */
+    public void onSwapWrite(int cnt) {
+        swapPuts.addAndGet(cnt);
+
+        if (delegate != null)
+            delegate.onSwapWrite(cnt);
+    }
+
+    /**
+     * Swap remove callback.
+     */
+    public void onSwapRemove() {
+        onSwapRemove(1);
+    }
+
+    /**
+     * Swap remove callback.
+     *
+     * @param cnt Amount of entries.
+     */
+    public void onSwapRemove(int cnt) {
+        swapRemoves.addAndGet(cnt);
+
+        if (delegate != null)
+            delegate.onSwapRemove(cnt);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
index e9d547c..966027a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsMXBeanImpl.java
@@ -49,16 +49,116 @@ class CacheMetricsMXBeanImpl implements CacheMetricsMXBean {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return cache.metrics0().getOffHeapGets();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return cache.metrics0().getOffHeapPuts();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return cache.metrics0().getOffHeapRemovals();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return cache.metrics0().getOffHeapEvictions();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return cache.metrics0().getOffHeapHits();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        return cache.metrics0().getOffHeapHitPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return cache.metrics0().getOffHeapMisses();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        return cache.metrics0().getOffHeapMissPercentage();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
         return cache.metrics0().getOffHeapEntriesCount();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return cache.metrics0().getOffHeapPrimaryEntriesCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return cache.metrics0().getOffHeapBackupEntriesCount();
+    }
+
+    /** {@inheritDoc} */
     @Override public long getOffHeapAllocatedSize() {
         return cache.metrics0().getOffHeapAllocatedSize();
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return cache.metrics0().getOffHeapMaxSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return cache.metrics0().getSwapGets();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return cache.metrics0().getSwapPuts();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return cache.metrics0().getSwapRemovals();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return cache.metrics0().getSwapHits();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return cache.metrics0().getSwapMisses();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        return cache.metrics0().getSwapHitPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        return cache.metrics0().getSwapMissPercentage();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        return cache.metrics0().getSwapEntriesCount();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        return cache.metrics0().getSwapSize();
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return cache.metrics0().getSize();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
index 4fe152a..cf16d9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsSnapshot.java
@@ -61,7 +61,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private float getAvgTimeNanos = 0;
 
     /** Remove time taken nanos. */
-    private float removeAvgTimeNanos = 0;
+    private float rmvAvgTimeNanos = 0;
 
     /** Commit transaction time taken nanos. */
     private float commitAvgTimeNanos = 0;
@@ -75,12 +75,60 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     /** Number of entries that was swapped to disk. */
     private long overflowSize;
 
+    /** Number of reads from off-heap. */
+    private long offHeapGets;
+
+    /** Number of writes to off-heap. */
+    private long offHeapPuts;
+
+    /** Number of removed entries from off-heap. */
+    private long offHeapRemoves;
+
+    /** Number of evictions from off-heap. */
+    private long offHeapEvicts;
+
+    /** Off-heap hits number. */
+    private long offHeapHits;
+
+    /** Off-heap misses number. */
+    private long offHeapMisses;
+
     /** Number of entries stored in off-heap memory. */
-    private long offHeapEntriesCount;
+    private long offHeapEntriesCnt;
+
+    /** Number of primary entries stored in off-heap memory. */
+    private long offHeapPrimaryEntriesCnt;
+
+    /** Number of backup entries stored in off-heap memory. */
+    private long offHeapBackupEntriesCnt;
 
     /** Memory size allocated in off-heap. */
     private long offHeapAllocatedSize;
 
+    /** Off-heap memory maximum size*/
+    private long offHeapMaxSize;
+
+    /** Number of reads from swap. */
+    private long swapGets;
+
+    /** Number of writes to swap. */
+    private long swapPuts;
+
+    /** Number of removed entries from swap. */
+    private long swapRemoves;
+
+    /** Number of entries stored in swap. */
+    private long swapEntriesCnt;
+
+    /** Swap hits number. */
+    private long swapHits;
+
+    /** Swap misses number. */
+    private long swapMisses;
+
+    /** Swap size. */
+    private long swapSize;
+
     /** Number of non-{@code null} values in the cache. */
     private int size;
 
@@ -91,7 +139,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private boolean isEmpty;
 
     /** Gets current size of evict queue used to batch up evictions. */
-    private int dhtEvictQueueCurrentSize;
+    private int dhtEvictQueueCurrSize;
 
     /** Transaction per-thread map size. */
     private int txThreadMapSize;
@@ -106,7 +154,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txPrepareQueueSize;
 
     /** Start version counts map size. */
-    private int txStartVersionCountsSize;
+    private int txStartVerCountsSize;
 
     /** Number of cached committed transaction IDs. */
     private int txCommittedVersionsSize;
@@ -127,7 +175,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int txDhtPrepareQueueSize;
 
     /** DHT start version counts map size. */
-    private int txDhtStartVersionCountsSize;
+    private int txDhtStartVerCountsSize;
 
     /** Number of cached committed DHT transaction IDs. */
     private int txDhtCommittedVersionsSize;
@@ -142,34 +190,34 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     private int writeBehindFlushSize;
 
     /** Count of worker threads. */
-    private int writeBehindFlushThreadCount;
+    private int writeBehindFlushThreadCnt;
 
     /** Flush frequency in milliseconds. */
-    private long writeBehindFlushFrequency;
+    private long writeBehindFlushFreq;
 
     /** Maximum size of batch. */
     private int writeBehindStoreBatchSize;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindTotalCriticalOverflowCount;
+    private int writeBehindTotalCriticalOverflowCnt;
 
     /** Count of cache overflow events since start. */
-    private int writeBehindCriticalOverflowCount;
+    private int writeBehindCriticalOverflowCnt;
 
     /** Count of entries in store-retry state. */
-    private int writeBehindErrorRetryCount;
+    private int writeBehindErrorRetryCnt;
 
     /** Total count of entries in cache store internal buffer. */
-    private int writeBehindBufferSize;
+    private int writeBehindBufSize;
 
     /** */
     private String keyType;
 
     /** */
-    private String valueType;
+    private String valType;
 
     /** */
-    private boolean isStoreByValue;
+    private boolean isStoreByVal;
 
     /** */
     private boolean isStatisticsEnabled;
@@ -207,45 +255,64 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = m.getAveragePutTime();
         getAvgTimeNanos = m.getAverageGetTime();
-        removeAvgTimeNanos = m.getAverageRemoveTime();
+        rmvAvgTimeNanos = m.getAverageRemoveTime();
         commitAvgTimeNanos = m.getAverageTxCommitTime();
         rollbackAvgTimeNanos = m.getAverageTxRollbackTime();
 
         cacheName = m.name();
         overflowSize = m.getOverflowSize();
-        offHeapEntriesCount = m.getOffHeapEntriesCount();
+
+        offHeapGets = m.getOffHeapGets();
+        offHeapPuts = m.getOffHeapPuts();
+        offHeapRemoves = m.getOffHeapRemovals();
+        offHeapEvicts = m.getOffHeapEvictions();
+        offHeapHits = m.getOffHeapHits();
+        offHeapMisses = m.getOffHeapMisses();
+        offHeapEntriesCnt = m.getOffHeapEntriesCount();
+        offHeapPrimaryEntriesCnt = m.getOffHeapPrimaryEntriesCount();
+        offHeapBackupEntriesCnt = m.getOffHeapBackupEntriesCount();
         offHeapAllocatedSize = m.getOffHeapAllocatedSize();
+        offHeapMaxSize = m.getOffHeapMaxSize();
+
+        swapGets = m.getSwapGets();
+        swapPuts = m.getSwapPuts();
+        swapRemoves = m.getSwapRemovals();
+        swapHits = m.getSwapHits();
+        swapMisses = m.getSwapMisses();
+        swapEntriesCnt = m.getSwapEntriesCount();
+        swapSize = m.getSwapSize();
+
         size = m.getSize();
         keySize = m.getKeySize();
         isEmpty = m.isEmpty();
-        dhtEvictQueueCurrentSize = m.getDhtEvictQueueCurrentSize();
+        dhtEvictQueueCurrSize = m.getDhtEvictQueueCurrentSize();
         txThreadMapSize = m.getTxThreadMapSize();
         txXidMapSize = m.getTxXidMapSize();
         txCommitQueueSize = m.getTxCommitQueueSize();
         txPrepareQueueSize = m.getTxPrepareQueueSize();
-        txStartVersionCountsSize = m.getTxStartVersionCountsSize();
+        txStartVerCountsSize = m.getTxStartVersionCountsSize();
         txCommittedVersionsSize = m.getTxCommittedVersionsSize();
         txRolledbackVersionsSize = m.getTxRolledbackVersionsSize();
         txDhtThreadMapSize = m.getTxDhtThreadMapSize();
         txDhtXidMapSize = m.getTxDhtXidMapSize();
         txDhtCommitQueueSize = m.getTxDhtCommitQueueSize();
         txDhtPrepareQueueSize = m.getTxDhtPrepareQueueSize();
-        txDhtStartVersionCountsSize = m.getTxDhtStartVersionCountsSize();
+        txDhtStartVerCountsSize = m.getTxDhtStartVersionCountsSize();
         txDhtCommittedVersionsSize = m.getTxDhtCommittedVersionsSize();
         txDhtRolledbackVersionsSize = m.getTxDhtRolledbackVersionsSize();
         isWriteBehindEnabled = m.isWriteBehindEnabled();
         writeBehindFlushSize = m.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = m.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = m.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = m.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = m.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = m.getWriteBehindStoreBatchSize();
-        writeBehindTotalCriticalOverflowCount = m.getWriteBehindTotalCriticalOverflowCount();
-        writeBehindCriticalOverflowCount = m.getWriteBehindCriticalOverflowCount();
-        writeBehindErrorRetryCount = m.getWriteBehindErrorRetryCount();
-        writeBehindBufferSize = m.getWriteBehindBufferSize();
+        writeBehindTotalCriticalOverflowCnt = m.getWriteBehindTotalCriticalOverflowCount();
+        writeBehindCriticalOverflowCnt = m.getWriteBehindCriticalOverflowCount();
+        writeBehindErrorRetryCnt = m.getWriteBehindErrorRetryCount();
+        writeBehindBufSize = m.getWriteBehindBufferSize();
 
         keyType = m.getKeyType();
-        valueType = m.getValueType();
-        isStoreByValue = m.isStoreByValue();
+        valType = m.getValueType();
+        isStoreByVal = m.isStoreByValue();
         isStatisticsEnabled = m.isStatisticsEnabled();
         isManagementEnabled = m.isManagementEnabled();
         isReadThrough = m.isReadThrough();
@@ -263,21 +330,23 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         isEmpty = loc.isEmpty();
         isWriteBehindEnabled = loc.isWriteBehindEnabled();
         writeBehindFlushSize = loc.getWriteBehindFlushSize();
-        writeBehindFlushThreadCount = loc.getWriteBehindFlushThreadCount();
-        writeBehindFlushFrequency = loc.getWriteBehindFlushFrequency();
+        writeBehindFlushThreadCnt = loc.getWriteBehindFlushThreadCount();
+        writeBehindFlushFreq = loc.getWriteBehindFlushFrequency();
         writeBehindStoreBatchSize = loc.getWriteBehindStoreBatchSize();
-        writeBehindBufferSize = loc.getWriteBehindBufferSize();
+        writeBehindBufSize = loc.getWriteBehindBufferSize();
         size = loc.getSize();
         keySize = loc.getKeySize();
 
         keyType = loc.getKeyType();
-        valueType = loc.getValueType();
-        isStoreByValue = loc.isStoreByValue();
+        valType = loc.getValueType();
+        isStoreByVal = loc.isStoreByValue();
         isStatisticsEnabled = loc.isStatisticsEnabled();
         isManagementEnabled = loc.isManagementEnabled();
         isReadThrough = loc.isReadThrough();
         isWriteThrough = loc.isWriteThrough();
 
+        offHeapMaxSize = loc.getOffHeapMaxSize();
+
         for (CacheMetrics e : metrics) {
             reads += e.getCacheGets();
             puts += e.getCachePuts();
@@ -290,7 +359,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
             putAvgTimeNanos += e.getAveragePutTime();
             getAvgTimeNanos += e.getAverageGetTime();
-            removeAvgTimeNanos += e.getAverageRemoveTime();
+            rmvAvgTimeNanos += e.getAverageRemoveTime();
             commitAvgTimeNanos += e.getAverageTxCommitTime();
             rollbackAvgTimeNanos += e.getAverageTxRollbackTime();
 
@@ -299,19 +368,35 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
             else
                 overflowSize = -1;
 
-            offHeapEntriesCount += e.getOffHeapEntriesCount();
+            offHeapGets += e.getOffHeapGets();
+            offHeapPuts += e.getOffHeapPuts();
+            offHeapRemoves += e.getOffHeapRemovals();
+            offHeapEvicts += e.getOffHeapEvictions();
+            offHeapHits += e.getOffHeapHits();
+            offHeapMisses += e.getOffHeapMisses();
+            offHeapEntriesCnt += e.getOffHeapEntriesCount();
+            offHeapPrimaryEntriesCnt += e.getOffHeapPrimaryEntriesCount();
+            offHeapBackupEntriesCnt += e.getOffHeapBackupEntriesCount();
             offHeapAllocatedSize += e.getOffHeapAllocatedSize();
 
+            swapGets += e.getSwapGets();
+            swapPuts += e.getSwapPuts();
+            swapRemoves += e.getSwapRemovals();
+            swapHits += e.getSwapHits();
+            swapMisses += e.getSwapMisses();
+            swapEntriesCnt += e.getSwapEntriesCount();
+            swapSize += e.getSwapSize();
+
             if (e.getDhtEvictQueueCurrentSize() > -1)
-                dhtEvictQueueCurrentSize += e.getDhtEvictQueueCurrentSize();
+                dhtEvictQueueCurrSize += e.getDhtEvictQueueCurrentSize();
             else
-                dhtEvictQueueCurrentSize = -1;
+                dhtEvictQueueCurrSize = -1;
 
             txThreadMapSize += e.getTxThreadMapSize();
             txXidMapSize += e.getTxXidMapSize();
             txCommitQueueSize += e.getTxCommitQueueSize();
             txPrepareQueueSize += e.getTxPrepareQueueSize();
-            txStartVersionCountsSize += e.getTxStartVersionCountsSize();
+            txStartVerCountsSize += e.getTxStartVersionCountsSize();
             txCommittedVersionsSize += e.getTxCommittedVersionsSize();
             txRolledbackVersionsSize += e.getTxRolledbackVersionsSize();
 
@@ -336,9 +421,9 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtPrepareQueueSize = -1;
 
             if (e.getTxDhtStartVersionCountsSize() > -1)
-                txDhtStartVersionCountsSize += e.getTxDhtStartVersionCountsSize();
+                txDhtStartVerCountsSize += e.getTxDhtStartVersionCountsSize();
             else
-                txDhtStartVersionCountsSize = -1;
+                txDhtStartVerCountsSize = -1;
 
             if (e.getTxDhtCommittedVersionsSize() > -1)
                 txDhtCommittedVersionsSize += e.getTxDhtCommittedVersionsSize();
@@ -351,19 +436,19 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
                 txDhtRolledbackVersionsSize = -1;
 
             if (e.getWriteBehindTotalCriticalOverflowCount() > -1)
-                writeBehindTotalCriticalOverflowCount += e.getWriteBehindTotalCriticalOverflowCount();
+                writeBehindTotalCriticalOverflowCnt += e.getWriteBehindTotalCriticalOverflowCount();
             else
-                writeBehindTotalCriticalOverflowCount = -1;
+                writeBehindTotalCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindCriticalOverflowCount() > -1)
-                writeBehindCriticalOverflowCount += e.getWriteBehindCriticalOverflowCount();
+                writeBehindCriticalOverflowCnt += e.getWriteBehindCriticalOverflowCount();
             else
-                writeBehindCriticalOverflowCount = -1;
+                writeBehindCriticalOverflowCnt = -1;
 
             if (e.getWriteBehindErrorRetryCount() > -1)
-                writeBehindErrorRetryCount += e.getWriteBehindErrorRetryCount();
+                writeBehindErrorRetryCnt += e.getWriteBehindErrorRetryCount();
             else
-                writeBehindErrorRetryCount = -1;
+                writeBehindErrorRetryCnt = -1;
         }
 
         int size = metrics.size();
@@ -371,7 +456,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
         if (size > 1) {
             putAvgTimeNanos /= size;
             getAvgTimeNanos /= size;
-            removeAvgTimeNanos /= size;
+            rmvAvgTimeNanos /= size;
             commitAvgTimeNanos /= size;
             rollbackAvgTimeNanos /= size;
         }
@@ -435,7 +520,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public float getAverageRemoveTime() {
-        return removeAvgTimeNanos;
+        return rmvAvgTimeNanos;
     }
 
     /** {@inheritDoc} */
@@ -469,8 +554,63 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapGets() {
+        return offHeapGets;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPuts() {
+        return offHeapPuts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapRemovals() {
+        return offHeapRemoves;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapEvictions() {
+        return offHeapEvicts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapHits() {
+        return offHeapHits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapHitPercentage() {
+        if (offHeapHits == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapHits / offHeapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapMisses() {
+        return offHeapMisses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getOffHeapMissPercentage() {
+        if (offHeapMisses == 0 || offHeapGets == 0)
+            return 0;
+
+        return (float) offHeapMisses / offHeapGets * 100.0f;
+    }
+    /** {@inheritDoc} */
     @Override public long getOffHeapEntriesCount() {
-        return offHeapEntriesCount;
+        return offHeapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapPrimaryEntriesCount() {
+        return offHeapPrimaryEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getOffHeapBackupEntriesCount() {
+        return offHeapBackupEntriesCnt;
     }
 
     /** {@inheritDoc} */
@@ -479,6 +619,62 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public long getOffHeapMaxSize() {
+        return offHeapMaxSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapGets() {
+        return swapGets;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapPuts() {
+        return swapPuts;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapRemovals() {
+        return swapRemoves;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapHits() {
+        return swapHits;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapMisses() {
+        return swapMisses;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapHitPercentage() {
+        if (swapHits == 0 || swapGets == 0)
+            return 0;
+
+        return (float) swapHits / swapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getSwapMissPercentage() {
+        if (swapMisses == 0 || swapGets == 0)
+            return 0;
+
+        return (float) swapMisses / swapGets * 100.0f;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapEntriesCount() {
+        return swapEntriesCnt;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getSwapSize() {
+        return swapSize;
+    }
+
+    /** {@inheritDoc} */
     @Override public int getSize() {
         return size;
     }
@@ -495,7 +691,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getDhtEvictQueueCurrentSize() {
-        return dhtEvictQueueCurrentSize;
+        return dhtEvictQueueCurrSize;
     }
 
     /** {@inheritDoc} */
@@ -520,7 +716,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxStartVersionCountsSize() {
-        return txStartVersionCountsSize;
+        return txStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -555,7 +751,7 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getTxDhtStartVersionCountsSize() {
-        return txDhtStartVersionCountsSize;
+        return txDhtStartVerCountsSize;
     }
 
     /** {@inheritDoc} */
@@ -580,12 +776,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindFlushThreadCount() {
-        return writeBehindFlushThreadCount;
+        return writeBehindFlushThreadCnt;
     }
 
     /** {@inheritDoc} */
     @Override public long getWriteBehindFlushFrequency() {
-        return writeBehindFlushFrequency;
+        return writeBehindFlushFreq;
     }
 
     /** {@inheritDoc} */
@@ -595,22 +791,22 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindTotalCriticalOverflowCount() {
-        return writeBehindTotalCriticalOverflowCount;
+        return writeBehindTotalCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindCriticalOverflowCount() {
-        return writeBehindCriticalOverflowCount;
+        return writeBehindCriticalOverflowCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindErrorRetryCount() {
-        return writeBehindErrorRetryCount;
+        return writeBehindErrorRetryCnt;
     }
 
     /** {@inheritDoc} */
     @Override public int getWriteBehindBufferSize() {
-        return writeBehindBufferSize;
+        return writeBehindBufSize;
     }
 
     /** {@inheritDoc} */
@@ -620,12 +816,12 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
     /** {@inheritDoc} */
     @Override public String getValueType() {
-        return valueType;
+        return valType;
     }
 
     /** {@inheritDoc} */
     @Override public boolean isStoreByValue() {
-        return isStoreByValue;
+        return isStoreByVal;
     }
 
     /** {@inheritDoc} */
@@ -666,31 +862,49 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         out.writeFloat(putAvgTimeNanos);
         out.writeFloat(getAvgTimeNanos);
-        out.writeFloat(removeAvgTimeNanos);
+        out.writeFloat(rmvAvgTimeNanos);
         out.writeFloat(commitAvgTimeNanos);
         out.writeFloat(rollbackAvgTimeNanos);
 
         out.writeLong(overflowSize);
-        out.writeLong(offHeapEntriesCount);
+        out.writeLong(offHeapGets);
+        out.writeLong(offHeapPuts);
+        out.writeLong(offHeapRemoves);
+        out.writeLong(offHeapEvicts);
+        out.writeLong(offHeapHits);
+        out.writeLong(offHeapMisses);
+        out.writeLong(offHeapEntriesCnt);
+        out.writeLong(offHeapPrimaryEntriesCnt);
+        out.writeLong(offHeapBackupEntriesCnt);
         out.writeLong(offHeapAllocatedSize);
-        out.writeInt(dhtEvictQueueCurrentSize);
+        out.writeLong(offHeapMaxSize);
+
+        out.writeLong(swapGets);
+        out.writeLong(swapPuts);
+        out.writeLong(swapRemoves);
+        out.writeLong(swapHits);
+        out.writeLong(swapMisses);
+        out.writeLong(swapEntriesCnt);
+        out.writeLong(swapSize);
+
+        out.writeInt(dhtEvictQueueCurrSize);
         out.writeInt(txThreadMapSize);
         out.writeInt(txXidMapSize);
         out.writeInt(txCommitQueueSize);
         out.writeInt(txPrepareQueueSize);
-        out.writeInt(txStartVersionCountsSize);
+        out.writeInt(txStartVerCountsSize);
         out.writeInt(txCommittedVersionsSize);
         out.writeInt(txRolledbackVersionsSize);
         out.writeInt(txDhtThreadMapSize);
         out.writeInt(txDhtXidMapSize);
         out.writeInt(txDhtCommitQueueSize);
         out.writeInt(txDhtPrepareQueueSize);
-        out.writeInt(txDhtStartVersionCountsSize);
+        out.writeInt(txDhtStartVerCountsSize);
         out.writeInt(txDhtCommittedVersionsSize);
         out.writeInt(txDhtRolledbackVersionsSize);
-        out.writeInt(writeBehindTotalCriticalOverflowCount);
-        out.writeInt(writeBehindCriticalOverflowCount);
-        out.writeInt(writeBehindErrorRetryCount);
+        out.writeInt(writeBehindTotalCriticalOverflowCnt);
+        out.writeInt(writeBehindCriticalOverflowCnt);
+        out.writeInt(writeBehindErrorRetryCnt);
     }
 
     /** {@inheritDoc} */
@@ -706,30 +920,48 @@ public class CacheMetricsSnapshot implements CacheMetrics, Externalizable {
 
         putAvgTimeNanos = in.readFloat();
         getAvgTimeNanos = in.readFloat();
-        removeAvgTimeNanos = in.readFloat();
+        rmvAvgTimeNanos = in.readFloat();
         commitAvgTimeNanos = in.readFloat();
         rollbackAvgTimeNanos = in.readFloat();
 
         overflowSize = in.readLong();
-        offHeapEntriesCount = in.readLong();
+        offHeapGets = in.readLong();
+        offHeapPuts = in.readLong();
+        offHeapRemoves = in.readLong();
+        offHeapEvicts = in.readLong();
+        offHeapHits = in.readLong();
+        offHeapMisses = in.readLong();
+        offHeapEntriesCnt = in.readLong();
+        offHeapPrimaryEntriesCnt = in.readLong();
+        offHeapBackupEntriesCnt = in.readLong();
         offHeapAllocatedSize = in.readLong();
-        dhtEvictQueueCurrentSize = in.readInt();
+        offHeapMaxSize = in.readLong();
+
+        swapGets = in.readLong();
+        swapPuts = in.readLong();
+        swapRemoves = in.readLong();
+        swapHits = in.readLong();
+        swapMisses = in.readLong();
+        swapEntriesCnt = in.readLong();
+        swapSize = in.readLong();
+
+        dhtEvictQueueCurrSize = in.readInt();
         txThreadMapSize = in.readInt();
         txXidMapSize = in.readInt();
         txCommitQueueSize = in.readInt();
         txPrepareQueueSize = in.readInt();
-        txStartVersionCountsSize = in.readInt();
+        txStartVerCountsSize = in.readInt();
         txCommittedVersionsSize = in.readInt();
         txRolledbackVersionsSize = in.readInt();
         txDhtThreadMapSize = in.readInt();
         txDhtXidMapSize = in.readInt();
         txDhtCommitQueueSize = in.readInt();
         txDhtPrepareQueueSize = in.readInt();
-        txDhtStartVersionCountsSize = in.readInt();
+        txDhtStartVerCountsSize = in.readInt();
         txDhtCommittedVersionsSize = in.readInt();
         txDhtRolledbackVersionsSize = in.readInt();
-        writeBehindTotalCriticalOverflowCount = in.readInt();
-        writeBehindCriticalOverflowCount = in.readInt();
-        writeBehindErrorRetryCount = in.readInt();
+        writeBehindTotalCriticalOverflowCnt = in.readInt();
+        writeBehindCriticalOverflowCnt = in.readInt();
+        writeBehindErrorRetryCnt = in.readInt();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index ac3660e..dfc39c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -17,16 +17,18 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
 
-import java.io.*;
 import java.util.*;
 
 /**
  * Cache change batch.
  */
-public class DynamicCacheChangeBatch implements Serializable {
+public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -38,6 +40,9 @@ public class DynamicCacheChangeBatch implements Serializable {
     @GridToStringInclude
     private Map<String, Map<UUID, Boolean>> clientNodes;
 
+    /** Custom message ID. */
+    private IgniteUuid id = IgniteUuid.randomUuid();
+
     /**
      * @param reqs Requests.
      */
@@ -47,6 +52,11 @@ public class DynamicCacheChangeBatch implements Serializable {
         this.reqs = reqs;
     }
 
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
     /**
      * @return Collection of change requests.
      */
@@ -69,6 +79,21 @@ public class DynamicCacheChangeBatch implements Serializable {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean incrementMinorTopologyVersion() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheChangeBatch.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
index 6f6f422..9c6cc43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheDescriptor.java
@@ -61,7 +61,11 @@ public class DynamicCacheDescriptor {
     /** Cache plugin manager. */
     private final CachePluginManager pluginMgr;
 
+    /** */
+    private boolean updatesAllowed = true;
+
     /**
+     * @param ctx Context.
      * @param cacheCfg Cache configuration.
      * @param cacheType Cache type.
      * @param template {@code True} if this is template configuration.
@@ -76,6 +80,7 @@ public class DynamicCacheDescriptor {
         this.cacheType = cacheType;
         this.template = template;
         this.deploymentId = deploymentId;
+
         pluginMgr = new CachePluginManager(ctx, cacheCfg);
     }
 
@@ -206,6 +211,20 @@ public class DynamicCacheDescriptor {
         rmtCfgs = null;
     }
 
+    /**
+     * @return Updates allowed flag.
+     */
+    public boolean updatesAllowed() {
+        return updatesAllowed;
+    }
+
+    /**
+     * @param updatesAllowed Updates allowed flag.
+     */
+    public void updatesAllowed(boolean updatesAllowed) {
+        this.updatesAllowed = updatesAllowed;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheDescriptor.class, this, "cacheName", U.maskName(cacheCfg.getName()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index d390037..d8d029e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -348,7 +348,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @return Preloader.
      */
-    public abstract GridCachePreloader<K, V> preloader();
+    public abstract GridCachePreloader preloader();
 
     /** {@inheritDoc} */
     @Override public Affinity<K> affinity() {
@@ -395,6 +395,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
+        assert !CU.isUtilityCache(ctx.name());
+        assert !CU.isAtomicsCache(ctx.name());
+        assert !CU.isMarshallerCache(ctx.name());
+
         CacheOperationContext opCtx = new CacheOperationContext(false, null, false, plc);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
@@ -902,7 +906,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[]) null);
+        return keySet((CacheEntryPredicate[])null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<K> keySetx() {
+        return keySetx((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
@@ -1215,11 +1224,22 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
+        return getAllOutTxAsync(keys).get();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return getAllAsync(keys, !ctx.config().isReadFromBackup(), /*skip tx*/true, null, null, taskName, true, false)
-            .get();
+        return getAllAsync(keys,
+            !ctx.config().isReadFromBackup(),
+            /*skip tx*/true,
+            null,
+            null,
+            taskName,
+            !ctx.keepPortable(),
+            false);
     }
 
     /**
@@ -3249,7 +3269,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public long overflowSize() throws IgniteCheckedException {
-        return ctx.swap().swapSize();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.swapSize() : -1;
     }
 
     /**
@@ -3802,12 +3824,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public long offHeapEntriesCount() {
-        return ctx.swap().offHeapEntriesCount();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.offHeapEntriesCount() : -1;
     }
 
     /** {@inheritDoc} */
     @Override public long offHeapAllocatedSize() {
-        return ctx.swap().offHeapAllocatedSize();
+        GridCacheSwapManager swapMgr = ctx.swap();
+
+        return swapMgr != null ? swapMgr.offHeapAllocatedSize() : -1;
     }
 
     /** {@inheritDoc} */
@@ -4299,6 +4325,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * @param filter Filters to evaluate.
+     * @return Key set including internal keys.
+     */
+    public Set<K> keySetx(@Nullable CacheEntryPredicate... filter) {
+        return map.keySetx(filter);
+    }
+
+    /**
      * @param filter Primary key set.
      * @return Primary key set.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index fe7efd5..ea17df1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -140,6 +140,7 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
      *
      * @param topVer Topology version to calculate affinity for.
      * @param discoEvt Discovery event that causes this topology change.
+     * @return Affinity assignments.
      */
     public List<List<ClusterNode>> calculateAffinity(AffinityTopologyVersion topVer, DiscoveryEvent discoEvt) {
         assert !cctx.isLocal();
@@ -148,6 +149,19 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * Copies previous affinity assignment when discovery event does not cause affinity assignment changes
+     * (e.g. client node joins on leaves).
+     *
+     * @param evt Event.
+     * @param topVer Topology version.
+     */
+    public void clientEventTopologyChange(DiscoveryEvent evt, AffinityTopologyVersion topVer) {
+        assert !cctx.isLocal();
+
+        aff.clientEventTopologyChange(evt, topVer);
+    }
+
+    /**
      * @return Partition count.
      */
     public int partitions() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index bd3e0f2..db5eed1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -626,7 +626,19 @@ public class GridCacheConcurrentMap {
     public <K, V> Set<K> keySet(CacheEntryPredicate... filter) {
         checkWeakQueue();
 
-        return new KeySet<>(this, filter);
+        return new KeySet<>(this, filter, false);
+    }
+
+    /**
+     * Key set including internal keys.
+     *
+     * @param filter Filter.
+     * @return Set of the keys contained in this map.
+     */
+    public <K, V> Set<K> keySetx(CacheEntryPredicate... filter) {
+        checkWeakQueue();
+
+        return new KeySet<>(this, filter, true);
     }
 
     /**
@@ -1921,7 +1933,7 @@ public class GridCacheConcurrentMap {
 
         /** {@inheritDoc} */
         @Override public void clear() {
-            ctx.cache().clearLocally0(new KeySet<K, V>(map, filter));
+            ctx.cache().clearLocally0(new KeySet<K, V>(map, filter, false));
         }
 
         /** {@inheritDoc} */
@@ -2171,11 +2183,12 @@ public class GridCacheConcurrentMap {
         /**
          * @param map Base map.
          * @param filter Key filter.
+         * @param internal Whether to allow internal keys.
          */
-        private KeySet(GridCacheConcurrentMap map, CacheEntryPredicate[] filter) {
+        private KeySet(GridCacheConcurrentMap map, CacheEntryPredicate[] filter, boolean internal) {
             assert map != null;
 
-            set = new Set0<>(map, nonInternal(filter));
+            set = new Set0<>(map, internal ? filter : nonInternal(filter));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 2eeaed6..8a4e3b9 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
@@ -196,6 +196,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Dynamic cache deployment ID. */
     private IgniteUuid dynamicDeploymentId;
 
+    /** Updates allowed flag. */
+    private boolean updatesAllowed;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -209,6 +212,7 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param cacheCfg Cache configuration.
      * @param cacheType Cache type.
      * @param affNode {@code True} if local node is affinity node.
+     * @param updatesAllowed Updates allowed flag.
      * @param evtMgr Cache event manager.
      * @param swapMgr Cache swap manager.
      * @param storeMgr Store manager.
@@ -230,6 +234,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         CacheConfiguration cacheCfg,
         CacheType cacheType,
         boolean affNode,
+        boolean updatesAllowed,
 
         /*
          * Managers in starting order!
@@ -271,6 +276,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         this.cacheCfg = cacheCfg;
         this.cacheType = cacheType;
         this.affNode = affNode;
+        this.updatesAllowed = updatesAllowed;
 
         /*
          * Managers in starting order!
@@ -348,7 +354,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     public void awaitStarted() throws IgniteCheckedException {
         U.await(startLatch);
 
-        GridCachePreloader<K, V> prldr = preloader();
+        GridCachePreloader prldr = preloader();
 
         if (prldr != null)
             prldr.startFuture().get();
@@ -361,7 +367,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         if (startLatch.getCount() != 0)
             return false;
 
-        GridCachePreloader<K, V> prldr = preloader();
+        GridCachePreloader prldr = preloader();
 
         return prldr == null || prldr.startFuture().isDone();
     }
@@ -682,7 +688,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     /**
      * @return Preloader.
      */
-    public GridCachePreloader<K, V> preloader() {
+    public GridCachePreloader preloader() {
         return cache().preloader();
     }
 
@@ -1469,9 +1475,6 @@ public class GridCacheContext<K, V> implements Externalizable {
             Collection<ClusterNode> dhtNodeIds = new ArrayList<>(dhtRemoteNodes);
             Collection<ClusterNode> nearNodeIds = F.isEmpty(nearRemoteNodes) ? null : new ArrayList<>(nearRemoteNodes);
 
-            if (!F.isEmpty(nearNodeIds))
-                U.dumpStack("Added near mapped nodes: " + entry + ", " + nearNodeIds);
-
             entry.mappings(explicitLockVer, dhtNodeIds, nearNodeIds);
         }
 
@@ -1809,6 +1812,13 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * @return Updates allowed.
+     */
+    public boolean updatesAllowed() {
+        return updatesAllowed;
+    }
+
+    /**
      * Nulling references to potentially leak-prone objects.
      */
     public void cleanup() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 8d3d089..3857b35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.dr.*;
@@ -943,4 +944,9 @@ public interface GridCacheEntryEx {
      * @return {@code True} if value was removed, {@code false} otherwise.
      */
     public <V> boolean removeMeta(UUID name, V val);
+
+    /**
+     * Calls {@link GridDhtLocalPartition#onUnlock()} for this entry's partition.
+     */
+    public void onUnlock();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
index 9d680ef..d9d151c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
@@ -127,7 +127,7 @@ public class GridCacheGateway<K, V> {
         try {
             GridCacheAdapter<K, V> cache = ctx.cache();
 
-            GridCachePreloader<K, V> preldr = cache != null ? cache.preloader() : null;
+            GridCachePreloader preldr = cache != null ? cache.preloader() : null;
 
             if (preldr == null)
                 throw new IllegalStateException("Grid is in invalid state to perform this operation. " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 02f16c0..eef9fde 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
@@ -472,7 +472,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     req.miniId(),
                     false,
                     0,
-                    req.classError());
+                    req.classError(),
+                    null);
 
                 sendResponseOnFailedMessage(nodeId, res, cctx, ctx.ioPolicy());
             }
@@ -488,7 +489,10 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
                     req.miniId(),
                     req.version(),
                     req.version(),
-                    null, null, null);
+                    null,
+                    null,
+                    null,
+                    null);
 
                 res.error(req.classError());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 92035af..4680994 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -61,6 +61,9 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     private static final byte IS_UNSWAPPED_MASK = 0x02;
 
     /** */
+    private static final byte IS_OFFHEAP_PTR_MASK = 0x04;
+
+    /** */
     public static final GridCacheAtomicVersionComparator ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator();
 
     /**
@@ -433,6 +436,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         if (e.offheapPointer() > 0) {
                             offHeapPointer(e.offheapPointer());
 
+                            flags |= IS_OFFHEAP_PTR_MASK;
+
                             if (needVal) {
                                 CacheObject val = cctx.fromOffheap(offHeapPointer(), false);
 
@@ -498,7 +503,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 return;
             }
 
-            if (val == null && cctx.offheapTiered() && hasOffHeapPointer()) {
+            if (cctx.offheapTiered() && hasOffHeapPointer()) {
                 if (log.isDebugEnabled())
                     log.debug("Value did not change, skip write swap entry: " + this);
 
@@ -509,10 +514,16 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             }
 
             IgniteUuid valClsLdrId = null;
+            IgniteUuid keyClsLdrId = null;
 
-            if (val != null) {
-                valClsLdrId = cctx.deploy().getClassLoaderId(
-                    val.value(cctx.cacheObjectContext(), false).getClass().getClassLoader());
+            if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) {
+                if (val != null) {
+                    valClsLdrId = cctx.deploy().getClassLoaderId(
+                        U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                }
+
+                keyClsLdrId = cctx.deploy().getClassLoaderId(
+                    U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
             }
 
             IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
@@ -523,7 +534,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                 ver,
                 ttlExtras(),
                 expireTime,
-                cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))),
+                keyClsLdrId,
                 valClsLdrId);
 
             if (log.isDebugEnabled())
@@ -3617,6 +3628,8 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                         return true;
                     }
+                    else
+                        evictFailed(prev);
                 }
             }
             else {
@@ -3660,8 +3673,11 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
 
                             return true;
                         }
-                        else
+                        else {
+                            evictFailed(prevVal);
+
                             return false;
+                        }
                     }
                 }
             }
@@ -3680,6 +3696,27 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
         return false;
     }
 
+    /**
+     * @param prevVal Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void evictFailed(@Nullable CacheObject prevVal) throws IgniteCheckedException {
+        if (cctx.offheapTiered() && ((flags & IS_OFFHEAP_PTR_MASK) != 0)) {
+            flags &= ~IS_OFFHEAP_PTR_MASK;
+
+            if (prevVal != null) {
+                cctx.swap().removeOffheap(key());
+
+                value(prevVal);
+
+                GridCacheQueryManager qryMgr = cctx.queries();
+
+                if (qryMgr != null)
+                    qryMgr.onUnswap(key, prevVal);
+            }
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer)
         throws IgniteCheckedException {
@@ -3692,10 +3729,17 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
             if (!hasReaders() && markObsolete0(obsoleteVer, false)) {
                 if (!isStartVersion() && hasValueUnlocked()) {
                     IgniteUuid valClsLdrId = null;
+                    IgniteUuid keyClsLdrId = null;
 
-                    if (val != null)
-                        valClsLdrId = cctx.deploy().getClassLoaderId(
-                            U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                    if (cctx.kernalContext().config().isPeerClassLoadingEnabled()) {
+                        if (val != null) {
+                            valClsLdrId = cctx.deploy().getClassLoaderId(
+                                U.detectObjectClassLoader(val.value(cctx.cacheObjectContext(), false)));
+                        }
+
+                        keyClsLdrId = cctx.deploy().getClassLoaderId(
+                            U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false)));
+                    }
 
                     IgniteBiTuple<byte[], Byte> valBytes = valueBytes0();
 
@@ -3706,7 +3750,7 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
                         ver,
                         ttlExtras(),
                         expireTimeExtras(),
-                        cctx.deploy().getClassLoaderId(U.detectObjectClassLoader(key.value(cctx.cacheObjectContext(), false))),
+                        keyClsLdrId,
                         valClsLdrId);
                 }
 
@@ -4100,6 +4144,11 @@ public abstract class GridCacheMapEntry implements GridCacheEntryEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void onUnlock() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         // Identity comparison left on purpose.
         return o == this;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index c05e4b4..c528e08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -955,6 +955,21 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param topVer Topology version.
+     * @return Locked keys.
+     */
+    public Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> unfinishedLocks(AffinityTopologyVersion topVer) {
+        Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> cands = new HashMap<>();
+
+        for (FinishLockFuture fut : finishFuts) {
+            if (fut.topologyVersion().equals(topVer))
+                cands.putAll(fut.pendingLocks());
+        }
+
+        return cands;
+    }
+
+    /**
      * Creates a future that will wait for all explicit locks acquired on given topology
      * version to be released.
      *
@@ -1041,8 +1056,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         finishFuts.add(finishFut);
 
         finishFut.listen(new CI1<IgniteInternalFuture<?>>() {
-            @Override
-            public void apply(IgniteInternalFuture<?> e) {
+            @Override public void apply(IgniteInternalFuture<?> e) {
                 finishFuts.remove(finishFut);
 
                 // This call is required to make sure that the concurrent queue
@@ -1117,6 +1131,20 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         }
 
         /**
+         * @return Topology version.
+         */
+        AffinityTopologyVersion topologyVersion() {
+            return topVer;
+        }
+
+        /**
+         * @return Pending locks.
+         */
+        Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> pendingLocks() {
+            return pendingLocks;
+        }
+
+        /**
          * @return Filter.
          */
         private IgnitePredicate<GridCacheMvccCandidate> versionFilter() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 0ecaf97..3236bb5 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
@@ -97,7 +97,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     private final AtomicReference<AffinityTopologyVersion> readyTopVer =
         new AtomicReference<>(AffinityTopologyVersion.NONE);
 
-
     /**
      * Partition map futures.
      * This set also contains already completed exchange futures to address race conditions when coordinator
@@ -150,8 +149,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 else {
                     DiscoveryCustomEvent customEvt = (DiscoveryCustomEvent)e;
 
-                    if (customEvt.data() instanceof DynamicCacheChangeBatch) {
-                        DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customEvt.data();
+                    if (customEvt.customMessage() instanceof DynamicCacheChangeBatch) {
+                        DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)customEvt.customMessage();
 
                         Collection<DynamicCacheChangeRequest> valid = new ArrayList<>(batch.requests().size());
 
@@ -554,7 +553,14 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * Partition refresh callback.
      */
     void refreshPartitions() {
-        ClusterNode oldest = CU.oldest(cctx);
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+
+        if (oldest == null) {
+            if (log.isDebugEnabled())
+                log.debug("Skip partitions refresh, there are no server nodes [loc=" + cctx.localNodeId() + ']');
+
+            return;
+        }
 
         if (log.isDebugEnabled())
             log.debug("Refreshing partitions [oldest=" + oldest.id() + ", loc=" + cctx.localNodeId() + ']');
@@ -564,7 +570,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         try {
             // If this is the oldest node.
             if (oldest.id().equals(cctx.localNodeId())) {
-                rmts = CU.remoteNodes(cctx);
+                rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
 
                 if (log.isDebugEnabled())
                     log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
@@ -641,7 +647,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      */
     private boolean sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
         throws IgniteCheckedException {
-        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id, cctx.versions().last());
+        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
+            cctx.kernalContext().clientNode(),
+            cctx.versions().last());
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
             if (!cacheCtx.isLocal()) {
@@ -687,6 +695,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param exchId Exchange ID.
      * @param discoEvt Discovery event.
+     * @param reqs Cache change requests.
      * @return Exchange future.
      */
     GridDhtPartitionsExchangeFuture exchangeFuture(GridDhtPartitionExchangeId exchId,
@@ -696,9 +705,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         GridDhtPartitionsExchangeFuture old = exchFuts.addx(
             fut = new GridDhtPartitionsExchangeFuture(cctx, busyLock, exchId, reqs));
 
-        if (old != null)
+        if (old != null) {
             fut = old;
 
+            if (reqs != null)
+                fut.cacheChangeRequests(reqs);
+        }
+
         if (discoEvt != null)
             fut.onEvent(exchId, discoEvt);
 
@@ -827,7 +840,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @param node Node ID.
      * @param msg Message.
      */
-    private void processSinglePartitionUpdate(ClusterNode node, GridDhtPartitionsSingleMessage msg) {
+    private void processSinglePartitionUpdate(final ClusterNode node, final GridDhtPartitionsSingleMessage msg) {
         if (!enterBusy())
             return;
 
@@ -858,8 +871,22 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 if (updated)
                     scheduleResendPartitions();
             }
-            else
-                exchangeFuture(msg.exchangeId(), null, null).onReceive(node.id(), msg);
+            else {
+                if (msg.client()) {
+                    final GridDhtPartitionsExchangeFuture exchFut = exchangeFuture(msg.exchangeId(),
+                        null,
+                        null);
+
+                    exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                            // Finished future should reply only to sender client node.
+                            exchFut.onReceive(node.id(), msg);
+                        }
+                    });
+                }
+                else
+                    exchangeFuture(msg.exchangeId(), null, null).onReceive(node.id(), msg);
+            }
         }
         finally {
             leaveBusy();
@@ -982,7 +1009,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     busy = true;
 
-                    Map<Integer, GridDhtPreloaderAssignments<K, V>> assignsMap = new HashMap<>();
+                    Map<Integer, GridDhtPreloaderAssignments> assignsMap = null;
 
                     boolean dummyReassign = exchFut.dummyReassign();
                     boolean forcePreload = exchFut.forcePreload();
@@ -1017,7 +1044,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                 changed |= cacheCtx.topology().afterExchange(exchFut);
 
                                 // Preload event notification.
-                                if (cacheCtx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED)) {
+                                if (!exchFut.skipPreload() && cacheCtx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED)) {
                                     if (!cacheCtx.isReplicated() || !startEvtFired) {
                                         DiscoveryEvent discoEvt = exchFut.discoveryEvent();
 
@@ -1043,16 +1070,20 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             }
                         }
 
-                        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                            long delay = cacheCtx.config().getRebalanceDelay();
+                        if (!exchFut.skipPreload()) {
+                            assignsMap = new HashMap<>();
 
-                            GridDhtPreloaderAssignments<K, V> assigns = null;
+                            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                                long delay = cacheCtx.config().getRebalanceDelay();
+
+                                GridDhtPreloaderAssignments assigns = null;
 
-                            // Don't delay for dummy reassigns to avoid infinite recursion.
-                            if (delay == 0 || forcePreload)
-                                assigns = cacheCtx.preloader().assign(exchFut);
+                                // Don't delay for dummy reassigns to avoid infinite recursion.
+                                if (delay == 0 || forcePreload)
+                                    assigns = cacheCtx.preloader().assign(exchFut);
 
-                            assignsMap.put(cacheCtx.cacheId(), assigns);
+                                assignsMap.put(cacheCtx.cacheId(), assigns);
+                            }
                         }
                     }
                     finally {
@@ -1061,7 +1092,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     }
 
                     if (assignsMap != null) {
-                        for (Map.Entry<Integer, GridDhtPreloaderAssignments<K, V>> e : assignsMap.entrySet()) {
+                        for (Map.Entry<Integer, GridDhtPreloaderAssignments> e : assignsMap.entrySet()) {
                             int cacheId = e.getKey();
 
                             GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
@@ -1113,20 +1144,24 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         /** {@inheritDoc} */
         @Override public void onTimeout() {
-            if (!busyLock.readLock().tryLock())
-                return;
+            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                @Override public void run() {
+                    if (!busyLock.readLock().tryLock())
+                        return;
 
-            try {
-                if (started.compareAndSet(false, true))
-                    refreshPartitions();
-            }
-            finally {
-                busyLock.readLock().unlock();
+                    try {
+                        if (started.compareAndSet(false, true))
+                            refreshPartitions();
+                    }
+                    finally {
+                        busyLock.readLock().unlock();
 
-                cctx.time().removeTimeoutObject(this);
+                        cctx.time().removeTimeoutObject(ResendTimeoutObject.this);
 
-                pendingResend.compareAndSet(this, null);
-            }
+                        pendingResend.compareAndSet(ResendTimeoutObject.this, null);
+                    }
+                }
+            });
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 2e181f9..e0f6181 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -30,7 +30,7 @@ import java.util.*;
  * Cache preloader that is responsible for loading cache entries either from remote
  * nodes (for distributed cache) or anywhere else at cache startup.
  */
-public interface GridCachePreloader<K, V> {
+public interface GridCachePreloader {
     /**
      * Starts preloading.
      *
@@ -78,7 +78,7 @@ public interface GridCachePreloader<K, V> {
      * @param exchFut Exchange future to assign.
      * @return Assignments.
      */
-    public GridDhtPreloaderAssignments<K, V> assign(GridDhtPartitionsExchangeFuture exchFut);
+    public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut);
 
     /**
      * Adds assignments to preloader.
@@ -86,7 +86,7 @@ public interface GridCachePreloader<K, V> {
      * @param assignments Assignments to add.
      * @param forcePreload Force preload flag.
      */
-    public void addAssignments(GridDhtPreloaderAssignments<K, V> assignments, boolean forcePreload);
+    public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload);
 
     /**
      * @param p Preload predicate.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 80d3d6b..b4f386f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -31,9 +31,9 @@ import java.util.*;
 /**
  * Adapter for preloading which always assumes that preloading finished.
  */
-public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V> {
+public class GridCachePreloaderAdapter implements GridCachePreloader {
     /** Cache context. */
-    protected final GridCacheContext<K, V> cctx;
+    protected final GridCacheContext<?, ?> cctx;
 
     /** Logger.*/
     protected final IgniteLogger log;
@@ -50,7 +50,7 @@ public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V>
     /**
      * @param cctx Cache context.
      */
-    public GridCachePreloaderAdapter(GridCacheContext<K, V> cctx) {
+    public GridCachePreloaderAdapter(GridCacheContext<?, ?> cctx) {
         assert cctx != null;
 
         this.cctx = cctx;
@@ -126,17 +126,18 @@ public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V>
         // No-op.
     }
 
+    /** {@inheritDoc} */
     @Override public void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public GridDhtPreloaderAssignments<K, V> assign(GridDhtPartitionsExchangeFuture exchFut) {
+    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public void addAssignments(GridDhtPreloaderAssignments<K, V> assignments, boolean forcePreload) {
+    @Override public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload) {
         // No-op.
     }
 }



[03/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
new file mode 100644
index 0000000..a277fc8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteStreamSelfTestSuite.java
@@ -0,0 +1,39 @@
+/*
+ * 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 org.apache.ignite.stream.socket.*;
+
+import junit.framework.*;
+
+/**
+ * Stream test suite.
+ */
+public class IgniteStreamSelfTestSuite extends TestSuite {
+    /**
+     * @return Stream tests suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Stream Test Suite");
+
+        suite.addTest(new TestSuite(SocketStreamerSelfTest.class));
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
index 32cd038..1c75a7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteUtilSelfTestSuite.java
@@ -67,7 +67,7 @@ public class IgniteUtilSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridNioSelfTest.class);
         suite.addTestSuite(GridNioFilterChainSelfTest.class);
         suite.addTestSuite(GridNioSslSelfTest.class);
-        suite.addTestSuite(GridNioDelimitedBufferTest.class);
+        suite.addTestSuite(GridNioDelimitedBufferSelfTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java b/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
index b496f60..48991e8 100644
--- a/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
+++ b/modules/gce/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/gce/TcpDiscoveryGoogleStorageIpFinder.java
@@ -68,34 +68,37 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
  * Note that this finder is shared by default (see {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()}.
  */
 public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapter {
-    /* Default object's content. */
+    /** Default object's content. */
     private final static ByteArrayInputStream OBJECT_CONTENT =  new ByteArrayInputStream(new byte[0]);
 
     /** Grid logger. */
     @LoggerResource
     private IgniteLogger log;
 
-    /* Google Cloud Platform's project name.*/
+    /** Google Cloud Platform's project name.*/
     private String projectName;
 
-    /* Google Storage bucket name. */
+    /** Google Storage bucket name. */
     private String bucketName;
 
-    /* Service account p12 private key file name. */
-    private String serviceAccountP12FilePath;
+    /** Service account p12 private key file name. */
+    private String srvcAccountP12FilePath;
 
-    /* Service account id. */
-    private String serviceAccountId;
+    /** Service account id. */
+    private String srvcAccountId;
 
-    /* Google storage. */
+    /** Google storage. */
     private Storage storage;
 
-    /* Init routine guard. */
+    /** Init routine guard. */
     private final AtomicBoolean initGuard = new AtomicBoolean();
 
-    /* Init routine latch. */
+    /** Init routine latch. */
     private final CountDownLatch initLatch = new CountDownLatch(1);
 
+    /**
+     *
+     */
     public TcpDiscoveryGoogleStorageIpFinder() {
         setShared(true);
     }
@@ -221,7 +224,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     @IgniteSpiConfiguration(optional = false)
     public void setServiceAccountP12FilePath(String p12FileName) {
-        this.serviceAccountP12FilePath = p12FileName;
+        this.srvcAccountP12FilePath = p12FileName;
     }
 
     /**
@@ -235,7 +238,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     @IgniteSpiConfiguration(optional = false)
     public void setServiceAccountId(String id) {
-        this.serviceAccountId = id;
+        this.srvcAccountId = id;
     }
 
     /**
@@ -245,13 +248,13 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
      */
     private void init() throws IgniteSpiException {
         if (initGuard.compareAndSet(false, true)) {
-            if (serviceAccountId == null ||
-                serviceAccountP12FilePath == null ||
+            if (srvcAccountId == null ||
+                srvcAccountP12FilePath == null ||
                 projectName == null ||
                 bucketName == null) {
                 throw new IgniteSpiException(
                     "One or more of the required parameters is not set [serviceAccountId=" +
-                        serviceAccountId + ", serviceAccountP12FilePath=" + serviceAccountP12FilePath + ", projectName=" +
+                        srvcAccountId + ", serviceAccountP12FilePath=" + srvcAccountP12FilePath + ", projectName=" +
                         projectName + ", bucketName=" + bucketName + "]");
             }
 
@@ -265,12 +268,12 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
                     throw new IgniteSpiException(e);
                 }
 
-                GoogleCredential credential;
+                GoogleCredential cred;
 
                 try {
-                    credential = new GoogleCredential.Builder().setTransport(httpTransport)
-                        .setJsonFactory(JacksonFactory.getDefaultInstance()).setServiceAccountId(serviceAccountId)
-                        .setServiceAccountPrivateKeyFromP12File(new File(serviceAccountP12FilePath))
+                    cred = new GoogleCredential.Builder().setTransport(httpTransport)
+                        .setJsonFactory(JacksonFactory.getDefaultInstance()).setServiceAccountId(srvcAccountId)
+                        .setServiceAccountPrivateKeyFromP12File(new File(srvcAccountP12FilePath))
                         .setServiceAccountScopes(Collections.singleton(StorageScopes.DEVSTORAGE_FULL_CONTROL)).build();
 
                 }
@@ -279,7 +282,7 @@ public class TcpDiscoveryGoogleStorageIpFinder extends TcpDiscoveryIpFinderAdapt
                 }
 
                 try {
-                    storage = new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), credential)
+                    storage = new Storage.Builder(httpTransport, JacksonFactory.getDefaultInstance(), cred)
                         .setApplicationName(projectName).build();
                 }
                 catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 66e9761..d910507 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
@@ -20,10 +20,12 @@ package org.apache.ignite.hadoop.fs;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.*;
+import org.apache.hadoop.mapreduce.*;
 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.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 
 import java.io.*;
@@ -37,9 +39,6 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     public static final String PERFORMANCE_COUNTER_FILE_NAME = "performance";
 
     /** */
-    private static final String DEFAULT_USER_NAME = "anonymous";
-
-    /** */
     public static final String COUNTER_WRITER_DIR_PROPERTY = "ignite.counters.fswriter.directory";
 
     /** */
@@ -52,15 +51,14 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     @Override public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs)
         throws IgniteCheckedException {
 
-        Configuration hadoopCfg = new Configuration();
+        Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration();
 
         for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
             hadoopCfg.set(e.getKey(), e.getValue());
 
         String user = jobInfo.user();
 
-        if (F.isEmpty(user))
-            user = DEFAULT_USER_NAME;
+        user = IgfsUtils.fixUserName(user);
 
         String dir = jobInfo.property(COUNTER_WRITER_DIR_PROPERTY);
 
@@ -72,7 +70,9 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
         HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(cntrs, null);
 
         try {
-            FileSystem fs = jobStatPath.getFileSystem(hadoopCfg);
+            hadoopCfg.set(MRJobConfig.USER_NAME, user);
+
+            FileSystem fs = HadoopUtils.fileSystemForMrUser(jobStatPath.toUri(), hadoopCfg, true);
 
             fs.mkdirs(jobStatPath);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
index ba891f8..6a630fb 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopIgfsSecondaryFileSystem.java
@@ -20,15 +20,16 @@ package org.apache.ignite.hadoop.fs;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.ipc.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.processors.hadoop.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.jetbrains.annotations.*;
+import org.apache.ignite.internal.processors.hadoop.fs.HadoopLazyConcurrentMap.*;
 
 import java.io.*;
 import java.net.*;
@@ -37,15 +38,45 @@ import java.util.*;
 import static org.apache.ignite.internal.processors.igfs.IgfsEx.*;
 
 /**
- * Adapter to use any Hadoop file system {@link FileSystem} as  {@link IgfsSecondaryFileSystem}.
+ * Adapter to use any Hadoop file system {@link FileSystem} as {@link IgfsSecondaryFileSystem}.
+ * In fact, this class deals with different FileSystems depending on the user context,
+ * see {@link IgfsUserContext#currentUser()}.
  */
-public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem, AutoCloseable {
-    /** Hadoop file system. */
-    private final FileSystem fileSys;
-
-    /** Properties of file system */
+public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSystem {
+    /** Properties of file system, see {@link #properties()}
+     *
+     * See {@link IgfsEx#SECONDARY_FS_CONFIG_PATH}
+     * See {@link IgfsEx#SECONDARY_FS_URI}
+     * See {@link IgfsEx#SECONDARY_FS_USER_NAME}
+     * */
     private final Map<String, String> props = new HashMap<>();
 
+    /** Secondary file system provider. */
+    private final SecondaryFileSystemProvider secProvider;
+
+    /** The default user name. It is used if no user context is set. */
+    private final String dfltUserName;
+
+    /** FileSystem instance created for the default user.
+     * Stored outside the fileSysLazyMap due to performance reasons. */
+    private final FileSystem dfltFs;
+
+    /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
+    private final HadoopLazyConcurrentMap<String, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
+        new ValueFactory<String, FileSystem>() {
+            @Override public FileSystem createValue(String key) {
+                try {
+                    assert !F.isEmpty(key);
+
+                    return secProvider.createFileSystem(key);
+                }
+                catch (IOException ioe) {
+                    throw new IgniteException(ioe);
+                }
+            }
+        }
+    );
+
     /**
      * Simple constructor that is to be used by default.
      *
@@ -77,8 +108,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @throws IgniteCheckedException In case of error.
      */
     public IgniteHadoopIgfsSecondaryFileSystem(@Nullable String uri, @Nullable String cfgPath,
-        @Nullable String userName)
-            throws IgniteCheckedException {
+        @Nullable String userName) throws IgniteCheckedException {
         // Treat empty uri and userName arguments as nulls to improve configuration usability:
         if (F.isEmpty(uri))
             uri = null;
@@ -89,27 +119,31 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         if (F.isEmpty(userName))
             userName = null;
 
+        this.dfltUserName = IgfsUtils.fixUserName(userName);
+
         try {
-            SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(uri, cfgPath, userName);
+            this.secProvider = new SecondaryFileSystemProvider(uri, cfgPath);
 
-            fileSys = secProvider.createFileSystem();
+            // File system creation for the default user name.
+            // The value is *not* stored in the 'fileSysLazyMap' cache, but saved in field:
+            this.dfltFs = secProvider.createFileSystem(dfltUserName);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
 
-            uri = secProvider.uri().toString();
+        assert dfltFs != null;
 
-            if (!uri.endsWith("/"))
-                uri += "/";
+        uri = secProvider.uri().toString();
 
-            if (cfgPath != null)
-                props.put(SECONDARY_FS_CONFIG_PATH, cfgPath);
+        if (!uri.endsWith("/"))
+            uri += "/";
 
-            if (userName != null)
-                props.put(SECONDARY_FS_USER_NAME, userName);
+        if (cfgPath != null)
+            props.put(SECONDARY_FS_CONFIG_PATH, cfgPath);
 
-            props.put(SECONDARY_FS_URI, uri);
-        }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
-        }
+        props.put(SECONDARY_FS_URI, uri);
+        props.put(SECONDARY_FS_USER_NAME, dfltUserName);
     }
 
     /**
@@ -119,7 +153,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @return Hadoop path.
      */
     private Path convert(IgfsPath path) {
-        URI uri = fileSys.getUri();
+        URI uri = fileSysForUser().getUri();
 
         return new Path(uri.getScheme(), uri.getAuthority(), path.toString());
     }
@@ -131,14 +165,9 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
      * @param detailMsg Detailed error message.
      * @return Appropriate exception.
      */
-    @SuppressWarnings({"ThrowableResultOfMethodCallIgnored", "unchecked"})
     private IgfsException handleSecondaryFsError(IOException e, String detailMsg) {
-        boolean wrongVer = X.hasCause(e, RemoteException.class) ||
-            (e.getMessage() != null && e.getMessage().contains("Failed on local"));
-
-        return !wrongVer ? cast(detailMsg, e) :
-            new IgfsInvalidHdfsVersionException("HDFS version you are connecting to differs from local " +
-                "version.", e);    }
+        return cast(detailMsg, e);
+    }
 
     /**
      * Cast IO exception to IGFS exception.
@@ -178,7 +207,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public boolean exists(IgfsPath path) {
         try {
-            return fileSys.exists(convert(path));
+            return fileSysForUser().exists(convert(path));
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to check file existence [path=" + path + "]");
@@ -189,6 +218,8 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Nullable @Override public IgfsFile update(IgfsPath path, Map<String, String> props) {
         HadoopIgfsProperties props0 = new HadoopIgfsProperties(props);
 
+        final FileSystem fileSys = fileSysForUser();
+
         try {
             if (props0.userName() != null || props0.groupName() != null)
                 fileSys.setOwner(convert(path), props0.userName(), props0.groupName());
@@ -208,7 +239,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Override public void rename(IgfsPath src, IgfsPath dest) {
         // Delegate to the secondary file system.
         try {
-            if (!fileSys.rename(convert(src), convert(dest)))
+            if (!fileSysForUser().rename(convert(src), convert(dest)))
                 throw new IgfsException("Failed to rename (secondary file system returned false) " +
                     "[src=" + src + ", dest=" + dest + ']');
         }
@@ -220,7 +251,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public boolean delete(IgfsPath path, boolean recursive) {
         try {
-            return fileSys.delete(convert(path), recursive);
+            return fileSysForUser().delete(convert(path), recursive);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to delete file [path=" + path + ", recursive=" + recursive + "]");
@@ -230,7 +261,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path) {
         try {
-            if (!fileSys.mkdirs(convert(path)))
+            if (!fileSysForUser().mkdirs(convert(path)))
                 throw new IgniteException("Failed to make directories [path=" + path + "]");
         }
         catch (IOException e) {
@@ -241,7 +272,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path, @Nullable Map<String, String> props) {
         try {
-            if (!fileSys.mkdirs(convert(path), new HadoopIgfsProperties(props).permission()))
+            if (!fileSysForUser().mkdirs(convert(path), new HadoopIgfsProperties(props).permission()))
                 throw new IgniteException("Failed to make directories [path=" + path + ", props=" + props + "]");
         }
         catch (IOException e) {
@@ -252,7 +283,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public Collection<IgfsPath> listPaths(IgfsPath path) {
         try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
+            FileStatus[] statuses = fileSysForUser().listStatus(convert(path));
 
             if (statuses == null)
                 throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
@@ -275,7 +306,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public Collection<IgfsFile> listFiles(IgfsPath path) {
         try {
-            FileStatus[] statuses = fileSys.listStatus(convert(path));
+            FileStatus[] statuses = fileSysForUser().listStatus(convert(path));
 
             if (statuses == null)
                 throw new IgfsPathNotFoundException("Failed to list files (path not found): " + path);
@@ -302,13 +333,13 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
 
     /** {@inheritDoc} */
     @Override public IgfsSecondaryFileSystemPositionedReadable open(IgfsPath path, int bufSize) {
-        return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSys, convert(path), bufSize);
+        return new HadoopIgfsSecondaryFileSystemPositionedReadable(fileSysForUser(), convert(path), bufSize);
     }
 
     /** {@inheritDoc} */
     @Override public OutputStream create(IgfsPath path, boolean overwrite) {
         try {
-            return fileSys.create(convert(path), overwrite);
+            return fileSysForUser().create(convert(path), overwrite);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", overwrite=" + overwrite + "]");
@@ -322,8 +353,8 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
             new HadoopIgfsProperties(props != null ? props : Collections.<String, String>emptyMap());
 
         try {
-            return fileSys.create(convert(path), props0.permission(), overwrite, bufSize, (short)replication, blockSize,
-                null);
+            return fileSysForUser().create(convert(path), props0.permission(), overwrite, bufSize,
+                (short)replication, blockSize, null);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to create file [path=" + path + ", props=" + props +
@@ -336,7 +367,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     @Override public OutputStream append(IgfsPath path, int bufSize, boolean create,
         @Nullable Map<String, String> props) {
         try {
-            return fileSys.append(convert(path), bufSize);
+            return fileSysForUser().append(convert(path), bufSize);
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to append file [path=" + path + ", bufSize=" + bufSize + "]");
@@ -346,7 +377,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     /** {@inheritDoc} */
     @Override public IgfsFile info(final IgfsPath path) {
         try {
-            final FileStatus status = fileSys.getFileStatus(convert(path));
+            final FileStatus status = fileSysForUser().getFileStatus(convert(path));
 
             if (status == null)
                 return null;
@@ -421,7 +452,7 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
         try {
             // We don't use FileSystem#getUsed() since it counts only the files
             // in the filesystem root, not all the files recursively.
-            return fileSys.getContentSummary(new Path("/")).getSpaceConsumed();
+            return fileSysForUser().getContentSummary(new Path("/")).getSpaceConsumed();
         }
         catch (IOException e) {
             throw handleSecondaryFsError(e, "Failed to get used space size of file system.");
@@ -429,25 +460,57 @@ public class IgniteHadoopIgfsSecondaryFileSystem implements IgfsSecondaryFileSys
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<String, String> properties() {
+    @Override public Map<String, String> properties() {
         return props;
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
+    @Override public void close() throws IgniteException {
+        Exception e = null;
+
         try {
-            fileSys.close();
+            dfltFs.close();
         }
-        catch (IOException e) {
-            throw new IgniteCheckedException(e);
+        catch (Exception e0) {
+            e = e0;
+        }
+
+        try {
+            fileSysLazyMap.close();
+        }
+        catch (IgniteCheckedException ice) {
+            if (e == null)
+                e = ice;
         }
+
+        if (e != null)
+            throw new IgniteException(e);
     }
 
     /**
      * Gets the underlying {@link FileSystem}.
+     * This method is used solely for testing.
      * @return the underlying Hadoop {@link FileSystem}.
      */
     public FileSystem fileSystem() {
-        return fileSys;
+        return fileSysForUser();
+    }
+
+    /**
+     * Gets the FileSystem for the current context user.
+     * @return the FileSystem instance, never null.
+     */
+    private FileSystem fileSysForUser() {
+        String user = IgfsUserContext.currentUser();
+
+        if (F.isEmpty(user))
+            user = dfltUserName; // default is never empty.
+
+        assert !F.isEmpty(user);
+
+        if (F.eq(user, dfltUserName))
+            return dfltFs; // optimization
+
+        return fileSysLazyMap.getOrCreate(user);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
index 1f53a06..9d94e5b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v1/IgniteHadoopFileSystem.java
@@ -22,7 +22,7 @@ import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
 import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.security.*;
 import org.apache.hadoop.util.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
@@ -97,21 +97,8 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Grid remote client. */
     private HadoopIgfsWrapper rmtClient;
 
-    /** User name for each thread. */
-    private final ThreadLocal<String> userName = new ThreadLocal<String>(){
-        /** {@inheritDoc} */
-        @Override protected String initialValue() {
-            return DFLT_USER_NAME;
-        }
-    };
-
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>(){
-        /** {@inheritDoc} */
-        @Override protected Path initialValue() {
-            return getHomeDirectory();
-        }
-    };
+    /** working directory. */
+    private Path workingDir;
 
     /** Default replication factor. */
     private short dfltReplication;
@@ -129,6 +116,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Secondary URI string. */
     private URI secondaryUri;
 
+    /** The user name this file system was created on behalf of. */
+    private String user;
+
     /** IGFS mode resolver. */
     private IgfsModeResolver modeRslvr;
 
@@ -153,9 +143,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
     /** Custom-provided sequential reads before prefetch. */
     private int seqReadsBeforePrefetch;
 
-    /** The cache was disabled when the instance was creating. */
-    private boolean cacheEnabled;
-
     /** {@inheritDoc} */
     @Override public URI getUri() {
         if (uri == null)
@@ -182,6 +169,22 @@ public class IgniteHadoopFileSystem extends FileSystem {
     }
 
     /**
+     * Gets non-null user name as per the Hadoop file system viewpoint.
+     * @return the user name, never null.
+     */
+    public static String getFsHadoopUser() throws IOException {
+        UserGroupInformation currUgi = UserGroupInformation.getCurrentUser();
+
+        String user = currUgi.getShortUserName();
+
+        user = IgfsUtils.fixUserName(user);
+
+        assert user != null;
+
+        return user;
+    }
+
+    /**
      * Public setter that can be used by direct users of FS or Visor.
      *
      * @param colocateFileWrites Whether all ongoing file writes should be colocated.
@@ -207,10 +210,6 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             setConf(cfg);
 
-            String disableCacheName = String.format("fs.%s.impl.disable.cache", name.getScheme());
-
-            cacheEnabled = !cfg.getBoolean(disableCacheName, false);
-
             mgmt = cfg.getBoolean(IGFS_MANAGEMENT, false);
 
             if (!IGFS_SCHEME.equals(name.getScheme()))
@@ -221,7 +220,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             uriAuthority = uri.getAuthority();
 
-            setUser(cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+            user = getFsHadoopUser();
 
             // Override sequential reads before prefetch if needed.
             seqReadsBeforePrefetch = parameter(cfg, PARAM_IGFS_SEQ_READS_BEFORE_PREFETCH, uriAuthority, 0);
@@ -244,7 +243,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG, user);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -289,13 +288,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-                String secUserName = props.get(SECONDARY_FS_USER_NAME);
 
                 try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath,
-                        secUserName);
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createFileSystem(user);
 
-                    secondaryFs = secProvider.createFileSystem();
                     secondaryUri = secProvider.uri();
                 }
                 catch (IOException e) {
@@ -306,6 +304,9 @@ public class IgniteHadoopFileSystem extends FileSystem {
                             "will have no effect): " + e.getMessage());
                 }
             }
+
+            // set working directory to the home directory of the current Fs user:
+            setWorkingDirectory(null);
         }
         finally {
             leaveBusy();
@@ -337,15 +338,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
     @Override protected void finalize() throws Throwable {
         super.finalize();
 
-        close0();
+        close();
     }
 
     /** {@inheritDoc} */
     @Override public void close() throws IOException {
-        if (cacheEnabled && get(getUri(), getConf()) == this)
-            return;
-
-        close0();
+        if (closeGuard.compareAndSet(false, true))
+            close0();
     }
 
     /**
@@ -354,27 +353,25 @@ public class IgniteHadoopFileSystem extends FileSystem {
      * @throws IOException If failed.
      */
     private void close0() throws IOException {
-        if (closeGuard.compareAndSet(false, true)) {
-            if (LOG.isDebugEnabled())
-                LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
+        if (LOG.isDebugEnabled())
+            LOG.debug("File system closed [uri=" + uri + ", endpoint=" + uriAuthority + ']');
 
-            if (rmtClient == null)
-                return;
+        if (rmtClient == null)
+            return;
 
-            super.close();
+        super.close();
 
-            rmtClient.close(false);
+        rmtClient.close(false);
 
-            if (clientLog.isLogEnabled())
-                clientLog.close();
+        if (clientLog.isLogEnabled())
+            clientLog.close();
 
-            if (secondaryFs != null)
-                U.closeQuiet(secondaryFs);
+        if (secondaryFs != null)
+            U.closeQuiet(secondaryFs);
 
-            // Reset initialized resources.
-            uri = null;
-            rmtClient = null;
-        }
+        // Reset initialized resources.
+        uri = null;
+        rmtClient = null;
     }
 
     /** {@inheritDoc} */
@@ -849,22 +846,11 @@ public class IgniteHadoopFileSystem extends FileSystem {
 
     /** {@inheritDoc} */
     @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
+        Path path = new Path("/user/" + user);
 
         return path.makeQualified(getUri(), null);
     }
 
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(null);
-    }
-
     /** {@inheritDoc} */
     @Override public void setWorkingDirectory(Path newPath) {
         if (newPath == null) {
@@ -873,7 +859,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
             if (secondaryFs != null)
                 secondaryFs.setWorkingDirectory(toSecondary(homeDir));
 
-            workingDir.set(homeDir);
+            workingDir = homeDir;
         }
         else {
             Path fixedNewPath = fixRelativePart(newPath);
@@ -886,13 +872,13 @@ public class IgniteHadoopFileSystem extends FileSystem {
             if (secondaryFs != null)
                 secondaryFs.setWorkingDirectory(toSecondary(fixedNewPath));
 
-            workingDir.set(fixedNewPath);
+            workingDir = fixedNewPath;
         }
     }
 
     /** {@inheritDoc} */
     @Override public Path getWorkingDirectory() {
-        return workingDir.get();
+        return workingDir;
     }
 
     /** {@inheritDoc} */
@@ -1153,7 +1139,7 @@ public class IgniteHadoopFileSystem extends FileSystem {
             return null;
 
         return path.isAbsolute() ? new IgfsPath(path.toUri().getPath()) :
-            new IgfsPath(convert(workingDir.get()), path.toUri().getPath());
+            new IgfsPath(convert(workingDir), path.toUri().getPath());
     }
 
     /**
@@ -1191,9 +1177,16 @@ public class IgniteHadoopFileSystem extends FileSystem {
      */
     @SuppressWarnings("deprecation")
     private FileStatus convert(IgfsFile file) {
-        return new FileStatus(file.length(), file.isDirectory(), getDefaultReplication(),
-            file.groupBlockSize(), file.modificationTime(), file.accessTime(), permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME), file.property(PROP_GROUP_NAME, "users"),
+        return new FileStatus(
+            file.length(),
+            file.isDirectory(),
+            getDefaultReplication(),
+            file.groupBlockSize(),
+            file.modificationTime(),
+            file.accessTime(),
+            permission(file),
+            file.property(PROP_USER_NAME, user),
+            file.property(PROP_GROUP_NAME, "users"),
             convert(file.path())) {
             @Override public String toString() {
                 return "FileStatus [path=" + getPath() + ", isDir=" + isDir() + ", len=" + getLen() +
@@ -1247,4 +1240,12 @@ public class IgniteHadoopFileSystem extends FileSystem {
     @Override public String toString() {
         return S.toString(IgniteHadoopFileSystem.class, this);
     }
+
+    /**
+     * Returns the user name this File System is created on behalf of.
+     * @return the user name
+     */
+    public String user() {
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
index 9cfb79b..8330143 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/v2/IgniteHadoopFileSystem.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.Options;
 import org.apache.hadoop.fs.permission.*;
-import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.util.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
@@ -40,6 +39,7 @@ import java.util.*;
 import java.util.concurrent.atomic.*;
 
 import static org.apache.ignite.configuration.FileSystemConfiguration.*;
+import static org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem.*;
 import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils.*;
@@ -91,11 +91,14 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
     /** Grid remote client. */
     private HadoopIgfsWrapper rmtClient;
 
+    /** The name of the user this File System created on behalf of. */
+    private final String user;
+
     /** Working directory. */
     private IgfsPath workingDir;
 
     /** URI. */
-    private URI uri;
+    private final URI uri;
 
     /** Authority. */
     private String uriAuthority;
@@ -141,6 +144,8 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
         uri = name;
 
+        user = getFsHadoopUser();
+
         try {
             initialize(name, cfg);
         }
@@ -152,7 +157,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             throw e;
         }
 
-        workingDir = new IgfsPath("/user/" + cfg.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
+        workingDir = new IgfsPath("/user/" + user);
     }
 
     /** {@inheritDoc} */
@@ -240,7 +245,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
             String logDir = logDirFile != null ? logDirFile.getAbsolutePath() : null;
 
-            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG);
+            rmtClient = new HadoopIgfsWrapper(uriAuthority, logDir, cfg, LOG, user);
 
             // Handshake.
             IgfsHandshakeResponse handshake = rmtClient.handshake(logDir);
@@ -284,13 +289,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
 
                 String secUri = props.get(SECONDARY_FS_URI);
                 String secConfPath = props.get(SECONDARY_FS_CONFIG_PATH);
-                String secUserName = props.get(SECONDARY_FS_USER_NAME);
 
                 try {
-                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath,
-                        secUserName);
+                    SecondaryFileSystemProvider secProvider = new SecondaryFileSystemProvider(secUri, secConfPath);
+
+                    secondaryFs = secProvider.createAbstractFileSystem(user);
 
-                    secondaryFs = secProvider.createAbstractFileSystem();
                     secondaryUri = secProvider.uri();
                 }
                 catch (IOException e) {
@@ -929,7 +933,7 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
             file.modificationTime(),
             file.accessTime(),
             permission(file),
-            file.property(PROP_USER_NAME, DFLT_USER_NAME),
+            file.property(PROP_USER_NAME, user),
             file.property(PROP_GROUP_NAME, "users"),
             convert(file.path())) {
             @Override public String toString() {
@@ -983,4 +987,12 @@ public class IgniteHadoopFileSystem extends AbstractFileSystem implements Closea
     @Override public String toString() {
         return S.toString(IgniteHadoopFileSystem.class, this);
     }
-}
+
+    /**
+     * Returns the user name this File System is created on behalf of.
+     * @return the user name
+     */
+    public String user() {
+        return user;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 d0a327e..2e855d0 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
@@ -89,7 +89,7 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
             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-main");
+                        HadoopClassLoader ldr = new HadoopClassLoader(null, "hadoop-job");
 
                         jobCls = jobCls0 = ldr.loadClass(HadoopV2Job.class.getName());
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 00be422..68a9ef6 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,10 +26,16 @@ 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.*;
 
 /**
@@ -57,6 +63,41 @@ 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.
+     */
+    private HadoopUtils() {
+        // No-op.
+    }
+
     /**
      * Wraps native split.
      *
@@ -126,11 +167,13 @@ public class HadoopUtils {
                 break;
 
             case PHASE_REDUCE:
-                assert status.totalReducerCnt() > 0;
-
                 setupProgress = 1;
                 mapProgress = 1;
-                reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
+
+                if (status.totalReducerCnt() > 0)
+                    reduceProgress = 1f - status.pendingReducerCnt() / (float)status.totalReducerCnt();
+                else
+                    reduceProgress = 1f;
 
                 break;
 
@@ -300,9 +343,242 @@ public class HadoopUtils {
     }
 
     /**
-     * Constructor.
+     * Creates {@link Configuration} in a correct class loader context to avoid caching
+     * of inappropriate class loader in the Configuration object.
+     * @return New instance of {@link Configuration}.
      */
-    private HadoopUtils() {
-        // No-op.
+    public static Configuration safeCreateConfiguration() {
+        final ClassLoader cl0 = Thread.currentThread().getContextClassLoader();
+
+        Thread.currentThread().setContextClassLoader(Configuration.class.getClassLoader());
+
+        try {
+            return new Configuration();
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(cl0);
+        }
+    }
+
+    /**
+     * Creates {@link JobConf} in a correct class loader context to avoid caching
+     * of inappropriate class loader in the Configuration object.
+     * @return New instance of {@link JobConf}.
+     */
+    public static JobConf safeCreateJobConf() {
+        final ClassLoader cl0 = Thread.currentThread().getContextClassLoader();
+
+        Thread.currentThread().setContextClassLoader(JobConf.class.getClassLoader());
+
+        try {
+            return new JobConf();
+        }
+        finally {
+            Thread.currentThread().setContextClassLoader(cl0);
+        }
+    }
+
+    /**
+     * 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/1652fd18/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 27805f8..dd679de 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
@@ -19,26 +19,26 @@ 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.igfs.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.net.*;
+import java.security.*;
 
 /**
  * Encapsulates logic of secondary filesystem creation.
  */
 public class SecondaryFileSystemProvider {
     /** Configuration of the secondary filesystem, never null. */
-    private final Configuration cfg = new Configuration();
+    private final Configuration cfg = HadoopUtils.safeCreateConfiguration();
 
     /** The secondary filesystem URI, never null. */
     private final URI uri;
 
-    /** Optional user name to log into secondary filesystem with. */
-    private @Nullable final String userName;
-
     /**
      * Creates new provider with given config parameters. The configuration URL is optional. The filesystem URI must be
      * specified either explicitly or in the configuration provided.
@@ -47,13 +47,10 @@ public class SecondaryFileSystemProvider {
      * property in the provided configuration.
      * @param secConfPath the secondary Fs path (file path on the local file system, optional).
      * See {@link IgniteUtils#resolveIgniteUrl(String)} on how the path resolved.
-     * @param userName User name.
      * @throws IOException
      */
     public SecondaryFileSystemProvider(final @Nullable String secUri,
-        final @Nullable String secConfPath, @Nullable String userName) throws IOException {
-        this.userName = userName;
-
+        final @Nullable String secConfPath) throws IOException {
         if (secConfPath != null) {
             URL url = U.resolveIgniteUrl(secConfPath);
 
@@ -79,7 +76,7 @@ public class SecondaryFileSystemProvider {
         }
 
         // Disable caching:
-        String prop = String.format("fs.%s.impl.disable.cache", uri.getScheme());
+        String prop = HadoopUtils.disableFsCachePropertyName(uri.getScheme());
 
         cfg.setBoolean(prop, true);
     }
@@ -88,20 +85,18 @@ public class SecondaryFileSystemProvider {
      * @return {@link org.apache.hadoop.fs.FileSystem}  instance for this secondary Fs.
      * @throws IOException
      */
-    public FileSystem createFileSystem() throws IOException {
+    public FileSystem createFileSystem(String userName) throws IOException {
+        userName = IgfsUtils.fixUserName(userName);
+
         final FileSystem fileSys;
 
-        if (userName == null)
-            fileSys = FileSystem.get(uri, cfg);
-        else {
-            try {
-                fileSys = FileSystem.get(uri, cfg, userName);
-            }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
+        try {
+           fileSys = FileSystem.get(uri, cfg, userName);
+        }
+        catch (InterruptedException e) {
+           Thread.currentThread().interrupt();
 
-                throw new IOException("Failed to create file system due to interrupt.", e);
-            }
+           throw new IOException("Failed to create file system due to interrupt.", e);
         }
 
         return fileSys;
@@ -109,10 +104,26 @@ public class SecondaryFileSystemProvider {
 
     /**
      * @return {@link org.apache.hadoop.fs.AbstractFileSystem} instance for this secondary Fs.
-     * @throws IOException
+     * @throws IOException in case of error.
      */
-    public AbstractFileSystem createAbstractFileSystem() throws IOException {
-        return AbstractFileSystem.get(uri, cfg);
+    public AbstractFileSystem createAbstractFileSystem(String userName) throws IOException {
+        userName = IgfsUtils.fixUserName(userName);
+
+        String ticketCachePath = cfg.get(CommonConfigurationKeys.KERBEROS_TICKET_CACHE_PATH);
+
+        UserGroupInformation ugi = UserGroupInformation.getBestUGI(ticketCachePath, userName);
+
+        try {
+            return ugi.doAs(new PrivilegedExceptionAction<AbstractFileSystem>() {
+                @Override public AbstractFileSystem run() throws IOException {
+                    return AbstractFileSystem.get(uri, cfg);
+                }
+            });
+        } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+
+            throw new IOException("Failed to create file system due to interrupt.", ie);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
deleted file mode 100644
index 509f443..0000000
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopDistributedFileSystem.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.hadoop.fs;
-
-import org.apache.hadoop.conf.*;
-import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.mapreduce.*;
-
-import java.io.*;
-import java.net.*;
-
-import static org.apache.ignite.configuration.FileSystemConfiguration.*;
-
-/**
- * Wrapper of HDFS for support of separated working directory.
- */
-public class HadoopDistributedFileSystem extends DistributedFileSystem {
-    /** User name for each thread. */
-    private final ThreadLocal<String> userName = new ThreadLocal<String>() {
-        /** {@inheritDoc} */
-        @Override protected String initialValue() {
-            return DFLT_USER_NAME;
-        }
-    };
-
-    /** Working directory for each thread. */
-    private final ThreadLocal<Path> workingDir = new ThreadLocal<Path>() {
-        /** {@inheritDoc} */
-        @Override protected Path initialValue() {
-            return getHomeDirectory();
-        }
-    };
-
-    /** {@inheritDoc} */
-    @Override public void initialize(URI uri, Configuration conf) throws IOException {
-        super.initialize(uri, conf);
-
-        setUser(conf.get(MRJobConfig.USER_NAME, DFLT_USER_NAME));
-    }
-
-    /**
-     * Set user name and default working directory for current thread.
-     *
-     * @param userName User name.
-     */
-    public void setUser(String userName) {
-        this.userName.set(userName);
-
-        setWorkingDirectory(getHomeDirectory());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getHomeDirectory() {
-        Path path = new Path("/user/" + userName.get());
-
-        return path.makeQualified(getUri(), null);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setWorkingDirectory(Path dir) {
-        Path fixedDir = fixRelativePart(dir);
-
-        String res = fixedDir.toUri().getPath();
-
-        if (!DFSUtil.isValidName(res))
-            throw new IllegalArgumentException("Invalid DFS directory name " + res);
-
-        workingDir.set(fixedDir);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Path getWorkingDirectory() {
-        return workingDir.get();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 f3f51d4..d90bc28 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,8 +19,6 @@ package org.apache.ignite.internal.processors.hadoop.fs;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.hdfs.protocol.*;
-import org.apache.ignite.hadoop.fs.v1.*;
 
 /**
  * Utilities for configuring file systems to support the separate working directory per each thread.
@@ -30,19 +28,6 @@ public class HadoopFileSystemsUtils {
     public static final String LOC_FS_WORK_DIR_PROP = "fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".workDir";
 
     /**
-     * Set user name and default working directory for current thread if it's supported by file system.
-     *
-     * @param fs File system.
-     * @param userName User name.
-     */
-    public static void setUser(FileSystem fs, String userName) {
-        if (fs instanceof IgniteHadoopFileSystem)
-            ((IgniteHadoopFileSystem)fs).setUser(userName);
-        else if (fs instanceof HadoopDistributedFileSystem)
-            ((HadoopDistributedFileSystem)fs).setUser(userName);
-    }
-
-    /**
      * Setup wrappers of filesystems to support the separate working directory.
      *
      * @param cfg Config for setup.
@@ -51,7 +36,5 @@ public class HadoopFileSystemsUtils {
         cfg.set("fs." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl", HadoopLocalFileSystemV1.class.getName());
         cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
                 HadoopLocalFileSystemV2.class.getName());
-
-        cfg.set("fs." + HdfsConstants.HDFS_URI_SCHEME + ".impl", HadoopDistributedFileSystem.class.getName());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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
new file mode 100644
index 0000000..71b38c4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -0,0 +1,204 @@
+/*
+ * 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.ignite.*;
+import org.apache.ignite.internal.util.future.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.locks.*;
+
+/**
+ * Maps values by keys.
+ * Values are created lazily using {@link ValueFactory}.
+ *
+ * Despite of the name, does not depend on any Hadoop classes.
+ */
+public class HadoopLazyConcurrentMap<K, V extends Closeable> {
+    /** The map storing the actual values. */
+    private final ConcurrentMap<K, ValueWrapper> map = new ConcurrentHashMap8<>();
+
+    /** The factory passed in by the client. Will be used for lazy value creation. */
+    private final ValueFactory<K, V> factory;
+
+    /** Lock used to close the objects. */
+    private final ReadWriteLock closeLock = new ReentrantReadWriteLock();
+
+    /** Flag indicating that this map is closed and cleared. */
+    private boolean closed;
+
+    /**
+     * Constructor.
+     * @param factory the factory to create new values lazily.
+     */
+    public HadoopLazyConcurrentMap(ValueFactory<K, V> factory) {
+        this.factory = factory;
+    }
+
+    /**
+     * Gets cached or creates a new value of V.
+     * Never returns null.
+     * @param k the key to associate the value with.
+     * @return the cached or newly created value, never null.
+     * @throws IgniteException on error
+     */
+    public V getOrCreate(K k) {
+        ValueWrapper w = map.get(k);
+
+        if (w == null) {
+            closeLock.readLock().lock();
+
+            try {
+                if (closed)
+                    throw new IllegalStateException("Failed to create value for key [" + k
+                        + "]: the map is already closed.");
+
+                final ValueWrapper wNew = new ValueWrapper(k);
+
+                w = map.putIfAbsent(k, wNew);
+
+                if (w == null) {
+                    wNew.init();
+
+                    w = wNew;
+                }
+            }
+            finally {
+                closeLock.readLock().unlock();
+            }
+        }
+
+        try {
+            V v = w.getValue();
+
+            assert v != null;
+
+            return v;
+        }
+        catch (IgniteCheckedException ie) {
+            throw new IgniteException(ie);
+        }
+    }
+
+    /**
+     * Clears the map and closes all the values.
+     */
+    public void close() throws IgniteCheckedException {
+        closeLock.writeLock().lock();
+
+        try {
+            closed = true;
+
+            Exception err = null;
+
+            Set<K> keySet = map.keySet();
+
+            for (K key : keySet) {
+                V v = null;
+
+                try {
+                    v = map.get(key).getValue();
+                }
+                catch (IgniteCheckedException ignore) {
+                    // No-op.
+                }
+
+                if (v != null) {
+                    try {
+                        v.close();
+                    }
+                    catch (Exception err0) {
+                        if (err == null)
+                            err = err0;
+                    }
+                }
+            }
+
+            map.clear();
+
+            if (err != null)
+                throw new IgniteCheckedException(err);
+        }
+        finally {
+            closeLock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * Helper class that drives the lazy value creation.
+     */
+    private class ValueWrapper {
+        /** Future. */
+        private final GridFutureAdapter<V> fut = new GridFutureAdapter<>();
+
+        /** the key */
+        private final K key;
+
+        /**
+         * Creates new wrapper.
+         */
+        private ValueWrapper(K key) {
+            this.key = key;
+        }
+
+        /**
+         * Initializes the value using the factory.
+         */
+        private void init() {
+            try {
+                final V v0 = factory.createValue(key);
+
+                if (v0 == null)
+                    throw new IgniteException("Failed to create non-null value. [key=" + key + ']');
+
+                fut.onDone(v0);
+            }
+            catch (Throwable e) {
+                fut.onDone(e);
+            }
+        }
+
+        /**
+         * Gets the available value or blocks until the value is initialized.
+         * @return the value, never null.
+         * @throws IgniteCheckedException on error.
+         */
+        V getValue() throws IgniteCheckedException {
+            return fut.get();
+        }
+    }
+
+    /**
+     * Interface representing the factory that creates map values.
+     * @param <K> the type of the key.
+     * @param <V> the type of the value.
+     */
+    public interface ValueFactory <K, V> {
+        /**
+         * Creates the new value. Should never return null.
+         *
+         * @param key the key to create value for
+         * @return the value.
+         * @throws IgniteException on failure.
+         */
+        public V createValue(K key);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
index 2f19226..b9c5113 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsEx.java
@@ -85,4 +85,10 @@ public interface HadoopIgfsEx extends HadoopIgfs {
      * @throws IOException If failed.
      */
     public void flush(HadoopIgfsStreamDelegate delegate) throws IOException;
+
+    /**
+     * The user this Igfs instance works on behalf of.
+     * @return the user name.
+     */
+    public String user();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
index 44e531e..47ba0e8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsInProc.java
@@ -23,6 +23,7 @@ import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -46,25 +47,35 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     /** Logger. */
     private final Log log;
 
+    /** The user this Igfs works on behalf of. */
+    private final String user;
+
     /**
      * Constructor.
      *
      * @param igfs Target IGFS.
      * @param log Log.
      */
-    public HadoopIgfsInProc(IgfsEx igfs, Log log) {
+    public HadoopIgfsInProc(IgfsEx igfs, Log log, String userName) throws IgniteCheckedException {
+        this.user = IgfsUtils.fixUserName(userName);
+
         this.igfs = igfs;
+
         this.log = log;
 
         bufSize = igfs.configuration().getBlockSize() * 2;
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsHandshakeResponse handshake(String logDir) {
-        igfs.clientLogDirectory(logDir);
+    @Override public IgfsHandshakeResponse handshake(final String logDir) {
+        return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsHandshakeResponse>() {
+            @Override public IgfsHandshakeResponse apply() {
+                igfs.clientLogDirectory(logDir);
 
-        return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
-            igfs.globalSampling());
+                return new IgfsHandshakeResponse(igfs.name(), igfs.proxyPaths(), igfs.groupBlockSize(),
+                    igfs.globalSampling());
+                }
+         });
     }
 
     /** {@inheritDoc} */
@@ -82,9 +93,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile info(IgfsPath path) throws IgniteCheckedException {
+    @Override public IgfsFile info(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.info(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
+                @Override public IgfsFile apply() {
+                    return igfs.info(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -95,9 +110,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsFile update(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Override public IgfsFile update(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
         try {
-            return igfs.update(path, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsFile>() {
+                @Override public IgfsFile apply() {
+                    return igfs.update(path, props);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -108,9 +127,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean setTimes(IgfsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+    @Override public Boolean setTimes(final IgfsPath path, final long accessTime, final long modificationTime) throws IgniteCheckedException {
         try {
-            igfs.setTimes(path, accessTime, modificationTime);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.setTimes(path, accessTime, modificationTime);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -124,9 +149,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean rename(IgfsPath src, IgfsPath dest) throws IgniteCheckedException {
+    @Override public Boolean rename(final IgfsPath src, final IgfsPath dest) throws IgniteCheckedException {
         try {
-            igfs.rename(src, dest);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.rename(src, dest);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -139,9 +170,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean delete(IgfsPath path, boolean recursive) throws IgniteCheckedException {
+    @Override public Boolean delete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
         try {
-            return igfs.delete(path, recursive);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Boolean>() {
+                @Override public Boolean apply() {
+                    return igfs.delete(path, recursive);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -154,18 +189,32 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     /** {@inheritDoc} */
     @Override public IgfsStatus fsStatus() throws IgniteCheckedException {
         try {
-            return igfs.globalSpace();
+            return IgfsUserContext.doAs(user, new Callable<IgfsStatus>() {
+                @Override public IgfsStatus call() throws IgniteCheckedException {
+                    return igfs.globalSpace();
+                }
+            });
         }
         catch (IllegalStateException e) {
             throw new HadoopIgfsCommunicationException("Failed to get file system status because Grid is " +
                 "stopping.");
         }
+        catch (IgniteCheckedException | RuntimeException | Error e) {
+            throw e;
+        }
+        catch (Exception e) {
+            throw new AssertionError("Must never go there.");
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsPath> listPaths(IgfsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgfsPath> listPaths(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.listPaths(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsPath>>() {
+                @Override public Collection<IgfsPath> apply() {
+                    return igfs.listPaths(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -176,9 +225,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsFile> listFiles(IgfsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgfsFile> listFiles(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.listFiles(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsFile>>() {
+                @Override public Collection<IgfsFile> apply() {
+                    return igfs.listFiles(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -189,9 +242,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean mkdirs(IgfsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Override public Boolean mkdirs(final IgfsPath path, final Map<String, String> props) throws IgniteCheckedException {
         try {
-            igfs.mkdirs(path, props);
+            IgfsUserContext.doAs(user, new IgniteOutClosure<Void>() {
+                @Override public Void apply() {
+                    igfs.mkdirs(path, props);
+
+                    return null;
+                }
+            });
 
             return true;
         }
@@ -205,9 +264,13 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgfsPathSummary contentSummary(IgfsPath path) throws IgniteCheckedException {
+    @Override public IgfsPathSummary contentSummary(final IgfsPath path) throws IgniteCheckedException {
         try {
-            return igfs.summary(path);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<IgfsPathSummary>() {
+                @Override public IgfsPathSummary apply() {
+                    return igfs.summary(path);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -219,10 +282,14 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgfsBlockLocation> affinity(IgfsPath path, long start, long len)
+    @Override public Collection<IgfsBlockLocation> affinity(final IgfsPath path, final long start, final long len)
         throws IgniteCheckedException {
         try {
-            return igfs.affinity(path, start, len);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<Collection<IgfsBlockLocation>>() {
+                @Override public Collection<IgfsBlockLocation> apply() {
+                    return igfs.affinity(path, start, len);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -233,11 +300,15 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path) throws IgniteCheckedException {
         try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsInputStreamAdapter stream = igfs.open(path, bufSize);
 
-            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length());
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -248,12 +319,16 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate open(IgfsPath path, int seqReadsBeforePrefetch)
+    @Override public HadoopIgfsStreamDelegate open(final IgfsPath path, final int seqReadsBeforePrefetch)
         throws IgniteCheckedException {
         try {
-            IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsInputStreamAdapter stream = igfs.open(path, bufSize, seqReadsBeforePrefetch);
 
-            return new HadoopIgfsStreamDelegate(this, stream, stream.fileInfo().length());
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream, stream.fileInfo().length());
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -264,13 +339,17 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate create(IgfsPath path, boolean overwrite, boolean colocate,
-        int replication, long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate create(final IgfsPath path, final boolean overwrite, final boolean colocate,
+        final int replication, final long blockSize, final @Nullable Map<String, String> props) throws IgniteCheckedException {
         try {
-            IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
-                colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsOutputStream stream = igfs.create(path, bufSize, overwrite,
+                        colocate ? igfs.nextAffinityKey() : null, replication, blockSize, props);
 
-            return new HadoopIgfsStreamDelegate(this, stream);
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -281,12 +360,16 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopIgfsStreamDelegate append(IgfsPath path, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public HadoopIgfsStreamDelegate append(final IgfsPath path, final boolean create,
+        final @Nullable Map<String, String> props) throws IgniteCheckedException {
         try {
-            IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
+            return IgfsUserContext.doAs(user, new IgniteOutClosure<HadoopIgfsStreamDelegate>() {
+                @Override public HadoopIgfsStreamDelegate apply() {
+                    IgfsOutputStream stream = igfs.append(path, bufSize, create, props);
 
-            return new HadoopIgfsStreamDelegate(this, stream);
+                    return new HadoopIgfsStreamDelegate(HadoopIgfsInProc.this, stream);
+                }
+            });
         }
         catch (IgniteException e) {
             throw new IgniteCheckedException(e);
@@ -407,4 +490,9 @@ public class HadoopIgfsInProc implements HadoopIgfsEx {
         if (lsnr0 != null && log.isDebugEnabled())
             log.debug("Removed stream event listener [delegate=" + delegate + ']');
     }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return user;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
index 0264e7b..3561e95 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsIpcIo.java
@@ -41,7 +41,7 @@ import java.util.concurrent.locks.*;
 @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
 public class HadoopIgfsIpcIo implements HadoopIgfsIo {
     /** Logger. */
-    private Log log;
+    private final Log log;
 
     /** Request futures map. */
     private ConcurrentMap<Long, HadoopIgfsFuture> reqMap =


[09/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
index cb32b13..f2de8ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCrossCacheTxStoreSelfTest.java
@@ -101,19 +101,28 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         grid(0).cache("cacheA").removeAll();
         grid(0).cache("cacheB").removeAll();
         grid(0).cache("cacheC").removeAll();
+
+        for (CacheStore store : firstStores.values())
+            ((TestStore)store).clear();
+
+        for (CacheStore store : secondStores.values())
+            ((TestStore)store).clear();
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testWriteThrough() throws Exception {
+    public void testSameStore() throws Exception {
         IgniteEx grid = grid(0);
 
         TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
 
         assertNotNull(firstStore);
+        assertNotNull(secondStore);
 
-        Collection<String> evts = firstStore.events();
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
 
         try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
@@ -138,82 +147,122 @@ public class IgniteCrossCacheTxStoreSelfTest extends GridCommonAbstractTest {
         }
 
         assertEqualsCollections(F.asList(
-                "writeAll cacheA 2",
-                "writeAll cacheB 2",
-                "deleteAll cacheA 2",
-                "deleteAll cacheB 2",
-                "write cacheA",
-                "delete cacheA",
-                "write cacheB",
-                "sessionEnd true"
-            ),
-            evts);
+            "writeAll cacheA 2",
+            "writeAll cacheB 2",
+            "deleteAll cacheA 2",
+            "deleteAll cacheB 2",
+            "write cacheA",
+            "delete cacheA",
+            "write cacheB",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches1() throws Exception {
+    public void testDifferentStores() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
+
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
+
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
+
+        try (Transaction tx = grid.transactions().txStart()) {
             IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
             IgniteCache<Object, Object> cacheC = grid.cache("cacheC");
 
-            cacheA.put("1", "2");
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheC.put("1", "1");
+            cacheC.put("2", "2");
 
-            cacheC.put("1", "2");
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheC.remove("3");
+            cacheC.remove("4");
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
+            cacheA.put("5", "5");
+            cacheA.remove("6");
+
+            cacheC.put("7", "7");
+
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheC 2",
+            "deleteAll cacheC 2",
+            "write cacheC",
+            "sessionEnd true"
+        ),
+        secondStoreEvts);
     }
 
     /**
      * @throws Exception If failed.
      */
-    public void testIncompatibleCaches2() throws Exception {
+    public void testNonPersistentCache() throws Exception {
         IgniteEx grid = grid(0);
 
-        try (Transaction ignored = grid.transactions().txStart()) {
-            IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
-            IgniteCache<Object, Object> cacheC = grid.cache("cacheD");
+        TestStore firstStore = (TestStore)firstStores.get(grid.name());
+        TestStore secondStore = (TestStore)secondStores.get(grid.name());
 
-            cacheA.put("1", "2");
+        assertNotNull(firstStore);
+        assertNotNull(secondStore);
 
-            cacheC.put("1", "2");
+        Collection<String> firstStoreEvts = firstStore.events();
+        Collection<String> secondStoreEvts = secondStore.events();
 
-            fail("Must not allow to enlist caches with different stores to one transaction");
-        }
-        catch (CacheException e) {
-            assertTrue(e.getMessage().contains("Failed to enlist new cache to existing transaction"));
-        }
-    }
-
-    /**
-     * @param col1 Collection 1.
-     * @param col2 Collection 2.
-     */
-    private static void assertEqualsCollections(Collection<?> col1, Collection<?> col2) {
-        if (col1.size() != col2.size())
-            fail("Collections are not equal:\nExpected:\t" + col1 + "\nActual:\t" + col2);
+        try (Transaction tx = grid.transactions().txStart()) {
+            IgniteCache<Object, Object> cacheA = grid.cache("cacheA");
+            IgniteCache<Object, Object> cacheD = grid.cache("cacheD");
 
-        Iterator<?> it1 = col1.iterator();
-        Iterator<?> it2 = col2.iterator();
+            cacheA.put("1", "1");
+            cacheA.put("2", "2");
+            cacheD.put("1", "1");
+            cacheD.put("2", "2");
 
-        int idx = 0;
+            cacheA.remove("3");
+            cacheA.remove("4");
+            cacheD.remove("3");
+            cacheD.remove("4");
 
-        while (it1.hasNext()) {
-            Object item1 = it1.next();
-            Object item2 = it2.next();
+            cacheA.put("5", "5");
+            cacheA.remove("6");
 
-            if (!F.eq(item1, item2))
-                fail("Collections are not equal (position " + idx + "):\nExpected: " + col1 + "\nActual:   " + col2);
+            cacheD.put("7", "7");
 
-            idx++;
+            tx.commit();
         }
+
+        assertEqualsCollections(F.asList(
+            "writeAll cacheA 2",
+            "deleteAll cacheA 2",
+            "write cacheA",
+            "delete cacheA",
+            "sessionEnd true"
+        ),
+        firstStoreEvts);
+
+        assertEquals(0, secondStoreEvts.size());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
index 0d4d607..24f72a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheClientOnlySelfTest.java
@@ -17,20 +17,72 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
+import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
 /**
  * Client only test.
  */
-public class GridCacheClientOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
+@SuppressWarnings("RedundantMethodOverride")
+public abstract class GridCacheClientOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected NearCacheConfiguration nearConfiguration() {
         return null;
     }
 
-    /** {@inheritDoc} */
-    @Override protected boolean clientOnly() {
-        return true;
+    /** */
+    public static class CaseReplicatedAtomic extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseReplicatedTransactional extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedAtomic extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedTransactional extends GridCacheClientOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
new file mode 100644
index 0000000..09c643b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtClientRemoveFailureTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.dht;
+
+/**
+ *
+ */
+public class GridCacheDhtClientRemoveFailureTest extends GridCacheDhtRemoveFailureTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index aff5512..7adeba8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -77,11 +77,18 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
         // Set eviction queue size explicitly.
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
         cacheCfg.setEvictMaxOverflowRatio(0);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
 
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10));
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
 
         cacheCfg.setNearConfiguration(nearCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
index 97e8657..8eada9c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
@@ -76,14 +76,21 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
         cacheCfg.setBackups(1);
 
         NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-        nearCfg.setNearEvictionPolicy(new FifoEvictionPolicy(10000));
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(10000);
+
+        nearCfg.setNearEvictionPolicy(nearPlc);
         cacheCfg.setNearConfiguration(nearCfg);
 
         // Set eviction queue size explicitly.
         cacheCfg.setEvictMaxOverflowRatio(0);
         cacheCfg.setEvictSynchronizedKeyBufferSize(1);
-        cacheCfg.setEvictionPolicy(new FifoEvictionPolicy(10000));
 
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(10000);
+
+        cacheCfg.setEvictionPolicy(plc);
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
deleted file mode 100644
index 55b1f92..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheExColocatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.dht;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on colocated cache.
- */
-public class GridCacheExColocatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 5983c1b..9e54673 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -33,6 +32,10 @@ import org.apache.ignite.testframework.junits.common.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
 /**
  * Tests explicit lock.
  */
@@ -46,6 +49,9 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
     /** */
     private volatile boolean run = true;
 
+    /** */
+    private boolean client;
+
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         stopAllGrids();
@@ -66,16 +72,22 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(CACHE_NAME);
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
         ccfg.setBackups(2);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setCacheMode(PARTITIONED);
         ccfg.setStartSize(100000);
-        ccfg.setEvictionPolicy(new LruEvictionPolicy(100000));
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(100000);
+
+        ccfg.setEvictionPolicy(plc);
         ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);
 
+        c.setClientMode(client);
+
         return c;
     }
 
@@ -83,33 +95,50 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testExplicitLockOneKey() throws Exception {
-        checkExplicitLock(1);
+        checkExplicitLock(1, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testExplicitLockManyKeys() throws Exception {
-        checkExplicitLock(4);
+        checkExplicitLock(4, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitLockManyKeysWithClient() throws Exception {
+        checkExplicitLock(4, true);
     }
 
     /**
+     * @param keys Number of keys.
+     * @param testClient If {@code true} uses one client node.
      * @throws Exception If failed.
      */
-    public void checkExplicitLock(int keys) throws Exception {
+    public void checkExplicitLock(int keys, boolean testClient) throws Exception {
         Collection<Thread> threads = new ArrayList<>();
 
         try {
             // Start grid 1.
             IgniteEx grid1 = startGrid(1);
 
+            assertFalse(grid1.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid1.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
 
+            client = testClient; // If test client start on node in client mode.
+
             // Start grid 2.
             IgniteEx grid2 = startGrid(2);
 
+            assertEquals((Object)testClient, grid2.configuration().isClientMode());
+
+            client = false;
+
             threads.add(runCacheOperations(grid2.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -117,6 +146,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 3.
             IgniteEx grid3 = startGrid(3);
 
+            assertFalse(grid3.configuration().isClientMode());
+
+            if (testClient)
+                log.info("Started client node: " + grid3.name());
+
             threads.add(runCacheOperations(grid3.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -124,6 +158,8 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 4.
             IgniteEx grid4 = startGrid(4);
 
+            assertFalse(grid4.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid4.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -158,6 +194,7 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Cache.
+     * @param keys Number of keys.
      * @return Running thread.
      */
     @SuppressWarnings("TypeMayBeWeakened")

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
new file mode 100644
index 0000000..64414a4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientInvalidPartitionHandlingSelfTest.java
@@ -0,0 +1,29 @@
+/*
+ * 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.dht.atomic;
+
+/**
+ *
+ */
+public class GridCacheAtomicClientInvalidPartitionHandlingSelfTest
+    extends GridCacheAtomicInvalidPartitionHandlingSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
new file mode 100644
index 0000000..2edb125
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicClientRemoveFailureTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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.dht.atomic;
+
+/**
+ *
+ */
+public class GridCacheAtomicClientRemoveFailureTest extends GridCacheAtomicRemoveFailureTest {
+    /** {@inheritDoc} */
+    @Override protected boolean testClientNode() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index a68423b..054a110 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -68,16 +68,15 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheConfiguration());
 
         cfg.setCommunicationSpi(new DelayCommunicationSpi());
 
+        if (testClientNode() && getTestGridName(0).equals(gridName))
+            cfg.setClientMode(true);
+
         return cfg;
     }
 
@@ -109,6 +108,13 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     }
 
     /**
+     * @return {@code True} if test updates from client node.
+     */
+    protected boolean testClientNode() {
+        return false;
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testClockFullSync() throws Exception {
@@ -167,6 +173,8 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
         awaitPartitionMapExchange();
 
         try {
+            assertEquals(testClientNode(), (boolean)grid(0).configuration().isClientMode());
+
             final IgniteCache<Object, Object> cache = grid(0).cache(null);
 
             final int range = 100_000;
@@ -321,7 +329,10 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                                     assertEquals("Failed to check value for key [key=" + k + ", node=" +
                                         locNode.id() + ", primary=" + primary + ", recNodeId=" + nodeId + ']',
                                         val, CU.value(entry.rawGetOrUnmarshal(false), entry.context(), false));
-                                    assertEquals(ver, entry.version());
+
+                                    assertEquals("Failed to check version for key [key=" + k + ", node=" +
+                                        locNode.id() + ", primary=" + primary + ", recNodeId=" + nodeId + ']',
+                                        ver, entry.version());
                                 }
                             }
                             else

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
index fcc15e0..7cdf265 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -55,7 +55,8 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
 
         // Override node attributes in discovery spi.
         TcpDiscoverySpi spi = new TcpDiscoverySpi() {
-            @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
                 super.setNodeAttributes(attrs, ver);
 
                 // Set unique mac addresses for every group of three nodes.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
deleted file mode 100644
index 7b51d9e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearOnlySelfTest.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-
-/**
- * Test for atomic cache.
- */
-public class GridCacheAtomicNearOnlySelfTest extends GridCacheNearOnlySelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheAtomicityMode atomicityMode() {
-        return ATOMIC;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
deleted file mode 100644
index 11ec96a..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheExNearFullApiSelfTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.near;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on partitioned cache with near enabled.
- */
-public class GridCacheExNearFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return PARTITIONED;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected NearCacheConfiguration nearConfiguration() {
-        return new NearCacheConfiguration();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 337a921..e78b782 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
@@ -83,6 +83,8 @@ public class GridCacheNearOnlyMultiNodeFullApiSelfTest extends GridCachePartitio
             info("Use grid '" + gridName + "' as near-only.");
 
             cfg.setClientMode(true);
+
+            cfg.setCacheConfiguration();
         }
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
index da7d3ce..f3a62f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlySelfTest.java
@@ -21,15 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
 /**
  * Near only self test.
  */
-public class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean clientOnly() {
-        return false;
-    }
-
+@SuppressWarnings("RedundantMethodOverride")
+public abstract class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfTest {
     /**
      * @throws Exception If failed.
      */
@@ -59,4 +58,56 @@ public class GridCacheNearOnlySelfTest extends GridCacheClientModesAbstractSelfT
             assertEquals(i * i, nearOnlyCache.get(i));
         }
     }
+
+    /** */
+    public static class CaseReplicatedAtomic extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseReplicatedTransactional extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedAtomic extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CasePartitionedTransactional extends GridCacheNearOnlySelfTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return TRANSACTIONAL;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index 2893678..b6bc56e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -70,6 +70,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(IP_FINDER);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index c79c32a..6b7d1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 
 import java.util.*;
 
@@ -38,6 +39,7 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         cfg.setClientMode(true);
 
         return cfg;
@@ -63,7 +65,8 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteClosure<Throwable, Throwable> errorHandler() {
         return new IgniteClosure<Throwable, Throwable>() {
             @Override public Throwable apply(Throwable e) {
-                if (e instanceof IgniteException || e instanceof IgniteCheckedException || X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                if (e instanceof IgniteException || e instanceof IgniteCheckedException ||
+                    X.hasCause(e, ClusterTopologyCheckedException.class)) {
                     info("Discarding exception: " + e);
 
                     return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
index 28ad7a8..c0d36e2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEvictionSelfTest.java
@@ -78,8 +78,15 @@ public class GridCachePartitionedEvictionSelfTest extends GridCacheAbstractSelfT
 
         cc.setCacheMode(PARTITIONED);
         cc.setWriteSynchronizationMode(FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
-        cc.getNearConfiguration().setNearEvictionPolicy(new FifoEvictionPolicy(EVICT_CACHE_SIZE));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.setEvictionPolicy(plc);
+
+        FifoEvictionPolicy nearPlc = new FifoEvictionPolicy();
+        nearPlc.setMaxSize(EVICT_CACHE_SIZE);
+        cc.getNearConfiguration().setNearEvictionPolicy(nearPlc);
+
         cc.setSwapEnabled(false);
 
         // We set 1 backup explicitly.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
index 7e16ac7..da5d1bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
@@ -17,12 +17,15 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.near;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 
+import javax.cache.*;
+
 import static org.apache.ignite.cache.CacheMode.*;
 
 /**
@@ -72,4 +75,33 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
         for (int i = 0 ; i < aff.partitions(); i++)
             String.valueOf(cache.entrySet(i));
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUpdate() throws Exception {
+        if (gridCount() > 1) {
+            IgniteCache<Object, Object> cache = grid(0).cache(null);
+
+            Integer key = nearKey(cache);
+
+            primaryCache(key, null).put(key, 1);
+
+            assertEquals(1, cache.get(key));
+
+            primaryCache(key, null).put(key, 2);
+
+            if (cache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null)
+                assertEquals(2, cache.localPeek(key));
+
+            assertEquals(2, cache.get(key));
+
+            int cnt = 0;
+
+            for (Cache.Entry e : cache)
+                cnt++;
+
+            assertEquals(1, cnt);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
index 8131d3e..feb94b2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiThreadedPutGetSelfTest.java
@@ -66,7 +66,11 @@ public class GridCachePartitionedMultiThreadedPutGetSelfTest extends GridCommonA
         cc.setCacheMode(PARTITIONED);
         cc.setBackups(1);
         cc.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
-        cc.setEvictionPolicy(new FifoEvictionPolicy<>(1000));
+
+        FifoEvictionPolicy plc = new FifoEvictionPolicy();
+        plc.setMaxMemorySize(1000);
+
+        cc.setEvictionPolicy(plc);
         cc.setSwapEnabled(false);
         cc.setAtomicityMode(TRANSACTIONAL);
         cc.setEvictSynchronized(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
index 4392365..61c3563 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest.java
@@ -65,7 +65,7 @@ public class GridCachePartitionedOffHeapTieredMultiNodeFullApiSelfTest extends G
         cache.put(key, 5); // Put from near to add reader on primary.
 
         assertEquals(5, primaryCache.localPeek(key, CachePeekMode.ONHEAP).intValue());
-        assertEquals(5, primaryCache.localPeek(key, CachePeekMode.OFFHEAP).intValue());
+        assertNull(primaryCache.localPeek(key, CachePeekMode.OFFHEAP));
         assertEquals(5, cache.get(key).intValue());
         assertEquals(5, map.get(key));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
index 80651bf..3bab050 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
@@ -35,9 +36,12 @@ public class GridCacheRendezvousAffinityClientSelfTest extends GridCommonAbstrac
     /** Client node. */
     private boolean client;
 
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
         ccfg.setCacheMode(CacheMode.PARTITIONED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
deleted file mode 100644
index 32ee784..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheExReplicatedFullApiSelfTest.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Tests private cache interface on replicated cache.
- */
-public class GridCacheExReplicatedFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedClientOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedClientOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedClientOnlySelfTest.java
deleted file mode 100644
index 51ad1c4..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedClientOnlySelfTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.distributed.dht.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Client only test for REPLICATED cache.
- */
-public class GridCacheReplicatedClientOnlySelfTest extends GridCacheClientOnlySelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setAffinity(null);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNearOnlySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNearOnlySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNearOnlySelfTest.java
deleted file mode 100644
index 21b4fe7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedNearOnlySelfTest.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.replicated;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.cache.distributed.near.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Near only test for REPLICATED cache.
- */
-public class GridCacheReplicatedNearOnlySelfTest extends GridCacheNearOnlySelfTest {
-    /** {@inheritDoc} */
-    @Override protected CacheConfiguration cacheConfiguration(String gridName) throws Exception {
-        CacheConfiguration cfg = super.cacheConfiguration(gridName);
-
-        cfg.setAffinity(null);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected CacheMode cacheMode() {
-        return REPLICATED;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
index 9e6fc37..1a6146e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
@@ -58,7 +58,6 @@ public class GridCacheSyncReplicatedPreloadSelfTest extends GridCommonAbstractTe
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(ipFinder);
-        disco.setDebugMode(DISCO_DEBUG_MODE);
 
         cfg.setDiscoverySpi(disco);
 


[20/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index d57786e..8b5eaec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -18,10 +18,10 @@
 package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
@@ -40,6 +40,7 @@ import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
 import javax.cache.expiry.*;
 import javax.cache.processor.*;
 import java.io.*;
@@ -480,17 +481,26 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
      */
     @SuppressWarnings({"CatchGenericClass"})
     protected void batchStoreCommit(Iterable<IgniteTxEntry> writeEntries) throws IgniteCheckedException {
-        CacheStoreManager store = store();
+        if (!storeEnabled() || internal())
+            return;
+
+        Collection<CacheStoreManager> stores = stores();
+
+        if (stores == null || stores.isEmpty())
+            return;
+
+        assert isWriteToStoreFromDhtValid(stores) : "isWriteToStoreFromDht can't be different within one transaction";
 
-        if (store != null && store.isWriteThrough() && storeEnabled() &&
-            !internal() && (near() || store.isWriteToStoreFromDht())) {
+        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+
+        if (near() || isWriteToStoreFromDht) {
             try {
                 if (writeEntries != null) {
                     Map<Object, IgniteBiTuple<Object, GridCacheVersion>> putMap = null;
                     List<Object> rmvCol = null;
                     CacheStoreManager writeStore = null;
 
-                    boolean skipNear = near() && store.isWriteToStoreFromDht();
+                    boolean skipNear = near() && isWriteToStoreFromDht;
 
                     for (IgniteTxEntry e : writeEntries) {
                         if ((skipNear && e.cached().isNear()) || e.skipStore())
@@ -524,11 +534,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                             }
 
                             // Batch-process puts if cache ID has changed.
-                            if (writeStore != null && writeStore != cacheCtx.store() && putMap != null && !putMap.isEmpty()) {
-                                writeStore.putAll(this, putMap);
+                            if (writeStore != null && writeStore != cacheCtx.store()) {
+                                if (putMap != null && !putMap.isEmpty()) {
+                                    writeStore.putAll(this, putMap);
 
-                                // Reset.
-                                putMap.clear();
+                                    // Reset.
+                                    putMap.clear();
+                                }
 
                                 writeStore = null;
                             }
@@ -544,12 +556,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 val = cacheCtx.toCacheObject(cacheCtx.unwrapTemporary(interceptorVal));
                             }
 
-                            if (putMap == null)
-                                putMap = new LinkedHashMap<>(writeMap().size(), 1.0f);
+                            if (writeStore == null)
+                                writeStore = cacheCtx.store();
 
-                            putMap.put(CU.value(key, cacheCtx, false), F.t(CU.value(val, cacheCtx, false), ver));
+                            if (writeStore.isWriteThrough()) {
+                                if (putMap == null)
+                                    putMap = new LinkedHashMap<>(writeMap().size(), 1.0f);
 
-                            writeStore = cacheCtx.store();
+                                putMap.put(CU.value(key, cacheCtx, false), F.t(CU.value(val, cacheCtx, false), ver));
+                            }
                         }
                         else if (op == DELETE) {
                             // Batch-process all puts if needed.
@@ -564,11 +579,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                 writeStore = null;
                             }
 
-                            if (writeStore != null && writeStore != cacheCtx.store() && rmvCol != null && !rmvCol.isEmpty()) {
-                                writeStore.removeAll(this, rmvCol);
+                            if (writeStore != null && writeStore != cacheCtx.store()) {
+                                if (rmvCol != null && !rmvCol.isEmpty()) {
+                                    writeStore.removeAll(this, rmvCol);
 
-                                // Reset.
-                                rmvCol.clear();
+                                    // Reset.
+                                    rmvCol.clear();
+                                }
 
                                 writeStore = null;
                             }
@@ -581,12 +598,15 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                                     continue;
                             }
 
-                            if (rmvCol == null)
-                                rmvCol = new ArrayList<>();
+                            if (writeStore == null)
+                                writeStore = cacheCtx.store();
 
-                            rmvCol.add(key.value(cacheCtx.cacheObjectContext(), false));
+                            if (writeStore.isWriteThrough()) {
+                                if (rmvCol == null)
+                                    rmvCol = new ArrayList<>();
 
-                            writeStore = cacheCtx.store();
+                                rmvCol.add(key.value(cacheCtx.cacheObjectContext(), false));
+                            }
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Ignoring NOOP entry for batch store commit: " + e);
@@ -610,7 +630,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 }
 
                 // Commit while locks are held.
-                store.sessionEnd(this, true);
+                sessionEnd(stores, true);
             }
             catch (IgniteCheckedException ex) {
                 commitError(ex);
@@ -635,6 +655,10 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                 throw new IgniteCheckedException("Failed to commit transaction to database: " + this, ex);
             }
+            finally {
+                if (isRollbackOnly())
+                    sessionEnd(stores, false);
+            }
         }
     }
 
@@ -972,24 +996,6 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                 cctx.tm().resetContext();
             }
         }
-        else {
-            CacheStoreManager store = store();
-
-            if (store != null && !internal()) {
-                try {
-                    store.sessionEnd(this, true);
-                }
-                catch (IgniteCheckedException e) {
-                    commitError(e);
-
-                    setRollbackOnly();
-
-                    cctx.tm().removeCommittedTx(this);
-
-                    throw e;
-                }
-            }
-        }
 
         // Do not unlock transaction entries if one-phase commit.
         if (!onePhaseCommit()) {
@@ -1078,11 +1084,18 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
                 cctx.tm().rollbackTx(this);
 
-                CacheStoreManager store = store();
+                if (!internal()) {
+                    Collection<CacheStoreManager> stores = stores();
+
+                    if (stores != null && !stores.isEmpty()) {
+                        assert isWriteToStoreFromDhtValid(stores) :
+                            "isWriteToStoreFromDht can't be different within one transaction";
 
-                if (store != null && (near() || store.isWriteToStoreFromDht())) {
-                    if (!internal())
-                        store.sessionEnd(this, false);
+                        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+
+                        if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
+                            sessionEnd(stores, false);
+                    }
                 }
             }
             catch (Error | IgniteCheckedException | RuntimeException e) {
@@ -1094,6 +1107,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param stores Store managers.
+     * @param commit Commit flag.
+     * @throws IgniteCheckedException In case of error.
+     */
+    private void sessionEnd(Collection<CacheStoreManager> stores, boolean commit) throws IgniteCheckedException {
+        Iterator<CacheStoreManager> it = stores.iterator();
+
+        while (it.hasNext()) {
+            CacheStoreManager store = it.next();
+
+            store.sessionEnd(this, commit, !it.hasNext());
+        }
+    }
+
+    /**
      * Checks if there is a cached or swapped value for
      * {@link #getAllAsync(GridCacheContext, Collection, GridCacheEntryEx, boolean, boolean, boolean, boolean)} method.
      *
@@ -2507,6 +2535,19 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param cacheCtx Cache context.
+     * @throws IgniteCheckedException If updates are not allowed.
+     */
+    private void checkUpdatesAllowed(GridCacheContext cacheCtx) throws IgniteCheckedException {
+        if (!cacheCtx.updatesAllowed()) {
+            throw new IgniteTxRollbackCheckedException(new CacheException(
+                "Updates are not allowed for transactional cache: " + cacheCtx.name() + ". Configure " +
+                "persistence store on client or use remote closure execution to start transactions " +
+                "from server nodes."));
+        }
+    }
+
+    /**
      * Internal method for all put and transform operations. Only one of {@code map}, {@code transformMap}
      * maps must be non-null.
      *
@@ -2533,6 +2574,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     ) {
         assert filter == null || invokeMap == null;
 
+        try {
+            checkUpdatesAllowed(cacheCtx);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture(e);
+        }
+
         cacheCtx.checkSecurity(SecurityPermission.CACHE_PUT);
 
         if (retval)
@@ -2753,6 +2801,13 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
         @Nullable GridCacheEntryEx cached,
         final boolean retval,
         @Nullable final CacheEntryPredicate[] filter) {
+        try {
+            checkUpdatesAllowed(cacheCtx);
+        }
+        catch (IgniteCheckedException e) {
+            return new GridFinishedFuture(e);
+        }
+
         cacheCtx.checkSecurity(SecurityPermission.CACHE_REMOVE);
 
         if (retval)
@@ -2983,7 +3038,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
 
         // Check if we can enlist new cache to transaction.
         if (!activeCacheIds.contains(cacheId)) {
-            if (!cctx.txCompatible(this, activeCacheIds, cacheCtx)) {
+            String err = cctx.verifyTxCompatibility(this, activeCacheIds, cacheCtx);
+
+            if (err != null) {
                 StringBuilder cacheNames = new StringBuilder();
 
                 int idx = 0;
@@ -2995,9 +3052,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
                         cacheNames.append(", ");
                 }
 
-                throw new IgniteCheckedException("Failed to enlist new cache to existing transaction " +
-                    "(cache configurations are not compatible) [" +
-                    "activeCaches=[" + cacheNames + "]" +
+                throw new IgniteCheckedException("Failed to enlist new cache to existing transaction (" +
+                    err +
+                    ") [activeCaches=[" + cacheNames + "]" +
                     ", cacheName=" + cacheCtx.name() +
                     ", cacheSystem=" + cacheCtx.systemTx() +
                     ", txSystem=" + system() + ']');
@@ -3284,6 +3341,23 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter
     }
 
     /**
+     * @param stores Store managers.
+     * @return If {@code isWriteToStoreFromDht} value same for all stores.
+     */
+    private boolean isWriteToStoreFromDhtValid(Collection<CacheStoreManager> stores) {
+        if (stores != null && !stores.isEmpty()) {
+            boolean exp = F.first(stores).isWriteToStoreFromDht();
+
+            for (CacheStoreManager store : stores) {
+                if (store.isWriteToStoreFromDht() != exp)
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
      * Post-lock closure alias.
      *
      * @param <T> Return type.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 4666cca..b6c77f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -1221,9 +1221,6 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                 collectPendingVersions(dhtTxLoc);
             }
 
-            // 3.1 Call dataStructures manager.
-            cctx.kernalContext().dataStructures().onTxCommitted(tx);
-
             // 4. Unlock write resources.
             unlockMultiple(tx, tx.writeEntries());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index a04692d..f8e5a60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.processors.cacheobject;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -32,8 +30,9 @@ import org.jetbrains.annotations.*;
 public interface IgniteCacheObjectProcessor extends GridProcessor {
     /**
      * @see GridComponent#onKernalStart()
+     * @throws IgniteCheckedException If failed.
      */
-    public void onCacheProcessorStarted();
+    public void onUtilityCacheStarted() throws IgniteCheckedException;
 
     /**
      * @param typeName Type name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index fe5a356..45fc121 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -208,7 +208,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public void onCacheProcessorStarted() {
+    @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
new file mode 100644
index 0000000..91768a6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java
@@ -0,0 +1,63 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public abstract class AbstractContinuousMessage implements DiscoveryCustomMessage {
+    /** Routine ID. */
+    protected final UUID routineId;
+
+    /** Custom message ID. */
+    private final IgniteUuid id = IgniteUuid.randomUuid();
+
+    /**
+     * @param id Id.
+     */
+    protected AbstractContinuousMessage(UUID id) {
+        routineId = id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteUuid id() {
+        return id;
+    }
+
+    /**
+     * @return Routine ID.
+     */
+    public UUID routineId() {
+        return routineId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean incrementMinorTopologyVersion() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessageType.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessageType.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessageType.java
index eb33613..1b79430 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessageType.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousMessageType.java
@@ -23,18 +23,6 @@ import org.jetbrains.annotations.*;
  * Continuous processor message types.
  */
 enum GridContinuousMessageType {
-    /** Consume start request. */
-    MSG_START_REQ,
-
-    /** Consume start acknowledgement. */
-    MSG_START_ACK,
-
-    /** Consume stop request. */
-    MSG_STOP_REQ,
-
-    /** Consume stop acknowledgement. */
-    MSG_STOP_ACK,
-
     /** Remote event notification. */
     MSG_EVT_NOTIFICATION,
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index 0d76ad4..38d970b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.managers.deployment.*;
+import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
@@ -58,50 +59,44 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /** Local infos. */
     private final ConcurrentMap<UUID, LocalRoutineInfo> locInfos = new ConcurrentHashMap8<>();
 
+    /** Local infos. */
+    private final ConcurrentMap<UUID, Map<UUID, LocalRoutineInfo>> clientInfos = new ConcurrentHashMap8<>();
+
     /** Remote infos. */
     private final ConcurrentMap<UUID, RemoteRoutineInfo> rmtInfos = new ConcurrentHashMap8<>();
 
     /** Start futures. */
     private final ConcurrentMap<UUID, StartFuture> startFuts = new ConcurrentHashMap8<>();
 
-    /** Start ack wait lists. */
-    private final ConcurrentMap<UUID, Collection<UUID>> waitForStartAck = new ConcurrentHashMap8<>();
-
     /** Stop futures. */
     private final ConcurrentMap<UUID, StopFuture> stopFuts = new ConcurrentHashMap8<>();
 
-    /** Stop ack wait lists. */
-    private final ConcurrentMap<UUID, Collection<UUID>> waitForStopAck = new ConcurrentHashMap8<>();
-
     /** Threads started by this processor. */
     private final Collection<IgniteThread> threads = new GridConcurrentHashSet<>();
 
-    /** Pending start requests. */
-    private final Map<UUID, Collection<GridContinuousMessage>> pending = new HashMap<>();
-
     /** */
     private final ConcurrentMap<IgniteUuid, SyncMessageAckFuture> syncMsgFuts = new ConcurrentHashMap8<>();
 
     /** Stopped IDs. */
     private final Collection<UUID> stopped = new HashSet<>();
 
-    /** Lock for pending requests. */
-    private final Lock pendingLock = new ReentrantLock();
-
     /** Lock for stop process. */
     private final Lock stopLock = new ReentrantLock();
 
+    /** Marshaller. */
+    private Marshaller marsh;
+
     /** Delay in milliseconds between retries. */
     private long retryDelay = 1000;
 
     /** Number of retries using to send messages. */
     private int retryCnt = 3;
 
-    /** Acknowledgement timeout. */
-    private long ackTimeout;
+    /** */
+    private final ReentrantReadWriteLock processorStopLock = new ReentrantReadWriteLock();
 
-    /** Marshaller. */
-    private Marshaller marsh;
+    /** */
+    private boolean processorStopped;
 
     /**
      * @param ctx Kernal context.
@@ -117,15 +112,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         retryDelay = ctx.config().getNetworkSendRetryDelay();
         retryCnt = ctx.config().getNetworkSendRetryCount();
-        ackTimeout = ctx.config().getNetworkTimeout();
-
-        if (ackTimeout < retryDelay * retryCnt) {
-            U.warn(log, "Acknowledgement timeout for continuous operations is less than message send " +
-                "retry delay multiplied by retries count (will increase timeout value) [ackTimeout=" +
-                ackTimeout + ", retryDelay=" + retryDelay + ", retryCnt=" + retryCnt + ']');
-
-            ackTimeout = retryDelay * retryCnt;
-        }
 
         marsh = ctx.config().getMarshaller();
 
@@ -133,114 +119,112 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             @SuppressWarnings({"fallthrough", "TooBroadScope"})
             @Override public void onEvent(Event evt) {
                 assert evt instanceof DiscoveryEvent;
+                assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
 
                 UUID nodeId = ((DiscoveryEvent)evt).eventNode().id();
 
-                Collection<GridContinuousMessage> reqs;
+                clientInfos.remove(nodeId);
 
-                pendingLock.lock();
+                // Unregister handlers created by left node.
+                for (Map.Entry<UUID, RemoteRoutineInfo> e : rmtInfos.entrySet()) {
+                    UUID routineId = e.getKey();
+                    RemoteRoutineInfo info = e.getValue();
 
-                try {
-                    // Remove pending requests to send to joined node
-                    // (if node is left or failed, they are dropped).
-                    reqs = pending.remove(nodeId);
-                }
-                finally {
-                    pendingLock.unlock();
+                    if (info.autoUnsubscribe && nodeId.equals(info.nodeId))
+                        unregisterRemote(routineId);
                 }
 
-                switch (evt.type()) {
-                    case EVT_NODE_JOINED:
-                        if (reqs != null) {
-                            UUID routineId = null;
+                for (Map.Entry<IgniteUuid, SyncMessageAckFuture> e : syncMsgFuts.entrySet()) {
+                    SyncMessageAckFuture fut = e.getValue();
 
-                            // Send pending requests.
-                            try {
-                                for (GridContinuousMessage req : reqs) {
-                                    routineId = req.routineId();
+                    if (fut.nodeId().equals(nodeId)) {
+                        SyncMessageAckFuture fut0 = syncMsgFuts.remove(e.getKey());
 
-                                    sendWithRetries(nodeId, req, null);
-                                }
-                            }
-                            catch (ClusterTopologyCheckedException ignored) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to send pending start request to node (is node alive?): " +
-                                        nodeId);
-                            }
-                            catch (IgniteCheckedException e) {
-                                U.error(log, "Failed to send pending start request to node: " + nodeId, e);
+                        if (fut0 != null) {
+                            ClusterTopologyCheckedException err = new ClusterTopologyCheckedException(
+                                "Node left grid while sending message to: " + nodeId);
 
-                                completeStartFuture(routineId);
-                            }
+                            fut0.onDone(err);
                         }
+                    }
+                }
+            }
+        }, EVT_NODE_LEFT, EVT_NODE_FAILED);
 
-                        break;
+        ctx.event().addLocalEventListener(new GridLocalEventListener() {
+            @Override public void onEvent(Event evt) {
+                for (Iterator<StartFuture> itr = startFuts.values().iterator(); itr.hasNext(); ) {
+                    StartFuture fut = itr.next();
 
-                    case EVT_NODE_LEFT:
-                    case EVT_NODE_FAILED:
-                        // Do not wait for start acknowledgements from left node.
-                        for (Map.Entry<UUID, Collection<UUID>> e : waitForStartAck.entrySet()) {
-                            Collection<UUID> nodeIds = e.getValue();
+                    itr.remove();
 
-                            for (Iterator<UUID> it = nodeIds.iterator(); it.hasNext();) {
-                                if (nodeId.equals(it.next())) {
-                                    it.remove();
+                    fut.onDone(new IgniteException("Topology segmented"));
+                }
 
-                                    break;
-                                }
-                            }
+                for (Iterator<StopFuture> itr = stopFuts.values().iterator(); itr.hasNext(); ) {
+                    StopFuture fut = itr.next();
 
-                            if (nodeIds.isEmpty())
-                                completeStartFuture(e.getKey());
-                        }
+                    itr.remove();
 
-                        // Do not wait for stop acknowledgements from left node.
-                        for (Map.Entry<UUID, Collection<UUID>> e : waitForStopAck.entrySet()) {
-                            Collection<UUID> nodeIds = e.getValue();
+                    fut.onDone(new IgniteException("Topology segmented"));
+                }
+            }
+        }, EVT_NODE_SEGMENTED);
 
-                            for (Iterator<UUID> it = nodeIds.iterator(); it.hasNext();) {
-                                if (nodeId.equals(it.next())) {
-                                    it.remove();
+        ctx.discovery().setCustomEventListener(StartRoutineDiscoveryMessage.class,
+            new CustomEventListener<StartRoutineDiscoveryMessage>() {
+                @Override public void onCustomEvent(ClusterNode snd, StartRoutineDiscoveryMessage msg) {
+                    if (!snd.id().equals(ctx.localNodeId()) && !ctx.isStopping())
+                        processStartRequest(snd, msg);
+                }
+            });
 
-                                    break;
-                                }
-                            }
+        ctx.discovery().setCustomEventListener(StartRoutineAckDiscoveryMessage.class,
+            new CustomEventListener<StartRoutineAckDiscoveryMessage>() {
+                @Override public void onCustomEvent(ClusterNode snd, StartRoutineAckDiscoveryMessage msg) {
+                    StartFuture fut = startFuts.remove(msg.routineId());
 
-                            if (nodeIds.isEmpty())
-                                completeStopFuture(e.getKey());
-                        }
+                    if (fut != null) {
+                        if (msg.errs().isEmpty())
+                            fut.onRemoteRegistered();
+                        else {
+                            IgniteCheckedException firstEx = F.first(msg.errs().values());
 
-                        // Unregister handlers created by left node.
-                        for (Map.Entry<UUID, RemoteRoutineInfo> e : rmtInfos.entrySet()) {
-                            UUID routineId = e.getKey();
-                            RemoteRoutineInfo info = e.getValue();
+                            fut.onDone(firstEx);
 
-                            if (info.autoUnsubscribe && nodeId.equals(info.nodeId))
-                                unregisterRemote(routineId);
+                            stopRoutine(msg.routineId());
                         }
+                    }
+                }
+            });
 
-                        for (Map.Entry<IgniteUuid, SyncMessageAckFuture> e : syncMsgFuts.entrySet()) {
-                            SyncMessageAckFuture fut = e.getValue();
+        ctx.discovery().setCustomEventListener(StopRoutineDiscoveryMessage.class,
+            new CustomEventListener<StopRoutineDiscoveryMessage>() {
+                @Override public void onCustomEvent(ClusterNode snd, StopRoutineDiscoveryMessage msg) {
+                    if (!snd.id().equals(ctx.localNodeId())) {
+                        UUID routineId = msg.routineId();
 
-                            if (fut.nodeId().equals(nodeId)) {
-                                SyncMessageAckFuture fut0 = syncMsgFuts.remove(e.getKey());
+                        unregisterRemote(routineId);
 
-                                if (fut0 != null) {
-                                    ClusterTopologyCheckedException err = new ClusterTopologyCheckedException(
-                                        "Node left grid while sending message to: " + nodeId);
+                        if (snd.isClient()) {
+                            Map<UUID, LocalRoutineInfo> infoMap = clientInfos.get(snd.id());
 
-                                    fut0.onDone(err);
-                                }
-                            }
+                            if (infoMap != null)
+                                infoMap.remove(msg.routineId());
                         }
+                    }
+                }
+            });
 
-                        break;
+        ctx.discovery().setCustomEventListener(StopRoutineAckDiscoveryMessage.class,
+            new CustomEventListener<StopRoutineAckDiscoveryMessage>() {
+                @Override public void onCustomEvent(ClusterNode snd, StopRoutineAckDiscoveryMessage msg) {
+                    StopFuture fut = stopFuts.remove(msg.routineId());
 
-                    default:
-                        assert false : "Unexpected event received: " + evt.shortDisplay();
+                    if (fut != null)
+                        fut.onDone();
                 }
-            }
-        }, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED);
+            });
 
         ctx.io().addMessageListener(TOPIC_CONTINUOUS, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object obj) {
@@ -258,26 +242,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 }
 
                 switch (msg.type()) {
-                    case MSG_START_REQ:
-                        processStartRequest(nodeId, msg);
-
-                        break;
-
-                    case MSG_START_ACK:
-                        processStartAck(nodeId, msg);
-
-                        break;
-
-                    case MSG_STOP_REQ:
-                        processStopRequest(nodeId, msg);
-
-                        break;
-
-                    case MSG_STOP_ACK:
-                        processStopAck(nodeId, msg);
-
-                        break;
-
                     case MSG_EVT_NOTIFICATION:
                         processNotification(nodeId, msg);
 
@@ -298,6 +262,41 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             log.debug("Continuous processor started.");
     }
 
+    /**
+     * @return {@code true} if lock successful, {@code false} if processor already stopped.
+     */
+    @SuppressWarnings("LockAcquiredButNotSafelyReleased")
+    public boolean lockStopping() {
+        processorStopLock.readLock().lock();
+
+        if (processorStopped) {
+            processorStopLock.readLock().unlock();
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     *
+     */
+    public void unlockStopping() {
+        processorStopLock.readLock().unlock();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onKernalStop(boolean cancel) {
+        processorStopLock.writeLock().lock();
+
+        try {
+            processorStopped = true;
+        }
+        finally {
+            processorStopLock.writeLock().unlock();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (ctx.config().isDaemon())
@@ -320,29 +319,19 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
         if (!nodeId.equals(ctx.localNodeId())) {
-            pendingLock.lock();
+            DiscoveryData data = new DiscoveryData(ctx.localNodeId(), clientInfos);
 
-            try {
-                // Create empty pending set.
-                pending.put(nodeId, new HashSet<GridContinuousMessage>());
-
-                DiscoveryData data = new DiscoveryData(ctx.localNodeId());
-
-                // Collect listeners information (will be sent to
-                // joining node during discovery process).
-                for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet()) {
-                    UUID routineId = e.getKey();
-                    LocalRoutineInfo info = e.getValue();
-
-                    data.addItem(new DiscoveryDataItem(routineId, info.prjPred,
-                        info.hnd, info.bufSize, info.interval));
-                }
+            // Collect listeners information (will be sent to
+            // joining node during discovery process).
+            for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet()) {
+                UUID routineId = e.getKey();
+                LocalRoutineInfo info = e.getValue();
 
-                return data;
-            }
-            finally {
-                pendingLock.unlock();
+                data.addItem(new DiscoveryDataItem(routineId, info.prjPred,
+                    info.hnd, info.bufSize, info.interval));
             }
+
+            return data;
         }
         else
             return null;
@@ -366,6 +355,18 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     }
                 }
             }
+
+            for (Map.Entry<UUID, Map<UUID, LocalRoutineInfo>> entry : data.clientInfos.entrySet()) {
+                Map<UUID, LocalRoutineInfo> map = clientInfos.get(entry.getKey());
+
+                if (map == null) {
+                    map = new HashMap<>();
+
+                    clientInfos.put(entry.getKey(), map);
+                }
+
+                map.putAll(entry.getValue());
+            }
         }
     }
 
@@ -447,14 +448,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     if (dep == null)
                         throw new IgniteDeploymentCheckedException("Failed to deploy projection predicate: " + prjPred);
 
-                    reqData.clsName = clsName;
-                    reqData.depInfo = new GridDeploymentInfoBean(dep);
+                    reqData.className(clsName);
+                    reqData.deploymentInfo(new GridDeploymentInfoBean(dep));
 
                     reqData.p2pMarshal(marsh);
                 }
 
                 // Handle peer deployment for other handler-specific objects.
-                reqData.hnd.p2pMarshal(ctx);
+                reqData.handler().p2pMarshal(ctx);
             }
         }
         catch (IgniteCheckedException e) {
@@ -486,103 +487,24 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             });
         }
 
-        Collection<? extends ClusterNode> nodes;
-        Collection<UUID> nodeIds;
-
-        pendingLock.lock();
-
-        try {
-            // Nodes that participate in routine (request will be sent to these nodes directly).
-            nodes = F.view(ctx.discovery().allNodes(), F.and(prjPred, F.remoteNodes(ctx.localNodeId())));
-
-            // Stop with exception if projection is empty.
-            if (nodes.isEmpty() && !locIncluded) {
-                return new GridFinishedFuture<>(
-                    new ClusterTopologyCheckedException("Failed to register remote continuous listener (projection is empty)."));
-            }
-
-            // IDs of nodes where request will be sent.
-            nodeIds = new GridConcurrentHashSet<>(F.viewReadOnly(nodes, F.node2id()));
-
-            // If there are currently joining nodes, add request to their pending lists.
-            // Node IDs set is updated to make sure that we wait for acknowledgement from
-            // these nodes.
-            for (Map.Entry<UUID, Collection<GridContinuousMessage>> e : pending.entrySet()) {
-                if (nodeIds.add(e.getKey()))
-                    e.getValue().add(new GridContinuousMessage(MSG_START_REQ, routineId, null, reqData, false));
-            }
-
-            // Register routine locally.
-            locInfos.put(routineId, new LocalRoutineInfo(prjPred, hnd, bufSize, interval));
-        }
-        finally {
-            pendingLock.unlock();
-        }
+        // Register routine locally.
+        locInfos.put(routineId, new LocalRoutineInfo(prjPred, hnd, bufSize, interval));
 
         StartFuture fut = new StartFuture(ctx, routineId);
 
-        if (!nodeIds.isEmpty()) {
-            // Wait for acknowledgements.
-            waitForStartAck.put(routineId, nodeIds);
-
-            startFuts.put(routineId, fut);
-
-            // Register acknowledge timeout (timeout object will be removed when
-            // future is completed).
-            fut.addTimeoutObject(new GridTimeoutObjectAdapter(ackTimeout) {
-                @Override public void onTimeout() {
-                    // Stop waiting for acknowledgements.
-                    Collection<UUID> ids = waitForStartAck.remove(routineId);
+        startFuts.put(routineId, fut);
 
-                    if (ids != null) {
-                        StartFuture f = startFuts.remove(routineId);
-
-                        assert f != null;
-
-                        // If there are still nodes without acknowledgements,
-                        // Stop routine with exception. Continue and complete
-                        // future otherwise.
-                        if (!ids.isEmpty()) {
-                            f.onDone(new IgniteCheckedException("Failed to get start acknowledgement from nodes (timeout " +
-                                "expired): " + ids + ". Will unregister all continuous listeners."));
-
-                            stopRoutine(routineId);
-                        }
-                        else
-                            f.onRemoteRegistered();
-                    }
-                }
-            });
+        try {
+            ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData));
         }
+        catch (IgniteException e) { // Marshaller exception may occurs if user pass unmarshallable filter.
+            startFuts.remove(routineId);
 
-        if (!nodes.isEmpty()) {
-            // Do not send projection predicate (nodes already filtered).
-            reqData.prjPred = null;
-            reqData.prjPredBytes = null;
-
-            // Send start requests.
-            try {
-                GridContinuousMessage req = new GridContinuousMessage(MSG_START_REQ, routineId, null, reqData, false);
+            locInfos.remove(routineId);
 
-                sendWithRetries(nodes, req, null);
-            }
-            catch (IgniteCheckedException e) {
-                startFuts.remove(routineId);
-                waitForStartAck.remove(routineId);
+            fut.onDone(e);
 
-                fut.onDone(e);
-
-                stopRoutine(routineId);
-
-                locIncluded = false;
-            }
-        }
-        else {
-            // There are no remote nodes, but we didn't throw topology exception.
-            assert locIncluded;
-
-            // Do not wait anything from remote nodes.
-            fut.onRemoteRegistered();
+            return fut;
         }
 
         // Register local handler if needed.
@@ -640,61 +562,10 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             // Unregister handler locally.
             unregisterHandler(routineId, routine.hnd, true);
 
-            pendingLock.lock();
-
-            try {
-                // Remove pending requests for this routine.
-                for (Collection<GridContinuousMessage> msgs : pending.values()) {
-                    Iterator<GridContinuousMessage> it = msgs.iterator();
-
-                    while (it.hasNext()) {
-                        if (it.next().routineId().equals(routineId))
-                            it.remove();
-                    }
-                }
-            }
-            finally {
-                pendingLock.unlock();
-            }
-
-            // Nodes where to send stop requests.
-            Collection<? extends ClusterNode> nodes = F.view(ctx.discovery().allNodes(),
-                F.and(routine.prjPred, F.remoteNodes(ctx.localNodeId())));
-
-            if (!nodes.isEmpty()) {
-                // Wait for acknowledgements.
-                waitForStopAck.put(routineId, new GridConcurrentHashSet<>(F.viewReadOnly(nodes, F.node2id())));
-
-                // Register acknowledge timeout (timeout object will be removed when
-                // future is completed).
-                fut.addTimeoutObject(new StopTimeoutObject(ackTimeout, routineId,
-                    new GridContinuousMessage(MSG_STOP_REQ, routineId, null, null, false)));
-
-                // Send stop requests.
-                try {
-                    for (ClusterNode node : nodes) {
-                        try {
-                            sendWithRetries(node.id(),
-                                new GridContinuousMessage(MSG_STOP_REQ, routineId, null, null, false),
-                                null);
-                        }
-                        catch (ClusterTopologyCheckedException ignored) {
-                            U.warn(log, "Failed to send stop request (node left topology): " + node.id());
-                        }
-                    }
-                }
-                catch (IgniteCheckedException e) {
-                    stopFuts.remove(routineId);
-                    waitForStopAck.remove(routineId);
-
-                    fut.onDone(e);
-                }
-            }
-            else {
-                stopFuts.remove(routineId);
+            ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
 
+            if (ctx.isStopping())
                 fut.onDone();
-            }
         }
 
         return fut;
@@ -721,13 +592,16 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         assert !nodeId.equals(ctx.localNodeId());
 
+        if (processorStopped)
+            return;
+
         RemoteRoutineInfo info = rmtInfos.get(routineId);
 
         if (info != null) {
             assert info.interval == 0 || !sync;
 
             if (sync) {
-                SyncMessageAckFuture fut = new SyncMessageAckFuture(ctx, nodeId);
+                SyncMessageAckFuture fut = new SyncMessageAckFuture(nodeId);
 
                 IgniteUuid futId = IgniteUuid.randomUuid();
 
@@ -779,29 +653,26 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param nodeId Sender ID.
+     * @param node Sender.
      * @param req Start request.
      */
-    private void processStartRequest(UUID nodeId, GridContinuousMessage req) {
-        assert nodeId != null;
-        assert req != null;
-
+    private void processStartRequest(ClusterNode node, StartRoutineDiscoveryMessage req) {
         UUID routineId = req.routineId();
-        StartRequestData data = req.data();
+        StartRequestData data = req.startRequestData();
 
-        GridContinuousHandler hnd = data.hnd;
+        GridContinuousHandler hnd = data.handler();
 
         IgniteCheckedException err = null;
 
         try {
             if (ctx.config().isPeerClassLoadingEnabled()) {
-                String clsName = data.clsName;
+                String clsName = data.className();
 
                 if (clsName != null) {
-                    GridDeploymentInfo depInfo = data.depInfo;
+                    GridDeploymentInfo depInfo = data.deploymentInfo();
 
                     GridDeployment dep = ctx.deploy().getGlobalDeployment(depInfo.deployMode(), clsName, clsName,
-                        depInfo.userVersion(), nodeId, depInfo.classLoaderId(), depInfo.participants(), null);
+                        depInfo.userVersion(), node.id(), depInfo.classLoaderId(), depInfo.participants(), null);
 
                     if (dep == null)
                         throw new IgniteDeploymentCheckedException("Failed to obtain deployment for class: " + clsName);
@@ -809,127 +680,58 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     data.p2pUnmarshal(marsh, dep.classLoader());
                 }
 
-                hnd.p2pUnmarshal(nodeId, ctx);
+                hnd.p2pUnmarshal(node.id(), ctx);
             }
         }
         catch (IgniteCheckedException e) {
             err = e;
 
-            U.error(log, "Failed to register handler [nodeId=" + nodeId + ", routineId=" + routineId + ']', e);
+            U.error(log, "Failed to register handler [nodeId=" + node.id() + ", routineId=" + routineId + ']', e);
+        }
+
+        if (node.isClient()) {
+            Map<UUID, LocalRoutineInfo> clientRouteMap = clientInfos.get(node.id());
+
+            if (clientRouteMap == null) {
+                clientRouteMap = new HashMap<>();
+
+                Map<UUID, LocalRoutineInfo> old = clientInfos.put(node.id(), clientRouteMap);
+
+                assert old == null;
+            }
+
+            clientRouteMap.put(routineId, new LocalRoutineInfo(data.projectionPredicate(), hnd, data.bufferSize(),
+                data.interval()));
         }
 
         boolean registered = false;
 
         if (err == null) {
             try {
-                IgnitePredicate<ClusterNode> prjPred = data.prjPred;
+                IgnitePredicate<ClusterNode> prjPred = data.projectionPredicate();
+
+                ctx.resource().injectGeneric(prjPred);
 
                 if (prjPred == null || prjPred.apply(ctx.discovery().node(ctx.localNodeId()))) {
-                    registered = registerHandler(nodeId, routineId, hnd, data.bufSize, data.interval,
-                        data.autoUnsubscribe, false);
+                    registered = registerHandler(node.id(), routineId, hnd, data.bufferSize(), data.interval(),
+                        data.autoUnsubscribe(), false);
                 }
             }
             catch (IgniteCheckedException e) {
                 err = e;
 
-                U.error(log, "Failed to register handler [nodeId=" + nodeId + ", routineId=" + routineId + ']', e);
+                U.error(log, "Failed to register handler [nodeId=" + node.id() + ", routineId=" + routineId + ']', e);
             }
         }
 
-        try {
-            sendWithRetries(nodeId, new GridContinuousMessage(MSG_START_ACK, routineId, null, err, false), null);
-        }
-        catch (ClusterTopologyCheckedException ignored) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to send start acknowledgement to node (is node alive?): " + nodeId);
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send start acknowledgement to node: " + nodeId, e);
-        }
+        if (err != null)
+            req.addError(ctx.localNodeId(), err);
 
         if (registered)
             hnd.onListenerRegistered(routineId, ctx);
     }
 
     /**
-     * @param nodeId Sender ID.
-     * @param ack Start acknowledgement.
-     */
-    private void processStartAck(UUID nodeId, GridContinuousMessage ack) {
-        assert nodeId != null;
-        assert ack != null;
-
-        UUID routineId = ack.routineId();
-
-        final IgniteCheckedException err = ack.data();
-
-        if (err != null) {
-            if (waitForStartAck.remove(routineId) != null) {
-                final StartFuture fut = startFuts.remove(routineId);
-
-                if (fut != null) {
-                    fut.onDone(err);
-
-                    stopRoutine(routineId);
-                }
-            }
-        }
-
-        Collection<UUID> nodeIds = waitForStartAck.get(routineId);
-
-        if (nodeIds != null) {
-            nodeIds.remove(nodeId);
-
-            if (nodeIds.isEmpty())
-                completeStartFuture(routineId);
-        }
-    }
-
-    /**
-     * @param nodeId Sender ID.
-     * @param req Stop request.
-     */
-    private void processStopRequest(UUID nodeId, GridContinuousMessage req) {
-        assert nodeId != null;
-        assert req != null;
-
-        UUID routineId = req.routineId();
-
-        unregisterRemote(routineId);
-
-        try {
-            sendWithRetries(nodeId, new GridContinuousMessage(MSG_STOP_ACK, routineId, null, null, false), null);
-        }
-        catch (ClusterTopologyCheckedException ignored) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to send stop acknowledgement to node (is node alive?): " + nodeId);
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to send stop acknowledgement to node: " + nodeId, e);
-        }
-    }
-
-    /**
-     * @param nodeId Sender ID.
-     * @param ack Stop acknowledgement.
-     */
-    private void processStopAck(UUID nodeId, GridContinuousMessage ack) {
-        assert nodeId != null;
-        assert ack != null;
-
-        UUID routineId = ack.routineId();
-
-        Collection<UUID> nodeIds = waitForStopAck.get(routineId);
-
-        if (nodeIds != null) {
-            nodeIds.remove(nodeId);
-
-            if (nodeIds.isEmpty())
-                completeStopFuture(routineId);
-        }
-    }
-
-    /**
      * @param msg Message.
      */
     private void processMessageAck(GridContinuousMessage msg) {
@@ -972,36 +774,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * @param routineId Consume ID.
-     */
-    private void completeStartFuture(UUID routineId) {
-        assert routineId != null;
-
-        if (waitForStartAck.remove(routineId) != null) {
-            StartFuture fut = startFuts.remove(routineId);
-
-            assert fut != null;
-
-            fut.onRemoteRegistered();
-        }
-    }
-
-    /**
-     * @param routineId Consume ID.
-     */
-    private void completeStopFuture(UUID routineId) {
-        assert routineId != null;
-
-        if (waitForStopAck.remove(routineId) != null) {
-            GridFutureAdapter <?> fut = stopFuts.remove(routineId);
-
-            assert fut != null;
-
-            fut.onDone();
-        }
-    }
-
-    /**
      * @param nodeId Node ID.
      * @param routineId Consume ID.
      * @param hnd Handler.
@@ -1231,7 +1003,10 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * Local routine info.
      */
     @SuppressWarnings("PackageVisibleInnerClass")
-    static class LocalRoutineInfo {
+    static class LocalRoutineInfo implements Serializable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
         /** Projection predicate. */
         private final IgnitePredicate<ClusterNode> prjPred;
 
@@ -1430,133 +1205,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
-     * Start request data.
-     */
-    private static class StartRequestData implements Externalizable {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** Projection predicate. */
-        private IgnitePredicate<ClusterNode> prjPred;
-
-        /** Serialized projection predicate. */
-        private byte[] prjPredBytes;
-
-        /** Deployment class name. */
-        private String clsName;
-
-        /** Deployment info. */
-        private GridDeploymentInfo depInfo;
-
-        /** Handler. */
-        private GridContinuousHandler hnd;
-
-        /** Buffer size. */
-        private int bufSize;
-
-        /** Time interval. */
-        private long interval;
-
-        /** Automatic unsubscribe flag. */
-        private boolean autoUnsubscribe;
-
-        /**
-         * Required by {@link Externalizable}.
-         */
-        public StartRequestData() {
-            // No-op.
-        }
-
-        /**
-         * @param prjPred Serialized projection predicate.
-         * @param hnd Handler.
-         * @param bufSize Buffer size.
-         * @param interval Time interval.
-         * @param autoUnsubscribe Automatic unsubscribe flag.
-         */
-        StartRequestData(@Nullable IgnitePredicate<ClusterNode> prjPred, GridContinuousHandler hnd,
-            int bufSize, long interval, boolean autoUnsubscribe) {
-            assert hnd != null;
-            assert bufSize > 0;
-            assert interval >= 0;
-
-            this.prjPred = prjPred;
-            this.hnd = hnd;
-            this.bufSize = bufSize;
-            this.interval = interval;
-            this.autoUnsubscribe = autoUnsubscribe;
-        }
-
-        /**
-         * @param marsh Marshaller.
-         * @throws IgniteCheckedException In case of error.
-         */
-        void p2pMarshal(Marshaller marsh) throws IgniteCheckedException {
-            assert marsh != null;
-
-            prjPredBytes = marsh.marshal(prjPred);
-        }
-
-        /**
-         * @param marsh Marshaller.
-         * @param ldr Class loader.
-         * @throws IgniteCheckedException In case of error.
-         */
-        void p2pUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
-            assert marsh != null;
-
-            assert prjPred == null;
-            assert prjPredBytes != null;
-
-            prjPred = marsh.unmarshal(prjPredBytes, ldr);
-        }
-
-        /** {@inheritDoc} */
-        @Override public void writeExternal(ObjectOutput out) throws IOException {
-            boolean b = prjPredBytes != null;
-
-            out.writeBoolean(b);
-
-            if (b) {
-                U.writeByteArray(out, prjPredBytes);
-                U.writeString(out, clsName);
-                out.writeObject(depInfo);
-            }
-            else
-                out.writeObject(prjPred);
-
-            out.writeObject(hnd);
-            out.writeInt(bufSize);
-            out.writeLong(interval);
-            out.writeBoolean(autoUnsubscribe);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("unchecked")
-        @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
-            boolean b = in.readBoolean();
-
-            if (b) {
-                prjPredBytes = U.readByteArray(in);
-                clsName = U.readString(in);
-                depInfo = (GridDeploymentInfo)in.readObject();
-            }
-            else
-                prjPred = (IgnitePredicate<ClusterNode>)in.readObject();
-
-            hnd = (GridContinuousHandler)in.readObject();
-            bufSize = in.readInt();
-            interval = in.readLong();
-            autoUnsubscribe = in.readBoolean();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(StartRequestData.class, this);
-        }
-    }
-
-    /**
      * Discovery data.
      */
     private static class DiscoveryData implements Externalizable {
@@ -1570,6 +1218,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         @GridToStringInclude
         private Collection<DiscoveryDataItem> items;
 
+        private Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos;
+
         /**
          * Required by {@link Externalizable}.
          */
@@ -1580,11 +1230,13 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         /**
          * @param nodeId Node ID.
          */
-        DiscoveryData(UUID nodeId) {
+        DiscoveryData(UUID nodeId, Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos) {
             assert nodeId != null;
 
             this.nodeId = nodeId;
 
+            this.clientInfos = clientInfos;
+
             items = new ArrayList<>();
         }
 
@@ -1599,12 +1251,14 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         @Override public void writeExternal(ObjectOutput out) throws IOException {
             U.writeUuid(out, nodeId);
             U.writeCollection(out, items);
+            U.writeMap(out, clientInfos);
         }
 
         /** {@inheritDoc} */
         @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
             nodeId = U.readUuid(in);
             items = U.readCollection(in);
+            clientInfos = U.readMap(in);
         }
 
         /** {@inheritDoc} */
@@ -1716,13 +1370,6 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         private volatile GridTimeoutObject timeoutObj;
 
         /**
-         * Required by {@link Externalizable}.
-         */
-        public StartFuture() {
-            // No-op.
-        }
-
-        /**
          * @param ctx Kernal context.
          * @param routineId Consume ID.
          */
@@ -1833,10 +1480,9 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         private UUID nodeId;
 
         /**
-         * @param ctx Kernal context.
          * @param nodeId Master node ID.
          */
-        SyncMessageAckFuture(GridKernalContext ctx, UUID nodeId) {
+        SyncMessageAckFuture(UUID nodeId) {
             this.nodeId = nodeId;
         }
 
@@ -1852,76 +1498,4 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             return S.toString(SyncMessageAckFuture.class, this);
         }
     }
-
-    /**
-     * Timeout object for stop process.
-     */
-    private class StopTimeoutObject extends GridTimeoutObjectAdapter {
-        /** Timeout. */
-        private final long timeout;
-
-        /** Routine ID. */
-        private final UUID routineId;
-
-        /** Request. */
-        private final GridContinuousMessage req;
-
-        /**
-         * @param timeout Timeout.
-         * @param routineId Routine ID.
-         * @param req Request.
-         */
-        protected StopTimeoutObject(long timeout, UUID routineId, GridContinuousMessage req) {
-            super(timeout);
-
-            assert routineId != null;
-            assert req != null;
-
-            this.timeout = timeout;
-            this.routineId = routineId;
-            this.req = req;
-        }
-
-        /** {@inheritDoc} */
-        @Override public void onTimeout() {
-            Collection<UUID> ids = waitForStopAck.remove(routineId);
-
-            if (ids != null) {
-                U.warn(log, "Failed to get stop acknowledgement from nodes (timeout expired): " + ids +
-                    ". Will retry.");
-
-                StopFuture f = stopFuts.get(routineId);
-
-                if (f != null) {
-                    if (!ids.isEmpty()) {
-                        waitForStopAck.put(routineId, ids);
-
-                        // Resend requests.
-                        for (UUID id : ids) {
-                            try {
-                                sendWithRetries(id, req, null);
-                            }
-                            catch (ClusterTopologyCheckedException ignored) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Failed to resend stop request to node (is node alive?): " + id);
-                            }
-                            catch (IgniteCheckedException e) {
-                                U.error(log, "Failed to resend stop request to node: " + id, e);
-
-                                ids.remove(id);
-
-                                if (ids.isEmpty())
-                                    f.onDone(e);
-                            }
-                        }
-
-                        // Reschedule timeout.
-                        ctx.timeout().addTimeoutObject(new StopTimeoutObject(timeout, routineId, req));
-                    }
-                    else if (stopFuts.remove(routineId) != null)
-                        f.onDone();
-                }
-            }
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java
new file mode 100644
index 0000000..c721d44
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRequestData.java
@@ -0,0 +1,267 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.managers.deployment.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.marshaller.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+
+/**
+ * Start request data.
+ */
+class StartRequestData implements Externalizable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Projection predicate. */
+    private IgnitePredicate<ClusterNode> prjPred;
+
+    /** Serialized projection predicate. */
+    private byte[] prjPredBytes;
+
+    /** Deployment class name. */
+    private String clsName;
+
+    /** Deployment info. */
+    private GridDeploymentInfo depInfo;
+
+    /** Handler. */
+    private GridContinuousHandler hnd;
+
+    /** Buffer size. */
+    private int bufSize;
+
+    /** Time interval. */
+    private long interval;
+
+    /** Automatic unsubscribe flag. */
+    private boolean autoUnsubscribe;
+
+    /**
+     * Required by {@link java.io.Externalizable}.
+     */
+    public StartRequestData() {
+        // No-op.
+    }
+
+    /**
+     * @param prjPred Serialized projection predicate.
+     * @param hnd Handler.
+     * @param bufSize Buffer size.
+     * @param interval Time interval.
+     * @param autoUnsubscribe Automatic unsubscribe flag.
+     */
+    StartRequestData(@Nullable IgnitePredicate<ClusterNode> prjPred, GridContinuousHandler hnd,
+        int bufSize, long interval, boolean autoUnsubscribe) {
+        assert hnd != null;
+        assert bufSize > 0;
+        assert interval >= 0;
+
+        this.prjPred = prjPred;
+        this.hnd = hnd;
+        this.bufSize = bufSize;
+        this.interval = interval;
+        this.autoUnsubscribe = autoUnsubscribe;
+    }
+
+    /**
+     * @param marsh Marshaller.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    void p2pMarshal(Marshaller marsh) throws IgniteCheckedException {
+        assert marsh != null;
+
+        prjPredBytes = marsh.marshal(prjPred);
+    }
+
+    /**
+     * @param marsh Marshaller.
+     * @param ldr Class loader.
+     * @throws org.apache.ignite.IgniteCheckedException In case of error.
+     */
+    void p2pUnmarshal(Marshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
+        assert marsh != null;
+
+        assert prjPred == null;
+        assert prjPredBytes != null;
+
+        prjPred = marsh.unmarshal(prjPredBytes, ldr);
+    }
+
+    /**
+     * @return Projection predicate.
+     */
+    public IgnitePredicate<ClusterNode> projectionPredicate() {
+        return prjPred;
+    }
+
+    /**
+     * @param prjPred New projection predicate.
+     */
+    public void projectionPredicate(IgnitePredicate<ClusterNode> prjPred) {
+        this.prjPred = prjPred;
+    }
+
+    /**
+     * @return Serialized projection predicate.
+     */
+    public byte[] projectionPredicateBytes() {
+        return prjPredBytes;
+    }
+
+    /**
+     * @param prjPredBytes New serialized projection predicate.
+     */
+    public void projectionPredicateBytes(byte[] prjPredBytes) {
+        this.prjPredBytes = prjPredBytes;
+    }
+
+    /**
+     * @return Deployment class name.
+     */
+    public String className() {
+        return clsName;
+    }
+
+    /**
+     * @param clsName New deployment class name.
+     */
+    public void className(String clsName) {
+        this.clsName = clsName;
+    }
+
+    /**
+     * @return Deployment info.
+     */
+    public GridDeploymentInfo deploymentInfo() {
+        return depInfo;
+    }
+
+    /**
+     * @param depInfo New deployment info.
+     */
+    public void deploymentInfo(GridDeploymentInfo depInfo) {
+        this.depInfo = depInfo;
+    }
+
+    /**
+     * @return Handler.
+     */
+    public GridContinuousHandler handler() {
+        return hnd;
+    }
+
+    /**
+     * @param hnd New handler.
+     */
+    public void handler(GridContinuousHandler hnd) {
+        this.hnd = hnd;
+    }
+
+    /**
+     * @return Buffer size.
+     */
+    public int bufferSize() {
+        return bufSize;
+    }
+
+    /**
+     * @param bufSize New buffer size.
+     */
+    public void bufferSize(int bufSize) {
+        this.bufSize = bufSize;
+    }
+
+    /**
+     * @return Time interval.
+     */
+    public long interval() {
+        return interval;
+    }
+
+    /**
+     * @param interval New time interval.
+     */
+    public void interval(long interval) {
+        this.interval = interval;
+    }
+
+    /**
+     * @return Automatic unsubscribe flag.
+     */
+    public boolean autoUnsubscribe() {
+        return autoUnsubscribe;
+    }
+
+    /**
+     * @param autoUnsubscribe New automatic unsubscribe flag.
+     */
+    public void autoUnsubscribe(boolean autoUnsubscribe) {
+        this.autoUnsubscribe = autoUnsubscribe;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        boolean b = prjPredBytes != null;
+
+        out.writeBoolean(b);
+
+        if (b) {
+            U.writeByteArray(out, prjPredBytes);
+            U.writeString(out, clsName);
+            out.writeObject(depInfo);
+        }
+        else
+            out.writeObject(prjPred);
+
+        out.writeObject(hnd);
+        out.writeInt(bufSize);
+        out.writeLong(interval);
+        out.writeBoolean(autoUnsubscribe);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        boolean b = in.readBoolean();
+
+        if (b) {
+            prjPredBytes = U.readByteArray(in);
+            clsName = U.readString(in);
+            depInfo = (GridDeploymentInfo)in.readObject();
+        }
+        else
+            prjPred = (IgnitePredicate<ClusterNode>)in.readObject();
+
+        hnd = (GridContinuousHandler)in.readObject();
+        bufSize = in.readInt();
+        interval = in.readLong();
+        autoUnsubscribe = in.readBoolean();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartRequestData.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineAckDiscoveryMessage.java
new file mode 100644
index 0000000..5743dd4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineAckDiscoveryMessage.java
@@ -0,0 +1,63 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class StartRoutineAckDiscoveryMessage extends AbstractContinuousMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final Map<UUID, IgniteCheckedException> errs;
+
+    /**
+     * @param routineId Routine id.
+     * @param errs Errs.
+     */
+    public StartRoutineAckDiscoveryMessage(UUID routineId, Map<UUID, IgniteCheckedException> errs) {
+        super(routineId);
+
+        this.errs = new HashMap<>(errs);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /**
+     * @return Errs.
+     */
+    public Map<UUID, IgniteCheckedException> errs() {
+        return errs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartRoutineAckDiscoveryMessage.class, this, "routineId", routineId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
new file mode 100644
index 0000000..cff6239
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StartRoutineDiscoveryMessage.java
@@ -0,0 +1,85 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class StartRoutineDiscoveryMessage extends AbstractContinuousMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final StartRequestData startReqData;
+
+    /** */
+    private final Map<UUID, IgniteCheckedException> errs = new HashMap<>();
+
+    /**
+     * @param routineId Routine id.
+     * @param startReqData Start request data.
+     */
+    public StartRoutineDiscoveryMessage(UUID routineId, StartRequestData startReqData) {
+        super(routineId);
+
+        this.startReqData = startReqData;
+    }
+
+    /**
+     * @return Start request data.
+     */
+    public StartRequestData startRequestData() {
+        return startReqData;
+    }
+
+    /**
+     * @param nodeId Node id.
+     * @param e Exception.
+     */
+    public void addError(UUID nodeId, IgniteCheckedException e) {
+        errs.put(nodeId, e);
+    }
+
+    /**
+     * @return Errs.
+     */
+    public Map<UUID, IgniteCheckedException> errs() {
+        return errs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isMutable() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public DiscoveryCustomMessage ackMessage() {
+        return new StartRoutineAckDiscoveryMessage(routineId, errs);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StartRoutineDiscoveryMessage.class, this, "routineId", routineId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
new file mode 100644
index 0000000..256791a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineAckDiscoveryMessage.java
@@ -0,0 +1,49 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class StopRoutineAckDiscoveryMessage extends AbstractContinuousMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param routineId Routine id.
+     */
+    public StopRoutineAckDiscoveryMessage(UUID routineId) {
+        super(routineId);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StopRoutineAckDiscoveryMessage.class, this, "routineId", routineId());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineDiscoveryMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineDiscoveryMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineDiscoveryMessage.java
new file mode 100644
index 0000000..9dc2227
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/StopRoutineDiscoveryMessage.java
@@ -0,0 +1,49 @@
+/*
+ * 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.continuous;
+
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class StopRoutineDiscoveryMessage extends AbstractContinuousMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /**
+     * @param routineId Routine id.
+     */
+    public StopRoutineDiscoveryMessage(UUID routineId) {
+        super(routineId);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public DiscoveryCustomMessage ackMessage() {
+        return new StopRoutineAckDiscoveryMessage(routineId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StopRoutineDiscoveryMessage.class, this, "routineId", routineId());
+    }
+}


[06/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
deleted file mode 100644
index 3cec217..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedBatchEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,385 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Sorted batch eviction test.
- */
-public class GridCacheSortedBatchEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<SortedEvictionPolicy<String, String>>{
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            GridCacheEvictionAbstractTest.MockEntry e1 = new GridCacheEvictionAbstractTest.MockEntry("1", "1");
-            GridCacheEvictionAbstractTest.MockEntry e2 = new GridCacheEvictionAbstractTest.MockEntry("2", "2");
-            GridCacheEvictionAbstractTest.MockEntry e3 = new GridCacheEvictionAbstractTest.MockEntry("3", "3");
-            GridCacheEvictionAbstractTest.MockEntry e4 = new GridCacheEvictionAbstractTest.MockEntry("4", "4");
-            GridCacheEvictionAbstractTest.MockEntry e5 = new GridCacheEvictionAbstractTest.MockEntry("5", "5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.set(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.set(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.set(), e1, e2, e3);
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.set(), e1, e2, e3, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            assertEquals(4, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e5);
-
-            // Batch evicted.
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new GridCacheEvictionAbstractTest.MockEntry("1", "1"));
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertEquals(4, p.getCurrentSize());
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(4, p.getCurrentSize());
-
-            check(p.set(), e1, e3, e4, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-            p.setBatchSize(batchSize);
-
-            int cnt = max + batchSize;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(cnt - batchSize, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            int batchSize = 2;
-
-            p.setMaxSize(max);
-
-            p.setBatchSize(batchSize);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            GridCacheEvictionAbstractTest.MockEntry[] fifos = new GridCacheEvictionAbstractTest.MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                GridCacheEvictionAbstractTest.MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new GridCacheEvictionAbstractTest.MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize < max + batchSize :
-                "curSize < max + batchSize [curSize=" + curSize + ", max=" + max + ", batchSize=" + batchSize + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            GridCacheEvictionAbstractTest.MockEntry e1 = new GridCacheEvictionAbstractTest.MockEntry("1");
-
-            GridCacheEvictionAbstractTest.MockEntry e2 = new GridCacheEvictionAbstractTest.MockEntry("2");
-
-            GridCacheEvictionAbstractTest.MockEntry e3 = new GridCacheEvictionAbstractTest.MockEntry("3");
-
-            GridCacheEvictionAbstractTest.MockEntry e4 = new GridCacheEvictionAbstractTest.MockEntry("4");
-
-            GridCacheEvictionAbstractTest.MockEntry e5 = new GridCacheEvictionAbstractTest.MockEntry("5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setBatchSize(2);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 10;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            // Batch evicted.
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        plcBatchSize = 2;
-
-        super.testPartitionedNearDisabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new SortedEvictionPolicy<>(10, 2, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new SortedEvictionPolicy<>(nearMax, 2, null);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).set())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).set();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).set().size() <= plcMax + policy(i).getBatchSize();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
deleted file mode 100644
index 8d97a1e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicyPerformanceTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import org.jsr166.*;
-
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * {@link SortedEvictionPolicy} performance test.
- */
-public class GridCacheSortedEvictionPolicyPerformanceTest extends GridCommonAbstractTest {
-    /** Threads. */
-    private static final int THREADS = 8;
-
-    /** Keys. */
-    private static final int KEYS = 100_000;
-
-    /** Max size. */
-    private static final int MAX_SIZE = 1000;
-
-    /** Put probability. */
-    private static final int P_PUT = 50;
-
-    /** Get probability. */
-    private static final int P_GET = 30;
-
-    /** Measurement count. */
-    private static final int MEASUREMENT_CNT = 100;
-
-    /** Rnd. */
-    private static final ThreadLocalRandom8 RND = ThreadLocalRandom8.current();
-
-    /** Ignite. */
-    private static Ignite ignite;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        ignite = startGrid();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        CacheConfiguration ccfg = defaultCacheConfiguration();
-
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
-        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-        ccfg.setNearConfiguration(null);
-        ccfg.setEvictionPolicy(new SortedEvictionPolicy(MAX_SIZE));
-//        ccfg.setEvictionPolicy(new FifoEvictionPolicy(MAX_SIZE));
-       ccfg.setEvictSynchronized(false);
-
-        cfg.setPeerClassLoadingEnabled(false);
-
-        cfg.setCacheConfiguration(ccfg);
-
-        return cfg;
-    }
-
-    /**
-     * Tests throughput.
-     */
-    public void testThroughput() throws Exception {
-        final LongAdder8 cnt = new LongAdder8();
-        final AtomicBoolean finished = new AtomicBoolean();
-
-        final int pPut = P_PUT;
-        final int pGet = P_PUT + P_GET;
-
-        final IgniteCache<Integer, Integer> cache = ignite.cache(null);
-
-        multithreadedAsync(new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                for (;;) {
-                    U.sleep(1000);
-
-                    info("Ops/sec: " + cnt.sumThenReset());
-                }
-            }
-        }, 1);
-
-        multithreaded(
-            new Callable<Object>() {
-                @Override public Object call() throws Exception {
-                    while (!finished.get()) {
-                        int p = RND.nextInt(100);
-
-                        int key = RND.nextInt(KEYS);
-
-                        if (p >= 0 && p < pPut)
-                            cache.put(key, 0);
-                        else if (p >= pPut && p < pGet)
-                            cache.get(key);
-                        else
-                            cache.remove(key);
-
-                        cnt.increment();
-                    }
-
-                    return null;
-                }
-            }, THREADS);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
deleted file mode 100644
index 041234e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/GridCacheSortedEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,373 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.sorted;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.sorted.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * Sorted eviction test.
- */
-public class GridCacheSortedEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<SortedEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.set(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.set(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.set(), e1, e2, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.set(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e2.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.set(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.set(), e3, e4, e5);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.set(), e3, e4, e5);
-
-            assertTrue(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.set(), e3, e4, e5);
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e3);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assertFalse(e4.isEvicted());
-            assertFalse(e5.isEvicted());
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assertFalse(e5.isEvicted());
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assertTrue("curSize <= max [curSize=" + curSize + ", max=" + max + ']', curSize <= max);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            SortedEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assertTrue("Min cache size is too small: " + min, min >= plcMax);
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            assertTrue("Min cache size is too small: " + min, min >= plcMax);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new SortedEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new SortedEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).set())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).set();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).set().size() <= plcMax;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
new file mode 100644
index 0000000..a687da9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicyPerformanceTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.eviction.sorted;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.sorted.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import org.jsr166.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * {@link SortedEvictionPolicy} performance test.
+ */
+public class SortedEvictionPolicyPerformanceTest extends GridCommonAbstractTest {
+    /** Threads. */
+    private static final int THREADS = 8;
+
+    /** Keys. */
+    private static final int KEYS = 100_000;
+
+    /** Max size. */
+    private static final int MAX_SIZE = 1000;
+
+    /** Put probability. */
+    private static final int P_PUT = 50;
+
+    /** Get probability. */
+    private static final int P_GET = 30;
+
+    /** Rnd. */
+    private static final ThreadLocalRandom8 RND = ThreadLocalRandom8.current();
+
+    /** Ignite. */
+    private static Ignite ignite;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        ignite = startGrid();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setNearConfiguration(null);
+
+        SortedEvictionPolicy plc = new SortedEvictionPolicy();
+        plc.setMaxSize(MAX_SIZE);
+
+        ccfg.setEvictionPolicy(plc);
+        ccfg.setEvictSynchronized(false);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * Tests throughput.
+     */
+    public void testThroughput() throws Exception {
+        final LongAdder8 cnt = new LongAdder8();
+        final AtomicBoolean finished = new AtomicBoolean();
+
+        final int pPut = P_PUT;
+        final int pGet = P_PUT + P_GET;
+
+        final IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+        multithreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (;;) {
+                    U.sleep(1000);
+
+                    info("Ops/sec: " + cnt.sumThenReset());
+                }
+            }
+        }, 1);
+
+        multithreaded(
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    while (!finished.get()) {
+                        int p = RND.nextInt(100);
+
+                        int key = RND.nextInt(KEYS);
+
+                        if (p >= 0 && p < pPut)
+                            cache.put(key, 0);
+                        else if (p >= pPut && p < pGet)
+                            cache.get(key);
+                        else
+                            cache.remove(key);
+
+                        cnt.increment();
+                    }
+
+                    return null;
+                }
+            }, THREADS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
new file mode 100644
index 0000000..a283352
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/sorted/SortedEvictionPolicySelfTest.java
@@ -0,0 +1,266 @@
+/*
+ * 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.eviction.sorted;
+
+import org.apache.ignite.cache.eviction.sorted.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * Sorted eviction policy tests.
+ */
+public class SortedEvictionPolicySelfTest extends
+    EvictionAbstractTest<SortedEvictionPolicy<String, String>> {
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            SortedEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e5.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        try {
+            startGrid();
+
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            SortedEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted.
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e3, e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e3.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected SortedEvictionPolicy<String, String> createPolicy(int plcMax) {
+        SortedEvictionPolicy<String, String> plc = new SortedEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected SortedEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        SortedEvictionPolicy<String, String> plc = new SortedEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheClientNearCacheExpiryTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheClientNearCacheExpiryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheClientNearCacheExpiryTest.java
new file mode 100644
index 0000000..602ac18
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheClientNearCacheExpiryTest.java
@@ -0,0 +1,103 @@
+/*
+ * 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.expiry;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import javax.cache.expiry.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNearCacheExpiryTest extends IgniteCacheAbstractTest {
+    /** */
+    private static final int NODES = 3;
+
+    /** {@inheritDoc} */
+    @Override protected int gridCount() {
+        return NODES;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheMode cacheMode() {
+        return PARTITIONED;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return TRANSACTIONAL;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected NearCacheConfiguration nearConfiguration() {
+        return new NearCacheConfiguration();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (gridName.equals(getTestGridName(NODES - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExpirationOnClient() throws Exception {
+        Ignite ignite = grid(NODES - 1);
+
+        assertTrue(ignite.configuration().isClientMode());
+
+        IgniteCache<Object, Object> cache = ignite.cache(null);
+
+        assertTrue(((IgniteCacheProxy)cache).context().isNear());
+
+        for (int i = 0 ; i < 100; i++)
+            cache.put(i, i);
+
+        CreatedExpiryPolicy plc = new CreatedExpiryPolicy(new Duration(TimeUnit.MILLISECONDS, 500));
+
+        IgniteCache<Object, Object> cacheWithExpiry = cache.withExpiryPolicy(plc);
+
+        for (int i = 100 ; i < 200; i++) {
+            cacheWithExpiry.put(i, i);
+
+            assertEquals(i, cacheWithExpiry.localPeek(i));
+        }
+
+        U.sleep(1000);
+
+        for (int i = 0 ; i < 100; i++)
+            assertEquals(i, cacheWithExpiry.localPeek(i));
+
+        for (int i = 100 ; i < 200; i++)
+            assertNull(cache.localPeek(i));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java
index c006f69..c78ec5c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyTestSuite.java
@@ -50,6 +50,8 @@ public class IgniteCacheExpiryPolicyTestSuite extends TestSuite {
 
         suite.addTestSuite(IgniteCacheTtlCleanupSelfTest.class);
 
+        suite.addTestSuite(IgniteCacheClientNearCacheExpiryTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
new file mode 100644
index 0000000..3d44600
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/CacheLocalOffHeapAndSwapMetricsSelfTest.java
@@ -0,0 +1,412 @@
+/*
+ * 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.local;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.fifo.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+/**
+ *
+ */
+public class CacheLocalOffHeapAndSwapMetricsSelfTest extends GridCommonAbstractTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 1;
+
+    /** Keys count. */
+    private static final int KEYS_CNT = 1000;
+
+    /** Max size. */
+    private static final int MAX_SIZE = 100;
+
+    /** Entry size. */
+    private static final int ENTRY_SIZE = 86; // Calculated as allocated size divided on entries count.
+
+    /** Offheap max count. */
+    private static final int OFFHEAP_MAX_CNT = KEYS_CNT / 2;
+
+    /** Offheap max size. */
+    private static final int OFFHEAP_MAX_SIZE = ENTRY_SIZE * OFFHEAP_MAX_CNT;
+
+    /** Cache. */
+    private IgniteCache<Integer, Integer> cache;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
+
+        return cfg;
+    }
+
+    /**
+     * @param offHeapSize Max off-heap size.
+     * @param swapEnabled Swap enabled.
+     */
+    private void createCache(int offHeapSize, boolean swapEnabled) {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setStatisticsEnabled(true);
+
+        ccfg.setCacheMode(CacheMode.LOCAL);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+        ccfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
+
+        ccfg.setOffHeapMaxMemory(offHeapSize);
+        ccfg.setSwapEnabled(swapEnabled);
+
+        ccfg.setEvictionPolicy(new FifoEvictionPolicy(MAX_SIZE));
+
+        cache = grid(0).getOrCreateCache(ccfg);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(GRID_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (cache != null)
+            cache.close();
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testOffHeapMetrics() throws Exception {
+        createCache(0, false);
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.put(i, i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapGets());
+        assertEquals(0, cache.metrics().getOffHeapHits());
+        assertEquals(0f, cache.metrics().getOffHeapHitPercentage());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapMisses());
+        assertEquals(100f, cache.metrics().getOffHeapMissPercentage());
+        assertEquals(0, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(0, cache.metrics().getOffHeapEvictions());
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapEntriesCount());
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 2, cache.metrics().getOffHeapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapHits());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapMisses());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(0, cache.metrics().getOffHeapEvictions());
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 3, cache.metrics().getOffHeapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapHits());
+        assertEquals(100 / 3.0, cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getOffHeapMisses());
+        assertEquals(100 - (100 / 3.0), cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(0, cache.metrics().getOffHeapEvictions());
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.remove(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 4 - MAX_SIZE, cache.metrics().getOffHeapGets());
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.metrics().getOffHeapHits());
+        assertEquals(100 * (KEYS_CNT * 2.0 - MAX_SIZE) / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getOffHeapMisses());
+        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(0, cache.metrics().getOffHeapEvictions());
+        assertEquals(0, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testSwapMetrics() throws Exception {
+        createCache(-1, true);
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.put(i, i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapGets());
+        assertEquals(0, cache.metrics().getSwapHits());
+        assertEquals(0f, cache.metrics().getSwapHitPercentage());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
+        assertEquals(100f, cache.metrics().getSwapMissPercentage());
+        assertEquals(0, cache.metrics().getSwapRemovals());
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getSwapEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
+
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getSwapEntriesCount());
+
+        for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 3, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
+        assertEquals(100 / 3.0, cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
+        assertEquals(100 - (100 / 3.0), cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
+
+        assertEquals(KEYS_CNT - MAX_SIZE, cache.metrics().getSwapEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.remove(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 4 - MAX_SIZE, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.metrics().getSwapHits());
+        assertEquals(100 * (KEYS_CNT * 2.0 - MAX_SIZE) / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
+        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE, cache.metrics().getSwapRemovals());
+
+        assertEquals(0, cache.metrics().getSwapEntriesCount());
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    public void testOffHeapAndSwapMetrics() throws Exception {
+        createCache(OFFHEAP_MAX_SIZE, true);
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.put(i, i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapGets());
+        assertEquals(0, cache.metrics().getOffHeapHits());
+        assertEquals(0f, cache.metrics().getOffHeapHitPercentage());
+        assertEquals(KEYS_CNT, cache.metrics().getOffHeapMisses());
+        assertEquals(100f, cache.metrics().getOffHeapMissPercentage());
+        assertEquals(0, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(KEYS_CNT - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEvictions());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapGets());
+        assertEquals(0, cache.metrics().getSwapHits());
+        assertEquals(0f, cache.metrics().getSwapHitPercentage());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
+        assertEquals(100f, cache.metrics().getSwapMissPercentage());
+        assertEquals(0, cache.metrics().getSwapRemovals());
+
+        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 2, cache.metrics().getOffHeapGets());
+        assertEquals(0, cache.metrics().getOffHeapHits());
+        assertEquals(0.0, cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getOffHeapMisses());
+        assertEquals(100.0, cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(0, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(cache.metrics().getCacheEvictions() - OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEvictions());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapMisses());
+        assertEquals(100 * KEYS_CNT / (KEYS_CNT * 2.0), cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
+
+        assertEquals(KEYS_CNT - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getSwapEntriesCount());
+
+        for (int i = KEYS_CNT; i < KEYS_CNT * 2; i++)
+            cache.get(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 3, cache.metrics().getOffHeapGets());
+        assertEquals(0, cache.metrics().getOffHeapHits());
+        assertEquals(0.0, cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 3, cache.metrics().getOffHeapMisses());
+        assertEquals(100.0, cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(0, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(cache.metrics().getCacheEvictions() - OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEvictions());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 3, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT, cache.metrics().getSwapHits());
+        assertEquals(100 / 3.0, cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
+        assertEquals(100 - (100 / 3.0), cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT, cache.metrics().getSwapRemovals());
+
+        assertEquals(KEYS_CNT - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getSwapEntriesCount());
+
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.remove(i);
+
+        printStat();
+
+        assertEquals(cache.metrics().getCacheEvictions(), cache.metrics().getOffHeapPuts());
+        assertEquals(KEYS_CNT * 4 - MAX_SIZE, cache.metrics().getOffHeapGets());
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapHits());
+        assertEquals(100 * OFFHEAP_MAX_CNT / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getOffHeapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 4 - OFFHEAP_MAX_CNT - MAX_SIZE, cache.metrics().getOffHeapMisses());
+        assertEquals(100 * (KEYS_CNT * 4 - OFFHEAP_MAX_CNT - MAX_SIZE) / (KEYS_CNT * 4.0 - MAX_SIZE),
+            cache.metrics().getOffHeapMissPercentage(), 0.1);
+        assertEquals(OFFHEAP_MAX_CNT, cache.metrics().getOffHeapRemovals());
+
+        assertEquals(cache.metrics().getCacheEvictions() - OFFHEAP_MAX_CNT, cache.metrics().getOffHeapEvictions());
+        assertEquals(0, cache.metrics().getOffHeapEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapPrimaryEntriesCount());
+        assertEquals(0, cache.metrics().getOffHeapBackupEntriesCount());
+
+        assertEquals(cache.metrics().getOffHeapEvictions(), cache.metrics().getSwapPuts());
+        assertEquals(KEYS_CNT * 4 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getSwapGets());
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getSwapHits());
+        assertEquals(100 * (KEYS_CNT * 2.0 - MAX_SIZE - OFFHEAP_MAX_CNT) / (KEYS_CNT * 4.0 - MAX_SIZE - OFFHEAP_MAX_CNT),
+            cache.metrics().getSwapHitPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2, cache.metrics().getSwapMisses());
+        assertEquals(100 * KEYS_CNT * 2.0 / (KEYS_CNT * 4.0 - MAX_SIZE - OFFHEAP_MAX_CNT),
+            cache.metrics().getSwapMissPercentage(), 0.1);
+        assertEquals(KEYS_CNT * 2 - MAX_SIZE - OFFHEAP_MAX_CNT, cache.metrics().getSwapRemovals());
+
+        assertEquals(0, cache.metrics().getSwapEntriesCount());
+    }
+
+    /**
+     * Prints stats.
+     */
+    protected void printStat() {
+        System.out.println("!!! -------------------------------------------------------");
+        System.out.println("!!! Puts: cache = " + cache.metrics().getCachePuts() +
+            ", offheap = " + cache.metrics().getOffHeapPuts() +
+            ", swap = " + cache.metrics().getSwapPuts());
+        System.out.println("!!! Gets: cache = " + cache.metrics().getCacheGets() +
+            ", offheap = " + cache.metrics().getOffHeapGets() +
+            ", swap = " + cache.metrics().getSwapGets());
+        System.out.println("!!! Removes: cache = " + cache.metrics().getCacheRemovals() +
+            ", offheap = " + cache.metrics().getOffHeapRemovals() +
+            ", swap = " + cache.metrics().getSwapRemovals());
+        System.out.println("!!! Evictions: cache = " + cache.metrics().getCacheEvictions() +
+            ", offheap = " + cache.metrics().getOffHeapEvictions() +
+            ", swap = none" );
+        System.out.println("!!! Hits: cache = " + cache.metrics().getCacheHits() +
+            ", offheap = " + cache.metrics().getOffHeapHits() +
+            ", swap = " + cache.metrics().getSwapHits());
+        System.out.println("!!! Hit(%): cache = " + cache.metrics().getCacheHitPercentage() +
+            ", offheap = " + cache.metrics().getOffHeapHitPercentage() +
+            ", swap = " + cache.metrics().getSwapHitPercentage());
+        System.out.println("!!! Misses: cache = " + cache.metrics().getCacheMisses() +
+            ", offheap = " + cache.metrics().getOffHeapMisses() +
+            ", swap = " + cache.metrics().getSwapMisses());
+        System.out.println("!!! Miss(%): cache = " + cache.metrics().getCacheMissPercentage() +
+            ", offheap = " + cache.metrics().getOffHeapMissPercentage() +
+            ", swap = " + cache.metrics().getSwapMissPercentage());
+        System.out.println("!!! Entries: cache = " + cache.metrics().getSize() +
+            ", offheap = " + cache.metrics().getOffHeapEntriesCount() +
+            ", swap = " + cache.metrics().getSwapEntriesCount());
+        System.out.println("!!! Size: cache = none" +
+            ", offheap = " + cache.metrics().getOffHeapAllocatedSize() +
+            ", swap = " + cache.metrics().getSwapSize());
+        System.out.println();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
deleted file mode 100644
index 71c0495..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/local/GridCacheExLocalFullApiSelfTest.java
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.local;
-
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-
-/**
- * Tests private cache interface on local cache.
- */
-public class GridCacheExLocalFullApiSelfTest extends GridCacheExAbstractFullApiSelfTest {
-    @Override protected CacheMode cacheMode() {
-        return CacheMode.LOCAL;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index 5a78f9f..378d5a3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -42,6 +42,7 @@ import javax.cache.*;
 import javax.cache.configuration.*;
 import javax.cache.event.*;
 import javax.cache.integration.*;
+import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -57,7 +58,7 @@ import static org.apache.ignite.internal.processors.cache.query.CacheQueryType.*
 /**
  * Continuous queries tests.
  */
-public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommonAbstractTest {
+public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommonAbstractTest implements Serializable {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -177,10 +178,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             assertEquals(String.valueOf(i), 3, ((Map)U.field(proc, "locInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "rmtInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "startFuts")).size());
-            assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "waitForStartAck")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "stopFuts")).size());
-            assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "waitForStopAck")).size());
-            assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "pending")).size());
 
             CacheContinuousQueryManager mgr = grid(i).context().cache().internalCache().context().continuousQueries();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index d219a60..9ffef4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -28,9 +28,11 @@ import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 
-import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -42,7 +44,7 @@ import static org.apache.ignite.internal.processors.continuous.GridContinuousPro
 /**
  * Event consume test.
  */
-public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements Serializable {
+public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
     /** */
     private static final String PRJ_PRED_CLS_NAME = "org.apache.ignite.tests.p2p.GridEventConsumeProjectionPredicate";
 
@@ -55,6 +57,9 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
     /** Number of created consumes per thread in multithreaded test. */
     private static final int CONSUME_CNT = 500;
 
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** Consume latch. */
     private static volatile CountDownLatch consumeLatch;
 
@@ -71,6 +76,12 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        TcpDiscoverySpi disc = new TcpDiscoverySpi();
+
+        disc.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disc);
+
         if (include)
             cfg.setUserAttributes(F.asMap("include", true));
 
@@ -80,7 +91,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
+    @Override protected void beforeTest() throws Exception {
         assertTrue(GRID_CNT > 1);
 
         include = true;
@@ -93,32 +104,33 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
     }
 
     /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        assertEquals(GRID_CNT, grid(0).cluster().nodes().size());
+        try {
+            assertEquals(GRID_CNT, grid(0).cluster().nodes().size());
 
-        for (int i = 0; i < GRID_CNT; i++) {
-            IgniteKernal grid = (IgniteKernal)grid(i);
+            for (int i = 0; i < GRID_CNT; i++) {
+                IgniteEx grid = grid(i);
 
-            GridContinuousProcessor proc = grid.context().continuous();
+                GridContinuousProcessor proc = grid.context().continuous();
 
-            if (noAutoUnsubscribe) {
-                localRoutines(proc).clear();
+                try {
+                    if (!noAutoUnsubscribe) {
+                        Map rmtInfos = U.field(proc, "rmtInfos");
 
-                U.<Map>field(proc, "rmtInfos").clear();
-            }
+                        assertTrue("Unexpected remote infos: " + rmtInfos, rmtInfos.isEmpty());
+                    }
+                }
+                finally {
+                    U.<Map>field(proc, "rmtInfos").clear();
+                }
 
-            assertEquals(0, localRoutines(proc).size());
-            assertEquals(0, U.<Map>field(proc, "rmtInfos").size());
-            assertEquals(0, U.<Map>field(proc, "startFuts").size());
-            assertEquals(0, U.<Map>field(proc, "waitForStartAck").size());
-            assertEquals(0, U.<Map>field(proc, "stopFuts").size());
-            assertEquals(0, U.<Map>field(proc, "waitForStopAck").size());
-            assertEquals(0, U.<Map>field(proc, "pending").size());
+                assertEquals(0, U.<Map>field(proc, "rmtInfos").size());
+                assertEquals(0, U.<Map>field(proc, "startFuts").size());
+                assertEquals(0, U.<Map>field(proc, "stopFuts").size());
+            }
+        }
+        finally {
+            stopAllGrids();
         }
     }
 
@@ -511,28 +523,6 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
     /**
      * @throws Exception If failed.
      */
-    public void testEmptyProjection() throws Exception {
-        try {
-            events(grid(0).cluster().forPredicate(F.<ClusterNode>alwaysFalse())).remoteListen(
-                new P2<UUID, Event>() {
-                    @Override public boolean apply(UUID nodeId, Event evt) {
-                        return true;
-                    }
-                },
-                null
-            );
-
-            assert false : "Exception was not thrown.";
-        }
-        catch (IgniteException e) {
-            assertTrue(e.getMessage().startsWith(
-                "Failed to register remote continuous listener (projection is empty)."));
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
     public void testStopByCallback() throws Exception {
         final Collection<UUID> nodeIds = new HashSet<>();
         final AtomicInteger cnt = new AtomicInteger();
@@ -665,7 +655,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
         final CountDownLatch latch = new CountDownLatch(GRID_CNT + 1);
 
         UUID consumeId = grid(0).events().remoteListen(
-            new P2<UUID, Event>() {
+            notSerializableProxy(new P2<UUID, Event>() {
                 @Override public boolean apply(UUID nodeId, Event evt) {
                     info("Event from " + nodeId + " [" + evt.shortDisplay() + ']');
 
@@ -677,12 +667,12 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
 
                     return true;
                 }
-            },
-            new P1<Event>() {
+            }),
+            notSerializableProxy(new P1<Event>() {
                 @Override public boolean apply(Event evt) {
                     return evt.type() == EVT_JOB_STARTED;
                 }
-            },
+            }),
             EVT_JOB_STARTED, EVT_JOB_FINISHED
         );
 
@@ -873,14 +863,14 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
         final CountDownLatch latch = new CountDownLatch(GRID_CNT);
 
         for (int i = 0; i < GRID_CNT; i++) {
-            grid(0).events().localListen(new IgnitePredicate<Event>() {
+            grid(i).events().localListen(new IgnitePredicate<Event>() {
                 @Override public boolean apply(Event evt) {
                     if (nodeId.equals(((DiscoveryEvent) evt).eventNode().id()))
                         latch.countDown();
 
                     return true;
                 }
-            }, EVT_NODE_LEFT);
+            }, EVT_NODE_LEFT, EVT_NODE_FAILED);
         }
 
         g.events().remoteListen(
@@ -895,7 +885,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
 
         stopGrid("anotherGrid");
 
-        latch.await();
+        assert latch.await(3000, MILLISECONDS);
     }
 
     /**
@@ -941,7 +931,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest implements
 
         stopGrid("anotherGrid");
 
-        discoLatch.await();
+        discoLatch.await(3000, MILLISECONDS);
 
         grid(0).compute().broadcast(F.noop());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index 1f0319f..81751f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -84,6 +84,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
+        spi.setForceServerMode(true);
         spi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(spi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
index bea3809..e0092d4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerMultiThreadedSelfTest.java
@@ -41,6 +41,9 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
+    /** */
+    private boolean dynamicCache;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -50,13 +53,22 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoSpi);
 
+        if (!dynamicCache)
+            cfg.setCacheConfiguration(cacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
         ccfg.setCacheMode(PARTITIONED);
         ccfg.setBackups(1);
-        cfg.setCacheConfiguration(ccfg);
 
-        return cfg;
+        return ccfg;
     }
 
     /** {@inheritDoc} */
@@ -68,8 +80,22 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testStartStopIgnites() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-840");
+        startStopIgnites();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopIgnitesDynamicCache() throws Exception {
+        dynamicCache = true;
+
+        startStopIgnites();
+    }
 
+    /**
+     * @throws Exception If failed.
+     */
+    private void startStopIgnites() throws Exception {
         for (int attempt = 0; attempt < 3; ++attempt) {
             log.info("Iteration: " + attempt);
             
@@ -77,28 +103,29 @@ public class DataStreamerMultiThreadedSelfTest extends GridCommonAbstractTest {
 
             Set<IgniteFuture> futs = new HashSet<>();
 
-            IgniteInternalFuture<?> fut;
+            final AtomicInteger igniteId = new AtomicInteger(1);
 
-            try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) {
-                dataLdr.maxRemapCount(0);
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    for (int i = 1; i < 5; ++i)
+                        startGrid(igniteId.incrementAndGet());
 
-                final AtomicInteger igniteId = new AtomicInteger(1);
+                    return true;
+                }
+            }, 2, "start-node-thread");
 
-                fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
-                    @Override public Object call() throws Exception {
-                        for (int i = 1; i < 5; ++i)
-                            startGrid(igniteId.incrementAndGet());
+            if (dynamicCache)
+                ignite.getOrCreateCache(cacheConfiguration());
 
-                        return true;
-                    }
-                }, 2, "start-node-thread");
+            try (final DataStreamerImpl dataLdr = (DataStreamerImpl)ignite.dataStreamer(null)) {
+                dataLdr.maxRemapCount(0);
 
-                Random random = new Random();
+                Random rnd = new Random();
 
                 long endTime = U.currentTimeMillis() + 15_000;
 
                 while (!fut.isDone() && U.currentTimeMillis() < endTime)
-                    futs.add(dataLdr.addData(random.nextInt(100_000), random.nextInt(100_000)));
+                    futs.add(dataLdr.addData(rnd.nextInt(100_000), String.valueOf(rnd.nextInt(100_000))));
             }
 
             for (IgniteFuture f : futs)


[10/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
new file mode 100644
index 0000000..bbc9144
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
@@ -0,0 +1,168 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Tests {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} in client mode.
+ */
+@SuppressWarnings("RedundantMethodOverride")
+public abstract class GridCacheClientModesTcpClientDiscoveryAbstractTest extends GridCacheClientModesAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean isClientStartedLast() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(false);
+
+        return cfg;
+    }
+
+    /** */
+    public static class CaseNearReplicatedAtomic extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseNearReplicatedTransactional extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CaseNearPartitionedAtomic extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseNearPartitionedTransactional extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CaseClientReplicatedAtomic extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected NearCacheConfiguration nearConfiguration() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseClientReplicatedTransactional extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected NearCacheConfiguration nearConfiguration() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return REPLICATED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.TRANSACTIONAL;
+        }
+    }
+
+    /** */
+    public static class CaseClientPartitionedAtomic extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected NearCacheConfiguration nearConfiguration() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.ATOMIC;
+        }
+    }
+
+    /** */
+    public static class CaseClientPartitionedTransactional extends GridCacheClientModesTcpClientDiscoveryAbstractTest {
+        /** {@inheritDoc} */
+        @Override protected NearCacheConfiguration nearConfiguration() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheMode cacheMode() {
+            return PARTITIONED;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected CacheAtomicityMode atomicityMode() {
+            return CacheAtomicityMode.TRANSACTIONAL;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
index e19442f..a3c977f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 /**
@@ -31,6 +32,8 @@ public class GridCacheMixedModeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         cfg.setCacheConfiguration(cacheConfiguration(gridName));
 
         if (F.eq(gridName, getTestGridName(0)))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
new file mode 100644
index 0000000..6782ff4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -0,0 +1,1803 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.cache.affinity.*;
+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.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.processors.cache.version.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.resources.*;
+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.apache.ignite.transactions.*;
+import org.eclipse.jetty.util.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.concurrent.locks.*;
+
+import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private CacheConfiguration ccfg;
+
+    /** */
+    private boolean client;
+
+    /** */
+    private volatile CyclicBarrier updateBarrier;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
+
+        cfg.setClientMode(client);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutAllClockMode() throws Exception {
+        atomicPut(CLOCK, true, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutAllPrimaryMode() throws Exception {
+        atomicPut(PRIMARY, true, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutAllNearEnabledClockMode() throws Exception {
+        atomicPut(CLOCK, true, new NearCacheConfiguration());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutAllNearEnabledPrimaryMode() throws Exception {
+        atomicPut(PRIMARY, true, new NearCacheConfiguration());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutClockMode() throws Exception {
+        atomicPut(CLOCK, false, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPutPrimaryMode() throws Exception {
+        atomicPut(PRIMARY, false, null);
+    }
+
+    /**
+     * @param writeOrder Write order.
+     * @param putAll If {@code true} executes putAll.
+     * @param nearCfg Near cache configuration.
+     * @throws Exception If failed.
+     */
+    private void atomicPut(CacheAtomicWriteOrderMode writeOrder,
+        final boolean putAll,
+        @Nullable NearCacheConfiguration nearCfg) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(writeOrder);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        ccfg.setNearConfiguration(nearCfg);
+
+        client = true;
+
+        ccfg.setNearConfiguration(null);
+
+        Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        final int KEYS = putAll ? 100 : 1;
+
+        for (int i = 0; i < KEYS; i++)
+            map.put(i, i);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        // Block messages requests for both nodes.
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id());
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        assertEquals(writeOrder, cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                if (putAll)
+                    cache.putAll(map);
+                else
+                    cache.put(0, 0);
+
+                return null;
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        IgniteEx ignite3 = startGrid(3);
+
+        log.info("Stop block1.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        checkData(map, null, cache, 4);
+
+        ignite3.close();
+
+        map.clear();
+
+        for (int i = 0; i < KEYS; i++)
+            map.put(i, i + 1);
+
+        // Block messages requests for single node.
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id());
+
+        putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                if (putAll)
+                    cache.putAll(map);
+                else
+                    cache.put(0, 1);
+
+                return null;
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        startGrid(3);
+
+        log.info("Stop block2.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        checkData(map, null, cache, 4);
+
+        for (int i = 0; i < KEYS; i++)
+            map.put(i, i + 2);
+
+        if (putAll)
+            cache.putAll(map);
+        else
+            cache.put(0, 2);
+
+        checkData(map, null, cache, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNoRemapClockMode() throws Exception {
+        atomicNoRemap(CLOCK);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNoRemapPrimaryMode() throws Exception {
+        atomicNoRemap(PRIMARY);
+    }
+
+    /**
+     * @param writeOrder Write order.
+     * @throws Exception If failed.
+     */
+    private void atomicNoRemap(CacheAtomicWriteOrderMode writeOrder) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(writeOrder);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+        IgniteEx ignite2 = startGrid(2);
+
+        client = true;
+
+        Ignite ignite3 = startGrid(3);
+
+        assertTrue(ignite3.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        map.put(primaryKey(ignite0.cache(null)), 0);
+        map.put(primaryKey(ignite1.cache(null)), 1);
+        map.put(primaryKey(ignite2.cache(null)), 2);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        // Block messages requests for both nodes.
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id());
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite2.localNode().id());
+
+        spi.record(GridNearAtomicUpdateRequest.class);
+
+        final IgniteCache<Integer, Integer> cache = ignite3.cache(null);
+
+        assertEquals(writeOrder, cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                cache.putAll(map);
+
+                return null;
+            }
+        });
+
+        IgniteEx ignite4 = startGrid(4);
+
+        assertTrue(ignite4.configuration().isClientMode());
+
+        assertFalse(putFut.isDone());
+
+        log.info("Stop block.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        spi.record(null);
+
+        checkData(map, null, cache, 5);
+
+        List<Object> msgs = spi.recordedMessages();
+
+        assertEquals(3, msgs.size());
+
+        for (Object msg : msgs)
+            assertTrue(((GridNearAtomicUpdateRequest)msg).clientRequest());
+
+        map.put(primaryKey(ignite0.cache(null)), 3);
+        map.put(primaryKey(ignite1.cache(null)), 4);
+        map.put(primaryKey(ignite2.cache(null)), 5);
+
+        cache.putAll(map);
+
+        checkData(map, null, cache, 5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicGetAndPutClockMode() throws Exception {
+        atomicGetAndPut(CLOCK);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicGetAndPutPrimaryMode() throws Exception {
+        atomicGetAndPut(PRIMARY);
+    }
+
+    /**
+     * @param writeOrder Write order.
+     * @throws Exception If failed.
+     */
+    private void atomicGetAndPut(CacheAtomicWriteOrderMode writeOrder) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicWriteOrderMode(writeOrder);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        ignite0.cache(null).put(0, 0);
+
+        Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        map.put(0, 1);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        // Block messages requests for both nodes.
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearAtomicUpdateRequest.class, ignite1.localNode().id());
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        assertEquals(writeOrder, cache.getConfiguration(CacheConfiguration.class).getAtomicWriteOrderMode());
+
+        IgniteInternalFuture<Integer> putFut = GridTestUtils.runAsync(new Callable<Integer>() {
+            @Override public Integer call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                return cache.getAndPut(0, 1);
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        startGrid(3);
+
+        log.info("Stop block.");
+
+        spi.stopBlock();
+
+        Integer old = putFut.get();
+
+        checkData(map, null, cache, 4);
+
+        assertEquals((Object)0, old);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxPutAll() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearTxPrepareRequest.class, ignite1.localNode().id());
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                cache.putAll(map);
+
+                return null;
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        IgniteEx ignite3 = startGrid(3);
+
+        log.info("Stop block.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        checkData(map, null, cache, 4);
+
+        map.clear();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 1);
+
+        cache.putAll(map);
+
+        checkData(map, null, cache, 4);
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTx() throws Exception {
+        pessimisticTx(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTxNearEnabled() throws Exception {
+        pessimisticTx(new NearCacheConfiguration());
+    }
+
+    /**
+     * @param nearCfg Near cache configuration.
+     * @throws Exception If failed.
+     */
+    private void pessimisticTx(NearCacheConfiguration nearCfg) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setNearConfiguration(nearCfg);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        final Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
+
+        spi.record(GridNearLockRequest.class);
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    cache.putAll(map);
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        assertFalse(putFut.isDone());
+
+        client = false;
+
+        IgniteEx ignite3 = startGrid(3);
+
+        log.info("Stop block1.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        spi.record(null);
+
+        checkData(map, null, cache, 4);
+
+        List<Object> msgs = spi.recordedMessages();
+
+        assertTrue(((GridNearLockRequest)msgs.get(0)).firstClientRequest());
+        assertTrue(((GridNearLockRequest)msgs.get(1)).firstClientRequest());
+
+        for (int i = 2; i < msgs.size(); i++)
+            assertFalse(((GridNearLockRequest)msgs.get(i)).firstClientRequest());
+
+        ignite3.close();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 1);
+
+        spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
+
+        putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    for (Map.Entry<Integer, Integer> e : map.entrySet())
+                        cache.put(e.getKey(), e.getValue());
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        ignite3 = startGrid(3);
+
+        log.info("Stop block2.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        checkData(map, null, cache, 4);
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 2);
+
+        try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        checkData(map, null, cache, 4);
+    }
+
+    /**
+     * Tests specific scenario when mapping for first locked keys does not change, but changes for second one.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTx2() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+        IgniteEx ignite2 = startGrid(2);
+
+        client = true;
+
+        final Ignite ignite3 = startGrid(3);
+
+        assertTrue(ignite3.configuration().isClientMode());
+
+        AffinityTopologyVersion topVer1 = new AffinityTopologyVersion(4, 0);
+
+        assertEquals(topVer1, ignite0.context().cache().internalCache(null).context().topology().topologyVersion());
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        final Integer key1 = 0;
+        final Integer key2 = 7;
+
+        spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite2.localNode().id());
+
+        final IgniteCache<Integer, Integer> cache = ignite3.cache(null);
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    cache.put(key1, 1);
+                    cache.put(key2, 2);
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        client = false;
+
+        IgniteEx ignite4 = startGrid(4);
+
+        AffinityTopologyVersion topVer2 = new AffinityTopologyVersion(5, 0);
+
+        assertEquals(topVer2, ignite0.context().cache().internalCache(null).context().topology().topologyVersion());
+
+        GridCacheAffinityManager aff = ignite0.context().cache().internalCache(null).context().affinity();
+
+        List<ClusterNode> nodes1 = aff.nodes(key1, topVer1);
+        List<ClusterNode> nodes2 = aff.nodes(key1, topVer2);
+
+        assertEquals(nodes1, nodes2);
+
+        nodes1 = aff.nodes(key2, topVer1);
+        nodes2 = aff.nodes(key2, topVer2);
+
+        assertFalse(nodes1.get(0).equals(nodes2.get(0)));
+
+        assertFalse(putFut.isDone());
+
+        log.info("Stop block.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        checkData(F.asMap(key1, 1, key2, 2), null, cache, 5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTxNearEnabledNoRemap() throws Exception {
+        pessimisticTxNoRemap(new NearCacheConfiguration());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTxNoRemap() throws Exception {
+        pessimisticTxNoRemap(null);
+    }
+
+    /**
+     * @param nearCfg Near cache configuration.
+     * @throws Exception If failed.
+     */
+    private void pessimisticTxNoRemap(@Nullable NearCacheConfiguration nearCfg) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setNearConfiguration(nearCfg);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+        IgniteEx ignite2 = startGrid(2);
+
+        client = true;
+
+        final Ignite ignite3 = startGrid(3);
+
+        assertTrue(ignite3.configuration().isClientMode());
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        final Map<Integer, Integer> map = new HashMap<>();
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i);
+
+        spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite2.localNode().id());
+
+        spi.record(GridNearLockRequest.class);
+
+        final IgniteCache<Integer, Integer> cache = ignite3.cache(null);
+
+        IgniteInternalFuture<?> putFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                    for (Map.Entry<Integer, Integer> e : map.entrySet())
+                        cache.put(e.getKey(), e.getValue());
+
+                    tx.commit();
+                }
+
+                return null;
+            }
+        });
+
+        IgniteEx ignite4 = startGrid(4);
+
+        assertTrue(ignite4.configuration().isClientMode());
+
+        assertFalse(putFut.isDone());
+
+        log.info("Stop block.");
+
+        spi.stopBlock();
+
+        putFut.get();
+
+        spi.record(null);
+
+        checkData(map, null, cache, 5);
+
+        List<Object> msgs = spi.recordedMessages();
+
+        checkClientLockMessages(msgs, map.size());
+
+        for (int i = 0; i < 100; i++)
+            map.put(i, i + 1);
+
+        try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        checkData(map, null, cache, 5);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLock() throws Exception {
+        lock(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockNearEnabled() throws Exception {
+        lock(new NearCacheConfiguration());
+    }
+
+    /**
+     * @param nearCfg Near cache configuration.
+     * @throws Exception If failed.
+     */
+    private void lock(NearCacheConfiguration nearCfg) throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+        ccfg.setNearConfiguration(nearCfg);
+
+        final IgniteEx ignite0 = startGrid(0);
+        final IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        final Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        final List<Integer> keys = new ArrayList<>();
+
+        for (int i = 0; i < 100; i++)
+            keys.add(i);
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi.blockMessages(GridNearLockRequest.class, ignite0.localNode().id());
+        spi.blockMessages(GridNearLockRequest.class, ignite1.localNode().id());
+
+        final IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        final CountDownLatch lockedLatch = new CountDownLatch(1);
+
+        final CountDownLatch unlockLatch = new CountDownLatch(1);
+
+        IgniteInternalFuture<Lock> lockFut = GridTestUtils.runAsync(new Callable<Lock>() {
+            @Override public Lock call() throws Exception {
+                Thread.currentThread().setName("put-thread");
+
+                Lock lock = cache.lockAll(keys);
+
+                lock.lock();
+
+                log.info("Locked");
+
+                lockedLatch.countDown();
+
+                unlockLatch.await();
+
+                lock.unlock();
+
+                return lock;
+            }
+        });
+
+        client = false;
+
+        IgniteEx ignite3 = startGrid(3);
+
+        log.info("Stop block.");
+
+        assertEquals(1, lockedLatch.getCount());
+
+        spi.stopBlock();
+
+        assertTrue(lockedLatch.await(3000, TimeUnit.MILLISECONDS));
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.cache(null);
+
+        for (Integer key : keys) {
+            Lock lock = cache0.lock(key);
+
+            assertFalse(lock.tryLock());
+        }
+
+        unlockLatch.countDown();
+
+        lockFut.get();
+
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return unlocked(ignite0) && unlocked(ignite1);
+            }
+
+            private boolean unlocked(Ignite ignite) {
+                IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+                for (Integer key : keys) {
+                    if (cache.isLocalLocked(key, false)) {
+                        log.info("Key is locked [key=" + key + ", node=" + ignite.name() + ']');
+
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        for (Integer key : keys) {
+            Lock lock = cache0.lock(key);
+
+            assertTrue("Failed to lock: " + key, lock.tryLock());
+
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTxMessageClientFirstFlag() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+        IgniteEx ignite2 = startGrid(2);
+
+        client = true;
+
+        Ignite ignite3 = startGrid(3);
+
+        assertTrue(ignite3.configuration().isClientMode());
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        spi.record(GridNearLockRequest.class);
+
+        IgniteCache<Integer, Integer> cache = ignite3.cache(null);
+
+        try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.put(1, 1);
+            cache.put(2, 2);
+            cache.put(3, 3);
+
+            tx.commit();
+        }
+
+        checkClientLockMessages(spi.recordedMessages(), 3);
+
+        Map<Integer, Integer> map = new HashMap<>();
+
+        map.put(4, 4);
+        map.put(5, 5);
+        map.put(6, 6);
+        map.put(7, 7);
+
+        try (Transaction tx = ignite3.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.putAll(map);
+
+            tx.commit();
+        }
+
+        checkClientLockMessages(spi.recordedMessages(), 4);
+
+        spi.record(null);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+
+        spi0.record(GridNearLockRequest.class);
+
+        List<Integer> keys = primaryKeys(ignite1.cache(null), 3, 0);
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.cache(null);
+
+        try (Transaction tx = ignite0.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache0.put(keys.get(0), 0);
+            cache0.put(keys.get(1), 1);
+            cache0.put(keys.get(2), 2);
+
+            tx.commit();
+        }
+
+        List<Object> msgs = spi0.recordedMessages();
+
+        assertEquals(3, msgs.size());
+
+        for (Object msg : msgs)
+            assertFalse(((GridNearLockRequest)msg).firstClientRequest());
+    }
+
+    /**
+     * @param msgs Messages.
+     * @param expCnt Expected number of messages.
+     */
+    private void checkClientLockMessages(List<Object> msgs, int expCnt) {
+        assertEquals(expCnt, msgs.size());
+
+        assertTrue(((GridNearLockRequest)msgs.get(0)).firstClientRequest());
+
+        for (int i = 1; i < msgs.size(); i++)
+            assertFalse(((GridNearLockRequest)msgs.get(i)).firstClientRequest());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOptimisticTxMessageClientFirstFlag() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+        IgniteEx ignite2 = startGrid(2);
+
+        client = true;
+
+        Ignite ignite3 = startGrid(3);
+
+        assertTrue(ignite3.configuration().isClientMode());
+
+        TestCommunicationSpi spi = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        IgniteCache<Integer, Integer> cache = ignite3.cache(null);
+
+        List<Integer> keys0 = primaryKeys(ignite0.cache(null), 2, 0);
+        List<Integer> keys1 = primaryKeys(ignite1.cache(null), 2, 0);
+        List<Integer> keys2 = primaryKeys(ignite2.cache(null), 2, 0);
+
+        LinkedHashMap<Integer, Integer> map = new LinkedHashMap<>();
+
+        map.put(keys0.get(0), 1);
+        map.put(keys1.get(0), 2);
+        map.put(keys2.get(0), 3);
+        map.put(keys0.get(1), 4);
+        map.put(keys1.get(1), 5);
+        map.put(keys2.get(1), 6);
+
+        spi.record(GridNearTxPrepareRequest.class);
+
+        try (Transaction tx = ignite3.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
+            for (Map.Entry<Integer, Integer> e : map.entrySet())
+                cache.put(e.getKey(), e.getValue());
+
+            tx.commit();
+        }
+
+        checkClientPrepareMessages(spi.recordedMessages(), 6);
+
+        checkData(map, null, cache, 4);
+
+        cache.putAll(map);
+
+        checkClientPrepareMessages(spi.recordedMessages(), 6);
+
+        spi.record(null);
+
+        checkData(map, null, cache, 4);
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.cache(null);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+
+        spi0.record(GridNearTxPrepareRequest.class);
+
+        cache0.putAll(map);
+
+        spi0.record(null);
+
+        List<Object> msgs = spi0.recordedMessages();
+
+        assertEquals(4, msgs.size());
+
+        for (Object msg : msgs)
+            assertFalse(((GridNearTxPrepareRequest)msg).firstClientRequest());
+
+        checkData(map, null, cache, 4);
+    }
+
+    /**
+     * @param msgs Messages.
+     * @param expCnt Expected number of messages.
+     */
+    private void checkClientPrepareMessages(List<Object> msgs, int expCnt) {
+        assertEquals(expCnt, msgs.size());
+
+        assertTrue(((GridNearTxPrepareRequest)msgs.get(0)).firstClientRequest());
+
+        for (int i = 1; i < msgs.size(); i++)
+            assertFalse(((GridNearTxPrepareRequest) msgs.get(i)).firstClientRequest());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRemoveAfterClientFailed() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        client = true;
+
+        Ignite ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        IgniteCache<Integer, Integer> cache2 = ignite2.cache(null);
+
+        final Integer key = 0;
+
+        Lock lock2 = cache2.lock(key);
+
+        lock2.lock();
+
+        ignite2.close();
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.cache(null);
+
+        assertFalse(cache0.isLocalLocked(key, false));
+
+        IgniteCache<Integer, Integer> cache1 = ignite1.cache(null);
+
+        assertFalse(cache1.isLocalLocked(key, false));
+
+        Lock lock1 = cache1.lock(0);
+
+        assertTrue(lock1.tryLock(5000, TimeUnit.MILLISECONDS));
+
+        lock1.unlock();
+
+        ignite2 = startGrid(2);
+
+        assertTrue(ignite2.configuration().isClientMode());
+
+        cache2 = ignite2.cache(null);
+
+        lock2 = cache2.lock(0);
+
+        assertTrue(lock2.tryLock(5000, TimeUnit.MILLISECONDS));
+
+        lock2.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockFromClientBlocksExchange() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        startGrid(0);
+        startGrid(1);
+
+        client = true;
+
+        Ignite ignite2 = startGrid(2);
+
+        IgniteCache<Integer, Integer> cache = ignite2.cache(null);
+
+        Lock lock = cache.lock(0);
+
+        lock.lock();
+
+        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                client = false;
+
+                startGrid(3);
+
+                return null;
+            }
+        });
+
+        U.sleep(2000);
+
+        assertFalse(startFut.isDone());
+
+        AffinityTopologyVersion ver = new AffinityTopologyVersion(4);
+
+        List<IgniteInternalFuture<?>> futs = new ArrayList<>();
+
+        U.sleep(2000);
+
+        for (int i = 0; i < 3; i++) {
+            Ignite ignite = ignite(i);
+
+            IgniteInternalFuture<?> fut =
+                ((IgniteKernal)ignite).context().cache().context().exchange().affinityReadyFuture(ver);
+
+            assertNotNull(fut);
+
+            assertFalse(fut.isDone());
+
+            futs.add(fut);
+        }
+
+        lock.unlock();
+
+        for (IgniteInternalFuture<?> fut : futs)
+            fut.get(10_000);
+
+        startFut.get(10_000);
+    }
+
+    /**
+     * @param map Expected data.
+     * @param keys Expected keys (if expected data is not specified).
+     * @param clientCache Client cache.
+     * @param expNodes Expected nodes number.
+     * @throws Exception If failed.
+     */
+    private void checkData(final Map<Integer, Integer> map,
+        final Set<Integer> keys,
+        IgniteCache<?, ?> clientCache,
+        final int expNodes)
+        throws Exception
+    {
+        final List<Ignite> nodes = G.allGrids();
+
+        final Affinity<Integer> aff = nodes.get(0).affinity(null);
+
+        assertEquals(expNodes, nodes.size());
+
+        boolean hasNearCache = clientCache.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null;
+
+        final Ignite nearCacheNode = hasNearCache ? clientCache.unwrap(Ignite.class) : null;
+
+        boolean wait = GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                try {
+                    Set<Integer> keys0 = map != null ? map.keySet() : keys;
+
+                    assertNotNull(keys0);
+
+                    for (Integer key : keys0) {
+                        GridCacheVersion ver = null;
+                        Object val = null;
+
+                        for (Ignite node : nodes) {
+                            IgniteCache<Integer, Integer> cache = node.cache(null);
+
+                            boolean affNode = aff.isPrimaryOrBackup(node.cluster().localNode(), key);
+
+                            Object val0 = cache.localPeek(key);
+
+                            if (affNode || node == nearCacheNode) {
+                                if (map != null)
+                                    assertEquals("Unexpected value for " + node.name(), map.get(key), val0);
+                                else
+                                    assertNotNull("Unexpected value for " + node.name(), val0);
+
+                                GridCacheAdapter cache0 = ((IgniteKernal)node).internalCache(null);
+
+                                if (affNode && cache0.isNear())
+                                    cache0 = ((GridNearCacheAdapter)cache0).dht();
+
+                                GridCacheEntryEx entry = cache0.peekEx(key);
+
+                                assertNotNull("No entry [node=" + node.name() + ", key=" + key + ']', entry);
+
+                                GridCacheVersion ver0 = entry instanceof GridNearCacheEntry ?
+                                    ((GridNearCacheEntry)entry).dhtVersion() : entry.version();
+
+                                assertNotNull("Null version [node=" + node.name() + ", key=" + key + ']', ver0);
+
+                                if (ver == null) {
+                                    ver = ver0;
+                                    val = val0;
+                                }
+                                else {
+                                    assertEquals("Version check failed [node=" + node.name() +
+                                        ", key=" + key +
+                                        ", affNode=" + affNode +
+                                        ", primary=" + aff.isPrimary(node.cluster().localNode(), key) + ']',
+                                        ver0,
+                                        ver);
+
+                                    assertEquals("Value check failed [node=" + node.name() +
+                                        ", key=" + key +
+                                        ", affNode=" + affNode +
+                                        ", primary=" + aff.isPrimary(node.cluster().localNode(), key) + ']',
+                                        val0,
+                                        val);
+                                }
+                            }
+                            else
+                                assertNull("Unexpected non-null value for " + node.name(), val0);
+                        }
+                    }
+                }
+                catch (AssertionError e) {
+                    log.info("Check failed, will retry: " + e);
+
+                    return false;
+                }
+                catch (Exception e) {
+                    fail("Unexpected exception: " + e);
+                }
+
+                return true;
+            }
+        }, 10_000);
+
+        assertTrue("Data check failed.", wait);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicPrimaryPutAllMultinode() throws Exception {
+        multinode(PRIMARY, TestType.PUT_ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicClockPutAllMultinode() throws Exception {
+        multinode(CLOCK, TestType.PUT_ALL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOptimisticTxPutAllMultinode() throws Exception {
+        multinode(null, TestType.OPTIMISTIC_TX);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPessimisticTxPutAllMultinode() throws Exception {
+        multinode(null, TestType.PESSIMISTIC_TX);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockAllMultinode() throws Exception {
+        multinode(null, TestType.LOCK);
+    }
+
+    /**
+     * @param atomicWriteOrder Write order if test atomic cache.
+     * @param testType Test type.
+     * @throws Exception If failed.
+     */
+    private void multinode(final CacheAtomicWriteOrderMode atomicWriteOrder, final TestType testType)
+        throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(atomicWriteOrder != null ? ATOMIC : TRANSACTIONAL);
+        ccfg.setAtomicWriteOrderMode(atomicWriteOrder);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        final int SRV_CNT = 4;
+
+        for (int i = 0; i < SRV_CNT; i++)
+            startGrid(i);
+
+        final int CLIENT_CNT = 4;
+
+        final List<Ignite> clients = new ArrayList<>();
+
+        client = true;
+
+        for (int i = 0; i < CLIENT_CNT; i++) {
+            Ignite ignite = startGrid(SRV_CNT + i);
+
+            assertTrue(ignite.configuration().isClientMode());
+
+            clients.add(ignite);
+        }
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        final AtomicInteger threadIdx = new AtomicInteger(0);
+
+        final int THREADS = CLIENT_CNT * 3;
+
+        final ConcurrentHashSet<Integer> putKeys = new ConcurrentHashSet<>();
+
+        IgniteInternalFuture<?> fut;
+
+        try {
+            fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    int clientIdx = threadIdx.getAndIncrement() % CLIENT_CNT;
+
+                    Ignite ignite = clients.get(clientIdx);
+
+                    assertTrue(ignite.configuration().isClientMode());
+
+                    Thread.currentThread().setName("update-thread-" + ignite.name());
+
+                    IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+                    boolean useTx = testType == TestType.OPTIMISTIC_TX || testType == TestType.PESSIMISTIC_TX;
+
+                    if (useTx || testType == TestType.LOCK) {
+                        assertEquals(TRANSACTIONAL,
+                            cache.getConfiguration(CacheConfiguration.class).getAtomicityMode());
+                    }
+
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    int cntr = 0;
+
+                    while (!stop.get()) {
+                        TreeMap<Integer, Integer> map = new TreeMap<>();
+
+                        for (int i = 0; i < 100; i++) {
+                            Integer key = rnd.nextInt(0, 1000);
+
+                            map.put(key, rnd.nextInt());
+                        }
+
+                        try {
+                            if (testType == TestType.LOCK) {
+                                Lock lock = cache.lockAll(map.keySet());
+
+                                lock.lock();
+
+                                lock.unlock();
+                            }
+                            else {
+                                if (useTx) {
+                                    IgniteTransactions txs = ignite.transactions();
+
+                                    TransactionConcurrency concurrency =
+                                        testType == TestType.PESSIMISTIC_TX ? PESSIMISTIC : OPTIMISTIC;
+
+                                    try (Transaction tx = txs.txStart(concurrency, REPEATABLE_READ)) {
+                                        cache.putAll(map);
+
+                                        tx.commit();
+                                    }
+                                }
+                                else
+                                    cache.putAll(map);
+
+                                putKeys.addAll(map.keySet());
+                            }
+                        }
+                        catch (CacheException | IgniteException e) {
+                            log.info("Operation failed, ignore: " + e);
+                        }
+
+                        if (++cntr % 100 == 0)
+                            log.info("Iteration: " + cntr);
+
+                        if (updateBarrier != null)
+                            updateBarrier.await();
+                    }
+
+                    return null;
+                }
+            }, THREADS, "update-thread");
+
+            long stopTime = System.currentTimeMillis() + 60_000;
+
+            while (System.currentTimeMillis() < stopTime) {
+                boolean restartClient = ThreadLocalRandom.current().nextBoolean();
+
+                Integer idx = null;
+
+                if (restartClient) {
+                    log.info("Start client node.");
+
+                    client = true;
+
+                    IgniteEx ignite = startGrid(SRV_CNT + CLIENT_CNT);
+
+                    IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+                    assertNotNull(cache);
+                }
+                else {
+                    idx = ThreadLocalRandom.current().nextInt(0, SRV_CNT);
+
+                    log.info("Stop server node: " + idx);
+
+                    stopGrid(idx);
+                }
+
+                updateBarrier = new CyclicBarrier(THREADS + 1, new Runnable() {
+                    @Override public void run() {
+                        updateBarrier = null;
+                    }
+                });
+
+                try {
+                    updateBarrier.await(30_000, TimeUnit.MILLISECONDS);
+                }
+                catch (TimeoutException e) {
+                    log.error("Failed to wait for update.");
+
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
+                    U.dumpThreads(log);
+
+                    CyclicBarrier barrier0 = updateBarrier;
+
+                    if (barrier0 != null)
+                        barrier0.reset();
+
+                    fail("Failed to wait for update.");
+                }
+
+                U.sleep(500);
+
+                if (restartClient) {
+                    log.info("Stop client node.");
+
+                    stopGrid(SRV_CNT + CLIENT_CNT);
+                }
+                else {
+                    log.info("Start server node: " + idx);
+
+                    client = false;
+
+                    startGrid(idx);
+                }
+
+                updateBarrier = new CyclicBarrier(THREADS + 1, new Runnable() {
+                    @Override public void run() {
+                        updateBarrier = null;
+                    }
+                });
+
+                try {
+                    updateBarrier.await(30_000, TimeUnit.MILLISECONDS);
+                }
+                catch (TimeoutException e) {
+                    log.error("Failed to wait for update.");
+
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
+                    U.dumpThreads(log);
+
+                    CyclicBarrier barrier0 = updateBarrier;
+
+                    if (barrier0 != null)
+                        barrier0.reset();
+
+                    fail("Failed to wait for update.");
+                }
+
+                U.sleep(500);
+            }
+        }
+        finally {
+            stop.set(true);
+        }
+
+        fut.get(30_000);
+
+        if (testType != TestType.LOCK)
+            checkData(null, putKeys, grid(SRV_CNT).cache(null), SRV_CNT + CLIENT_CNT);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServersLeaveOnStart() throws Exception {
+        ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(SYNC);
+
+        Ignite ignite0 = startGrid(0);
+
+        client = true;
+
+        final AtomicInteger nodeIdx = new AtomicInteger(2);
+
+        final int CLIENTS = 10;
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                int idx = nodeIdx.getAndIncrement();
+
+                startGrid(idx);
+
+                return null;
+            }
+        }, CLIENTS, "start-client");
+
+        ignite0.close();
+
+        fut.get();
+
+        for (int i = 0; i < CLIENTS; i++) {
+            Ignite ignite = grid(i + 2);
+
+            assertEquals(CLIENTS, ignite.cluster().nodes().size());
+        }
+
+        client = false;
+
+        startGrid(0);
+        startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        for (int i = 0; i < CLIENTS; i++) {
+            Ignite ignite = grid(i + 2);
+
+            IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+            cache.put(i, i);
+
+            assertEquals((Object)i, cache.get(i));
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** */
+        private List<T2<ClusterNode, GridIoMessage>> blockedMsgs = new ArrayList<>();
+
+        /** */
+        private Map<Class<?>, Set<UUID>> blockCls = new HashMap<>();
+
+        /** */
+        private Class<?> recordCls;
+
+        /** */
+        private List<Object> recordedMsgs = new ArrayList<>();
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                Object msg0 = ((GridIoMessage)msg).message();
+
+                synchronized (this) {
+                    if (recordCls != null && msg0.getClass().equals(recordCls))
+                        recordedMsgs.add(msg0);
+
+                    Set<UUID> blockNodes = blockCls.get(msg0.getClass());
+
+                    if (F.contains(blockNodes, node.id())) {
+                        log.info("Block message [node=" + node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME) +
+                            ", msg=" + msg0 + ']');
+
+                        blockedMsgs.add(new T2<>(node, (GridIoMessage)msg));
+
+                        return;
+                    }
+                }
+            }
+
+            super.sendMessage(node, msg);
+        }
+
+        /**
+         * @param recordCls Message class to record.
+         */
+        void record(@Nullable Class<?> recordCls) {
+            synchronized (this) {
+                this.recordCls = recordCls;
+            }
+        }
+
+        /**
+         * @return Recorded messages.
+         */
+        List<Object> recordedMessages() {
+            synchronized (this) {
+                List<Object> msgs = recordedMsgs;
+
+                recordedMsgs = new ArrayList<>();
+
+                return msgs;
+            }
+        }
+
+        /**
+         * @param cls Message class.
+         * @param nodeId Node ID.
+         */
+        void blockMessages(Class<?> cls, UUID nodeId) {
+            synchronized (this) {
+                Set<UUID> set = blockCls.get(cls);
+
+                if (set == null) {
+                    set = new HashSet<>();
+
+                    blockCls.put(cls, set);
+                }
+
+                set.add(nodeId);
+            }
+        }
+
+        /**
+         *
+         */
+        void stopBlock() {
+            synchronized (this) {
+                blockCls.clear();
+
+                for (T2<ClusterNode, GridIoMessage> msg : blockedMsgs) {
+                    ClusterNode node = msg.get1();
+
+                    log.info("Send blocked message: [node=" + node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME) +
+                        ", msg=" + msg.get2().message() + ']');
+
+                    super.sendMessage(msg.get1(), msg.get2());
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    enum TestType {
+        /** */
+        PUT_ALL,
+
+        /** */
+        OPTIMISTIC_TX,
+
+        /** */
+        PESSIMISTIC_TX,
+
+        /** */
+        LOCK
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
new file mode 100644
index 0000000..bd74ece
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
@@ -0,0 +1,105 @@
+/*
+ * 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.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodeConcurrentStart extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES_CNT = 5;
+
+    /** */
+    private Set<Integer> clientNodes;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        assertNotNull(clientNodes);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        boolean client = false;
+
+        for (Integer clientIdx : clientNodes) {
+            if (getTestGridName(clientIdx).equals(gridName)) {
+                client = true;
+
+                break;
+            }
+        }
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setBackups(0);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStart() throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 3; i++) {
+            try {
+                clientNodes = new HashSet<>();
+
+                while (clientNodes.size() < 2)
+                    clientNodes.add(rnd.nextInt(0, NODES_CNT));
+
+                clientNodes.add(NODES_CNT - 1);
+
+                log.info("Test iteration [iter=" + i + ", clients=" + clientNodes + ']');
+
+                startGridsMultiThreaded(NODES_CNT, true);
+
+                for (int node : clientNodes) {
+                    Ignite ignite = grid(node);
+
+                    assertTrue(ignite.configuration().isClientMode());
+                }
+            }
+            finally {
+                stopAllGrids();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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
new file mode 100644
index 0000000..e5d30b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -0,0 +1,632 @@
+/*
+ * 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.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.resources.*;
+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 java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean fairAffinity;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        if (fairAffinity)
+            ccfg.setAffinity(new FairAffinityFunction());
+
+        cfg.setCacheConfiguration(ccfg);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerNodeLeave() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        client = true;
+
+        final Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        final Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        ignite0.close();
+
+        waitForTopologyUpdate(2, 4);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite1.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite2.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+
+        ignite1.close();
+
+        waitForTopologyUpdate(1, 5);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ignite2.cache(null).get(1);
+
+                return null;
+            }
+        }, CacheServerNotFoundException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSkipPreload() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        final CountDownLatch evtLatch0 = new CountDownLatch(1);
+
+        ignite0.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                log.info("Rebalance event: " + evt);
+
+                evtLatch0.countDown();
+
+                return true;
+            }
+        }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
+
+        client = true;
+
+        Ignite ignite1 = startGrid(1);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        ignite1.close();
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        ignite1 = startGrid(1);
+
+        final CountDownLatch evtLatch1 = new CountDownLatch(1);
+
+        ignite1.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                log.info("Rebalance event: " + evt);
+
+                evtLatch1.countDown();
+
+                return true;
+            }
+        }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+
+        client = false;
+
+        startGrid(2);
+
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertFalse(evtLatch1.await(1000, TimeUnit.MILLISECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionsExchange() throws Exception {
+        partitionsExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionsExchangeFairAffinity() throws Exception {
+        fairAffinity = true;
+
+        partitionsExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void partitionsExchange() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+
+        Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        TestCommunicationSpi spi1 = (TestCommunicationSpi)ignite1.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+
+        client = true;
+
+        log.info("Start client node1.");
+
+        Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        TestCommunicationSpi spi2 = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        log.info("Start client node2.");
+
+        Ignite ignite3 = startGrid(3);
+
+        waitForTopologyUpdate(4, 4);
+
+        TestCommunicationSpi spi3 = (TestCommunicationSpi)ignite3.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(0, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+        spi3.reset();
+
+        log.info("Start one more server node.");
+
+        client = false;
+
+        Ignite ignite4 = startGrid(4);
+
+        waitForTopologyUpdate(5, 5);
+
+        TestCommunicationSpi spi4 = (TestCommunicationSpi)ignite4.configuration().getCommunicationSpi();
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(4, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        assertEquals(1, spi4.partitionsSingleMessages());
+        assertEquals(0, spi4.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+        spi3.reset();
+
+        log.info("Stop server node.");
+
+        ignite4.close();
+
+        waitForTopologyUpdate(4, 6);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(3, spi0.partitionsFullMessages());
+
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        assertEquals(1, spi3.partitionsSingleMessages());
+        assertEquals(0, spi3.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        log.info("Stop client node2.");
+
+        ignite3.close();
+
+        waitForTopologyUpdate(3, 7);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        assertEquals(0, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        spi0.reset();
+        spi1.reset();
+
+        log.info("Stop client node1.");
+
+        ignite2.close();
+
+        waitForTopologyUpdate(2, 8);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+
+        log.info("Stop server node.");
+
+        ignite1.close();
+
+        waitForTopologyUpdate(1, 9);
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(0, spi0.partitionsFullMessages());
+    }
+
+    /**
+     * @param expNodes Expected number of nodes.
+     * @param topVer Expected topology version.
+     * @throws Exception If failed.
+     */
+    private void waitForTopologyUpdate(int expNodes, int topVer) throws Exception {
+        final AffinityTopologyVersion ver = new AffinityTopologyVersion(topVer, 0);
+
+        waitForTopologyUpdate(expNodes, ver);
+    }
+
+    /**
+     * @param expNodes Expected number of nodes.
+     * @param topVer Expected topology version.
+     * @throws Exception If failed.
+     */
+    private void waitForTopologyUpdate(int expNodes, final AffinityTopologyVersion topVer) throws Exception {
+        List<Ignite> nodes = G.allGrids();
+
+        assertEquals(expNodes, nodes.size());
+
+        for (Ignite ignite : nodes) {
+            final IgniteKernal kernal = (IgniteKernal)ignite;
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return topVer.equals(kernal.context().cache().context().exchange().readyAffinityVersion());
+                }
+            }, 10_000);
+
+            assertEquals("Unexpected affinity version for " + ignite.name(),
+                topVer,
+                kernal.context().cache().context().exchange().readyAffinityVersion());
+        }
+
+        Iterator<Ignite> it = nodes.iterator();
+
+        Ignite ignite0 = it.next();
+
+        Affinity<Integer> aff0 = ignite0.affinity(null);
+
+        while (it.hasNext()) {
+            Ignite ignite = it.next();
+
+            Affinity<Integer> aff = ignite.affinity(null);
+
+            assertEquals(aff0.partitions(), aff.partitions());
+
+            for (int part = 0; part < aff.partitions(); part++)
+                assertEquals(aff0.mapPartitionToPrimaryAndBackups(part), aff.mapPartitionToPrimaryAndBackups(part));
+        }
+
+        for (Ignite ignite : nodes) {
+            final IgniteKernal kernal = (IgniteKernal)ignite;
+
+            for (IgniteInternalCache cache : kernal.context().cache().caches()) {
+                GridDhtPartitionTopology top = cache.context().topology();
+
+                assertEquals("Unexpected topology version [node=" + ignite.name() + ", cache=" + cache.name() + ']',
+                    topVer,
+                    top.topologyVersion());
+            }
+        }
+
+        awaitPartitionMapExchange();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientOnlyCacheStart() throws Exception {
+        clientOnlyCacheStart(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearOnlyCacheStart() throws Exception {
+        clientOnlyCacheStart(true, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientOnlyCacheStartFromServerNode() throws Exception {
+        clientOnlyCacheStart(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNearOnlyCacheStartFromServerNode() throws Exception {
+        clientOnlyCacheStart(true, true);
+    }
+
+    /**
+     * @param nearCache If {@code true} creates near cache on client.
+     * @throws Exception If failed.
+     */
+    private void clientOnlyCacheStart(boolean nearCache, boolean srvNode) throws Exception {
+        Ignite ignite0 = startGrid(0);
+        Ignite ignite1 = startGrid(1);
+
+        waitForTopologyUpdate(2, 2);
+
+        final String CACHE_NAME1 = "cache1";
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(CACHE_NAME1);
+
+        if (srvNode)
+            ccfg.setNodeFilter(new TestFilter(getTestGridName(2)));
+
+        ignite0.createCache(ccfg);
+
+        client = !srvNode;
+
+        Ignite ignite2 = startGrid(2);
+
+        waitForTopologyUpdate(3, 3);
+
+        TestCommunicationSpi spi0 = (TestCommunicationSpi)ignite0.configuration().getCommunicationSpi();
+        TestCommunicationSpi spi1 = (TestCommunicationSpi)ignite1.configuration().getCommunicationSpi();
+        TestCommunicationSpi spi2 = (TestCommunicationSpi)ignite2.configuration().getCommunicationSpi();
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        assertNull(((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1"));
+
+        if (nearCache)
+            ignite2.getOrCreateNearCache(CACHE_NAME1, new NearCacheConfiguration<>());
+        else
+            ignite2.cache(CACHE_NAME1);
+
+        waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 1));
+
+        GridCacheAdapter cache = ((IgniteKernal)ignite2).context().cache().context().cache().internalCache("cache1");
+
+        assertNotNull(cache);
+        assertEquals(nearCache, cache.context().isNear());
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(1, spi0.partitionsFullMessages());
+        assertEquals(0, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+
+        ClusterNode clientNode = ((IgniteKernal)ignite2).localNode();
+
+        for (Ignite ignite : Ignition.allGrids()) {
+            GridDiscoveryManager disco = ((IgniteKernal)ignite).context().discovery();
+
+            assertTrue(disco.cacheNode(clientNode, CACHE_NAME1));
+            assertFalse(disco.cacheAffinityNode(clientNode, CACHE_NAME1));
+            assertEquals(nearCache, disco.cacheNearNode(clientNode, CACHE_NAME1));
+        }
+
+        spi0.reset();
+        spi1.reset();
+        spi2.reset();
+
+        final String CACHE_NAME2 = "cache2";
+
+        ccfg = new CacheConfiguration();
+
+        ccfg.setName(CACHE_NAME2);
+
+        ignite2.createCache(ccfg);
+
+        waitForTopologyUpdate(3, new AffinityTopologyVersion(3, 2));
+
+        assertEquals(0, spi0.partitionsSingleMessages());
+        assertEquals(2, spi0.partitionsFullMessages());
+        assertEquals(1, spi1.partitionsSingleMessages());
+        assertEquals(0, spi1.partitionsFullMessages());
+        assertEquals(1, spi2.partitionsSingleMessages());
+        assertEquals(0, spi2.partitionsFullMessages());
+    }
+
+    /**
+     *
+     */
+    private static class TestFilter implements IgnitePredicate<ClusterNode> {
+        /** */
+        private String exclNodeName;
+
+        /**
+         * @param exclNodeName Node name to exclude.
+         */
+        public TestFilter(String exclNodeName) {
+            this.exclNodeName = exclNodeName;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode clusterNode) {
+            return !exclNodeName.equals(clusterNode.attribute(IgniteNodeAttributes.ATTR_GRID_NAME));
+        }
+    }
+
+    /**
+     * Test communication SPI.
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        private AtomicInteger partSingleMsgs = new AtomicInteger();
+
+        /** */
+        private AtomicInteger partFullMsgs = new AtomicInteger();
+
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) {
+            super.sendMessage(node, msg);
+
+            Object msg0 = ((GridIoMessage)msg).message();
+
+            if (msg0 instanceof GridDhtPartitionsSingleMessage) {
+                if (((GridDhtPartitionsSingleMessage)msg0).exchangeId() != null) {
+                    log.info("Partitions message: " + msg0.getClass().getSimpleName());
+
+                    partSingleMsgs.incrementAndGet();
+                }
+            }
+            else if (msg0 instanceof GridDhtPartitionsFullMessage) {
+                if (((GridDhtPartitionsFullMessage)msg0).exchangeId() != null) {
+                    log.info("Partitions message: " + msg0.getClass().getSimpleName());
+
+                    partFullMsgs.incrementAndGet();
+                }
+            }
+        }
+
+        /**
+         *
+         */
+        void reset() {
+            partSingleMsgs.set(0);
+            partFullMsgs.set(0);
+        }
+
+        /**
+         * @return Sent partitions single messages.
+         */
+        int partitionsSingleMessages() {
+            return partSingleMsgs.get();
+        }
+
+        /**
+         * @return Sent partitions full messages.
+         */
+        int partitionsFullMessages() {
+            return partFullMsgs.get();
+        }
+    }
+
+}


[24/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 0e1a9c2..871cd77 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
@@ -27,6 +27,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
@@ -47,7 +48,6 @@ import org.apache.ignite.internal.processors.plugin.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -153,7 +153,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cfg.setMemoryMode(DFLT_MEMORY_MODE);
 
         if (cfg.getNodeFilter() == null)
-            cfg.setNodeFilter(CacheConfiguration.SERVER_NODES);
+            cfg.setNodeFilter(CacheConfiguration.ALL_NODES);
 
         if (cfg.getAffinity() == null) {
             if (cfg.getCacheMode() == PARTITIONED) {
@@ -541,10 +541,10 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         maxRebalanceOrder = validatePreloadOrder(ctx.config().getCacheConfiguration());
 
-        ctx.discovery().setCustomEventListener(new GridPlainInClosure<Serializable>() {
-            @Override public void apply(Serializable evt) {
-                if (evt instanceof DynamicCacheChangeBatch)
-                    onCacheChangeRequested((DynamicCacheChangeBatch)evt);
+        ctx.discovery().setCustomEventListener(DynamicCacheChangeBatch.class,
+            new CustomEventListener<DynamicCacheChangeBatch>() {
+            @Override public void onCustomEvent(ClusterNode snd, DynamicCacheChangeBatch msg) {
+                onCacheChangeRequested(msg);
             }
         });
 
@@ -567,7 +567,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration();
 
-        sharedCtx = createSharedContext(ctx);
+        sharedCtx = createSharedContext(ctx, CU.startStoreSessionListeners(ctx,
+            ctx.config().getCacheStoreSessionListenerFactories()));
 
         ctx.performance().add("Disable serializable transactions (set 'txSerializableEnabled' to false)",
             !ctx.config().getTransactionConfiguration().isTxSerializableEnabled());
@@ -622,9 +623,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 ctx.discovery().setCacheFilter(
                     cfg.getName(),
                     cfg.getNodeFilter(),
-                    cfg.getNearConfiguration() != null,
+                    cfg.getNearConfiguration() != null && cfg.getCacheMode() == PARTITIONED,
                     cfg.getCacheMode() == LOCAL);
 
+                ctx.discovery().addClientNode(cfg.getName(),
+                    ctx.localNodeId(),
+                    cfg.getNearConfiguration() != null);
+
                 if (!cacheType.userCache())
                     stopSeq.addLast(cfg.getName());
                 else
@@ -669,6 +674,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (!getBoolean(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK)) {
                 for (ClusterNode n : ctx.discovery().remoteNodes()) {
+                    if (n.attribute(ATTR_CONSISTENCY_CHECK_SKIPPED))
+                        continue;
+
                     checkTransactionConfiguration(n);
 
                     DeploymentMode locDepMode = ctx.config().getDeploymentMode();
@@ -683,7 +691,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                         if (rmtCfg != null) {
                             CacheConfiguration locCfg = desc.cacheConfiguration();
 
-                            checkCache(locCfg, rmtCfg, n);
+                            checkCache(locCfg, rmtCfg, n, desc);
 
                             // Check plugin cache configurations.
                             CachePluginManager pluginMgr = desc.pluginManager();
@@ -706,12 +714,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                 IgnitePredicate filter = ccfg.getNodeFilter();
 
-                if (filter.apply(locNode)) {
+                boolean loc = desc.locallyConfigured();
+
+                if (loc || CU.affinityNode(locNode, filter)) {
                     CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
                     CachePluginManager pluginMgr = desc.pluginManager();
 
-                    GridCacheContext ctx = createCache(ccfg, pluginMgr, desc.cacheType(), cacheObjCtx);
+                    GridCacheContext ctx = createCache(
+                        ccfg, pluginMgr, desc.cacheType(), cacheObjCtx, desc.updatesAllowed());
 
                     ctx.dynamicDeploymentId(desc.deploymentId());
 
@@ -754,8 +765,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 int order = cfg.getRebalanceOrder();
 
                 if (order > 0 && order != maxRebalanceOrder && cfg.getCacheMode() != LOCAL) {
-                    GridCompoundFuture<Object, Object> fut = (GridCompoundFuture<Object, Object>)preloadFuts
-                        .get(order);
+                    GridCompoundFuture fut = (GridCompoundFuture)preloadFuts.get(order);
 
                     if (fut == null) {
                         fut = new GridCompoundFuture<>();
@@ -774,18 +784,31 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter<?, ?> cache : caches.values())
             onKernalStart(cache);
 
+        boolean utilityCacheStarted = false;
+
         // Wait for caches in SYNC preload mode.
-        for (GridCacheAdapter<?, ?> cache : caches.values()) {
-            CacheConfiguration cfg = cache.configuration();
+        for (CacheConfiguration cfg : ctx.config().getCacheConfiguration()) {
+            GridCacheAdapter cache = caches.get(maskNull(cfg.getName()));
+
+            if (cache != null) {
+                if (cfg.getRebalanceMode() == SYNC) {
+                    if (cfg.getCacheMode() == REPLICATED ||
+                        (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0)) {
+                        cache.preloader().syncFuture().get();
+
+                        if (CU.isUtilityCache(cache.name())) {
+                            ctx.cacheObjects().onUtilityCacheStarted();
 
-            if (cfg.getRebalanceMode() == SYNC) {
-                if (cfg.getCacheMode() == REPLICATED ||
-                    (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
-                    cache.preloader().syncFuture().get();
+                            utilityCacheStarted = true;
+                        }
+                    }
+                }
             }
         }
 
-        ctx.cacheObjects().onCacheProcessorStarted();
+        assert caches.containsKey(CU.MARSH_CACHE_NAME) : "Marshaller cache should be started";
+        assert caches.containsKey(CU.UTILITY_CACHE_NAME) : "Utility cache should be started";
+        assert utilityCacheStarted;
     }
 
     /** {@inheritDoc} */
@@ -812,6 +835,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             mgr.stop(cancel);
         }
 
+        CU.stopStoreSessionListeners(ctx, sharedCtx.storeSessionListeners());
+
         sharedCtx.cleanup();
 
         if (log.isDebugEnabled())
@@ -1047,7 +1072,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private GridCacheContext createCache(CacheConfiguration<?, ?> cfg,
         @Nullable CachePluginManager pluginMgr,
         CacheType cacheType,
-        CacheObjectContext cacheObjCtx)
+        CacheObjectContext cacheObjCtx,
+        boolean updatesAllowed)
         throws IgniteCheckedException
     {
         assert cfg != null;
@@ -1105,6 +1131,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             cfg,
             cacheType,
             ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
+            updatesAllowed,
 
             /*
              * Managers in starting order!
@@ -1234,6 +1261,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 cfg,
                 cacheType,
                 ctx.discovery().cacheAffinityNode(ctx.discovery().localNode(), cfg.getName()),
+                true,
 
                 /*
                  * Managers in starting order!
@@ -1423,7 +1451,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         ClusterNode locNode = ctx.discovery().localNode();
 
-        boolean affNodeStart = !clientStartOnly && nodeFilter.apply(locNode);
+        boolean affNodeStart = !clientStartOnly && CU.affinityNode(locNode, nodeFilter);
         boolean clientNodeStart = locNode.id().equals(initiatingNodeId);
 
         if (sharedCtx.cacheContext(CU.cacheId(cfg.getName())) != null)
@@ -1437,7 +1465,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(ccfg);
 
-            GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx);
+            GridCacheContext cacheCtx = createCache(ccfg, null, cacheType, cacheObjCtx, true);
 
             cacheCtx.startTopologyVersion(topVer);
 
@@ -1562,10 +1590,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Creates shared context.
      *
      * @param kernalCtx Kernal context.
+     * @param storeSesLsnrs Store session listeners.
      * @return Shared context.
      */
     @SuppressWarnings("unchecked")
-    private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx) {
+    private GridCacheSharedContext createSharedContext(GridKernalContext kernalCtx,
+        Collection<CacheStoreSessionListener> storeSesLsnrs) {
         IgniteTxManager tm = new IgniteTxManager();
         GridCacheMvccManager mvccMgr = new GridCacheMvccManager();
         GridCacheVersionManager verMgr = new GridCacheVersionManager();
@@ -1580,7 +1610,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             mvccMgr,
             depMgr,
             exchMgr,
-            ioMgr
+            ioMgr,
+            storeSesLsnrs
         );
     }
 
@@ -1867,7 +1898,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
                     // Check if we were asked to start a near cache.
                     if (nearCfg != null) {
-                        if (descCfg.getNodeFilter().apply(ctx.discovery().localNode())) {
+                        if (CU.affinityNode(ctx.discovery().localNode(), descCfg.getNodeFilter())) {
                             // If we are on a data node and near cache was enabled, return success, else - fail.
                             if (descCfg.getNearConfiguration() != null)
                                 return new GridFinishedFuture<>();
@@ -1914,7 +1945,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 return new GridFinishedFuture<>(new CacheExistsException("Failed to start near cache " +
                     "(a cache with the given name is not started): " + cacheName));
 
-            if (ccfg.getNodeFilter().apply(ctx.discovery().localNode())) {
+            if (CU.affinityNode(ctx.discovery().localNode(), ccfg.getNodeFilter())) {
                 if (ccfg.getNearConfiguration() != null)
                     return new GridFinishedFuture<>();
                 else
@@ -2202,11 +2233,14 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * Checks that remote caches has configuration compatible with the local.
      *
+     * @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)
-        throws IgniteCheckedException {
+    private void checkCache(CacheConfiguration locCfg, CacheConfiguration rmtCfg, ClusterNode rmtNode,
+        DynamicCacheDescriptor desc) throws IgniteCheckedException {
         ClusterNode locNode = ctx.discovery().localNode();
 
         UUID rmt = rmtNode.id();
@@ -2214,6 +2248,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         GridCacheAttributes rmtAttr = new GridCacheAttributes(rmtCfg);
         GridCacheAttributes locAttr = new GridCacheAttributes(locCfg);
 
+        boolean isLocAff = CU.affinityNode(locNode, locCfg.getNodeFilter());
+        boolean isRmtAff = CU.affinityNode(rmtNode, rmtCfg.getNodeFilter());
+
         CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cacheMode", "Cache mode",
             locAttr.cacheMode(), rmtAttr.cacheMode(), true);
 
@@ -2227,8 +2264,18 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "cachePreloadMode",
                 "Cache preload mode", locAttr.cacheRebalanceMode(), rmtAttr.cacheRebalanceMode(), true);
 
-            if (locCfg.getAtomicityMode() == TRANSACTIONAL ||
-                (rmtCfg.getNodeFilter().apply(rmtNode) && locCfg.getNodeFilter().apply(locNode)))
+            boolean checkStore;
+
+            if (!isLocAff && isRmtAff && locCfg.getAtomicityMode() == TRANSACTIONAL) {
+                checkStore = locAttr.storeFactoryClassName() != null;
+
+                if (locAttr.storeFactoryClassName() == null && rmtAttr.storeFactoryClassName() != null)
+                    desc.updatesAllowed(false);
+            }
+            else
+                checkStore = isLocAff && isRmtAff;
+
+            if (checkStore)
                 CU.checkAttributeMismatch(log, rmtAttr.cacheName(), rmt, "storeFactory", "Store factory",
                     locAttr.storeFactoryClassName(), rmtAttr.storeFactoryClassName(), true);
 
@@ -2547,7 +2594,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache instance for given name.
      * @throws IgniteCheckedException If failed.
      */
-    public <K, V> IgniteCache<K, V> publicJCache(@Nullable String cacheName) throws IgniteCheckedException {
+    public <K, V> IgniteCacheProxy<K, V> publicJCache(@Nullable String cacheName) throws IgniteCheckedException {
         return publicJCache(cacheName, true);
     }
 
@@ -2561,7 +2608,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"unchecked", "ConstantConditions"})
-    @Nullable public <K, V> IgniteCache<K, V> publicJCache(@Nullable String cacheName, boolean failIfNotStarted)
+    @Nullable public <K, V> IgniteCacheProxy<K, V> publicJCache(@Nullable String cacheName, boolean failIfNotStarted)
         throws IgniteCheckedException
     {
         if (log.isDebugEnabled())
@@ -2569,7 +2616,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         String masked = maskNull(cacheName);
 
-        IgniteCache<K,V> cache = (IgniteCache<K, V>)jCacheProxies.get(masked);
+        IgniteCacheProxy<?, ?> cache = jCacheProxies.get(masked);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
 
@@ -2579,7 +2626,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (cache == null)
            cache = startJCache(cacheName, failIfNotStarted);
 
-        return cache;
+        return (IgniteCacheProxy<K, V>)cache;
     }
 
     /**
@@ -2589,7 +2636,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Cache instance for given name.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteCache startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
+    private IgniteCacheProxy startJCache(String cacheName, boolean failIfNotStarted) throws IgniteCheckedException {
         String masked = maskNull(cacheName);
 
         DynamicCacheDescriptor desc = registeredCaches.get(masked);
@@ -2619,7 +2666,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         F.first(initiateCacheChanges(F.asList(req))).get();
 
-        IgniteCache cache = jCacheProxies.get(masked);
+        IgniteCacheProxy cache = jCacheProxies.get(masked);
 
         if (cache == null && failIfNotStarted)
             throw new IllegalArgumentException("Cache is not started: " + cacheName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 55d2f84..63ba242 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -329,7 +329,7 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException {
+    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext prev = gate.enter(opCtx);
 
         try {
@@ -341,6 +341,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.getAllOutTxAsync(keys);
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isIgfsDataCache() {
         CacheOperationContext prev = gate.enter(opCtx);
 
@@ -741,6 +753,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public Set<K> keySetx() {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.keySetx();
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
         CacheOperationContext prev = gate.enter(opCtx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 294c2b0..1071ef2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -26,6 +27,7 @@ import org.apache.ignite.internal.managers.deployment.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.cache.store.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.timeout.*;
@@ -76,6 +78,9 @@ public class GridCacheSharedContext<K, V> {
     /** Preloaders start future. */
     private IgniteInternalFuture<Object> preloadersStartFut;
 
+    /** Store session listeners. */
+    private Collection<CacheStoreSessionListener> storeSesLsnrs;
+
     /**
      * @param txMgr Transaction manager.
      * @param verMgr Version manager.
@@ -88,7 +93,8 @@ public class GridCacheSharedContext<K, V> {
         GridCacheMvccManager mvccMgr,
         GridCacheDeploymentManager<K, V> depMgr,
         GridCachePartitionExchangeManager<K, V> exchMgr,
-        GridCacheIoManager ioMgr
+        GridCacheIoManager ioMgr,
+        Collection<CacheStoreSessionListener> storeSesLsnrs
     ) {
         this.kernalCtx = kernalCtx;
         this.mvccMgr = add(mvccMgr);
@@ -97,6 +103,7 @@ public class GridCacheSharedContext<K, V> {
         this.depMgr = add(depMgr);
         this.exchMgr = add(exchMgr);
         this.ioMgr = add(ioMgr);
+        this.storeSesLsnrs = storeSesLsnrs;
 
         txMetrics = new TransactionMetricsAdapter();
 
@@ -427,27 +434,38 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to check.
      * @param activeCacheIds Active cache IDs.
      * @param cacheCtx Cache context.
-     * @return {@code True} if cross-cache transaction can include this new cache.
+     * @return Error message if transactions are incompatible.
      */
-    public boolean txCompatible(IgniteInternalTx tx, Iterable<Integer> activeCacheIds, GridCacheContext<K, V> cacheCtx) {
-        if (cacheCtx.systemTx() ^ tx.system())
-            return false;
+    @Nullable public String verifyTxCompatibility(IgniteInternalTx tx, Iterable<Integer> activeCacheIds,
+        GridCacheContext<K, V> cacheCtx) {
+        if (cacheCtx.systemTx() && !tx.system())
+            return "system cache can be enlisted only in system transaction";
+
+        if (!cacheCtx.systemTx() && tx.system())
+            return "non-system cache can't be enlisted in system transaction";
 
         for (Integer cacheId : activeCacheIds) {
             GridCacheContext<K, V> activeCacheCtx = cacheContext(cacheId);
 
-            // System transactions may sap only one cache.
             if (cacheCtx.systemTx()) {
                 if (activeCacheCtx.cacheId() != cacheCtx.cacheId())
-                    return false;
+                    return "system transaction can include only one cache";
             }
 
-            // Check that caches have the same store.
-            if (activeCacheCtx.store().store() != cacheCtx.store().store())
-                return false;
+            CacheStoreManager store = cacheCtx.store();
+            CacheStoreManager activeStore = activeCacheCtx.store();
+
+            if (store.isLocal() != activeStore.isLocal())
+                return "caches with local and non-local stores can't be enlisted in one transaction";
+
+            if (store.isWriteBehind() != activeStore.isWriteBehind())
+                return "caches with different write-behind setting can't be enlisted in one transaction";
+
+            // If local and write-behind validations passed, this must be true.
+            assert store.isWriteToStoreFromDht() == activeStore.isWriteToStoreFromDht();
         }
 
-        return true;
+        return null;
     }
 
     /**
@@ -499,6 +517,7 @@ public class GridCacheSharedContext<K, V> {
     /**
      * @param tx Transaction to rollback.
      * @throws IgniteCheckedException If failed.
+     * @return Rollback future.
      */
     public IgniteInternalFuture rollbackTxAsync(IgniteInternalTx tx) throws IgniteCheckedException {
         Collection<Integer> cacheIds = tx.activeCacheIds();
@@ -512,6 +531,13 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @return Store session listeners.
+     */
+    @Nullable public Collection<CacheStoreSessionListener> storeSessionListeners() {
+        return storeSesLsnrs;
+    }
+
+    /**
      * @param mgr Manager to add.
      * @return Added manager.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index eb82218..772e849 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -121,6 +121,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         warnFirstEvict();
 
                     writeToSwap(part, cctx.toCacheKeyObject(kb), vb);
+
+                    if (cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onOffHeapEvict();
                 }
                 catch (IgniteCheckedException e) {
                     log.error("Failed to unmarshal off-heap entry [part=" + part + ", hash=" + hash + ']', e);
@@ -395,8 +398,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Reconstituted swap entry or {@code null} if entry is obsolete.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException
-    {
+    @Nullable private <X extends GridCacheSwapEntry> X swapEntry(X e) throws IgniteCheckedException {
         assert e != null;
 
         checkIteratorQueue();
@@ -425,9 +427,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         int part = cctx.affinity().partition(key);
 
         // First check off-heap store.
-        if (offheapEnabled)
-            if (offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext())))
+        if (offheapEnabled) {
+            boolean contains = offheap.contains(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onOffHeapRead(contains);
+
+            if (contains)
                 return true;
+        }
 
         if (swapEnabled) {
             assert key != null;
@@ -436,6 +444,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
                 cctx.deploy().globalLoader());
 
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onSwapRead(valBytes != null);
+
             return valBytes != null;
         }
 
@@ -444,7 +455,6 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
     /**
      * @param key Key to read.
-     * @param keyBytes Key bytes.
      * @param part Key partition.
      * @param entryLocked {@code True} if cache entry is locked.
      * @param readOffheap Read offheap flag.
@@ -481,6 +491,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             if (readOffheap && offheapEnabled) {
                 byte[] bytes = offheap.get(spaceName, part, key, keyBytes);
 
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapRead(bytes != null);
+
                 if (bytes != null)
                     return swapEntry(unmarshalSwapEntry(bytes));
             }
@@ -524,6 +537,13 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         if (offheapEnabled) {
             byte[] entryBytes = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
 
+            if (cctx.config().isStatisticsEnabled()) {
+                if (entryBytes != null)
+                    cctx.cache().metrics0().onOffHeapRemove();
+
+                cctx.cache().metrics0().onOffHeapRead(entryBytes != null);
+            }
+
             if (entryBytes != null) {
                 GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
 
@@ -567,8 +587,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @return Value from swap or {@code null}.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key,
-        final int part)
+    @Nullable private GridCacheSwapEntry readAndRemoveSwap(final KeyCacheObject key, final int part)
         throws IgniteCheckedException {
         if (!swapEnabled)
             return null;
@@ -582,6 +601,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         swapMgr.remove(spaceName, swapKey, new CI1<byte[]>() {
             @Override public void apply(byte[] rmv) {
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onSwapRead(rmv != null);
+
                 if (rmv != null) {
                     try {
                         GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(rmv));
@@ -611,6 +633,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                                 null);
                         }
 
+                        if (cctx.config().isStatisticsEnabled())
+                            cctx.cache().metrics0().onSwapRemove();
+
                         // Always fire this event, since preloading depends on it.
                         onUnswapped(part, key, entry);
 
@@ -649,12 +674,8 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         if (!offheapEnabled && !swapEnabled)
             return null;
 
-        return read(entry.key(),
-            entry.key().valueBytes(cctx.cacheObjectContext()),
-            entry.partition(),
-            locked,
-            readOffheap,
-            readSwap);
+        return read(entry.key(), entry.key().valueBytes(cctx.cacheObjectContext()), entry.partition(), locked,
+            readOffheap, readSwap);
     }
 
     /**
@@ -730,6 +751,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         final GridCacheQueryManager qryMgr = cctx.queries();
 
         Collection<SwapKey> unprocessedKeys = null;
+
         final Collection<GridCacheBatchSwapEntry> res = new ArrayList<>(keys.size());
 
         // First try removing from offheap.
@@ -737,8 +759,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
             for (KeyCacheObject key : keys) {
                 int part = cctx.affinity().partition(key);
 
-                byte[] entryBytes =
-                    offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                byte[] entryBytes = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+                if(entryBytes != null && cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapRemove();
 
                 if (entryBytes != null) {
                     GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(entryBytes));
@@ -848,6 +872,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                                     null);
                             }
 
+                            if (cctx.config().isStatisticsEnabled())
+                                cctx.cache().metrics0().onSwapRemove();
+
                             // Always fire this event, since preloading depends on it.
                             onUnswapped(swapKey.partition(), key, entry);
 
@@ -880,7 +907,12 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         int part = cctx.affinity().partition(key);
 
-        return offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+        boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+        if(rmv && cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onOffHeapRemove();
+
+        return rmv;
     }
 
     /**
@@ -925,6 +957,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                     return;
 
                 try {
+                    if (cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onSwapRemove();
+
                     GridCacheSwapEntry entry = swapEntry(unmarshalSwapEntry(rmv));
 
                     if (entry == null)
@@ -942,11 +977,12 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try offheap.
         if (offheapEnabled) {
-            byte[] val = offheap.remove(spaceName,
-                part,
-                key.value(cctx.cacheObjectContext(), false),
+            byte[] val = offheap.remove(spaceName, part, key.value(cctx.cacheObjectContext(), false),
                 key.valueBytes(cctx.cacheObjectContext()));
 
+            if(val != null && cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onOffHeapRemove();
+
             if (val != null) {
                 if (c != null)
                     c.apply(val); // Probably we should read value and apply closure before removing...
@@ -1007,6 +1043,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
         if (offheapEnabled) {
             offheap.put(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()), entry.marshal());
 
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onOffHeapWrite();
+
             if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
                 cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid)null, null,
                     EVT_CACHE_OBJECT_TO_OFFHEAP, null, false, null, true, null, null, null);
@@ -1035,11 +1074,11 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         if (offheapEnabled) {
             for (GridCacheBatchSwapEntry swapEntry : swapped) {
-                offheap.put(spaceName,
-                    swapEntry.partition(),
-                    swapEntry.key(),
-                    swapEntry.key().valueBytes(cctx.cacheObjectContext()),
-                    swapEntry.marshal());
+                offheap.put(spaceName, swapEntry.partition(), swapEntry.key(),
+                    swapEntry.key().valueBytes(cctx.cacheObjectContext()), swapEntry.marshal());
+
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapWrite();
 
                 if (cctx.events().isRecordable(EVT_CACHE_OBJECT_TO_OFFHEAP))
                     cctx.events().addEvent(swapEntry.partition(), swapEntry.key(), cctx.nodeId(),
@@ -1071,6 +1110,9 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                         qryMgr.onSwap(batchSwapEntry.key());
                 }
             }
+
+            if (cctx.config().isStatisticsEnabled())
+                cctx.cache().metrics0().onSwapWrite(batch.size());
         }
     }
 
@@ -1082,17 +1124,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
      * @param entry Entry bytes.
      * @throws IgniteCheckedException If failed.
      */
-    private void writeToSwap(int part,
-        KeyCacheObject key,
-        byte[] entry)
-        throws IgniteCheckedException
-    {
+    private void writeToSwap(int part, KeyCacheObject key, byte[] entry) throws IgniteCheckedException {
         checkIteratorQueue();
 
         swapMgr.write(spaceName,
             new SwapKey(key.value(cctx.cacheObjectContext(), false), part, key.valueBytes(cctx.cacheObjectContext())),
-            entry,
-            cctx.deploy().globalLoader());
+            entry, cctx.deploy().globalLoader());
+
+        if (cctx.config().isStatisticsEnabled())
+            cctx.cache().metrics0().onSwapWrite();
 
         if (cctx.events().isRecordable(EVT_CACHE_OBJECT_SWAPPED))
             cctx.events().addEvent(part, key, cctx.nodeId(), (IgniteUuid) null, null,
@@ -1274,7 +1314,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     int part = cctx.affinity().partition(key);
 
-                    offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+                    if(rmv && cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onOffHeapRemove();
                 }
                 else
                     it.removeX();
@@ -1432,6 +1475,7 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
                 return it.hasNext();
             }
 
+            @SuppressWarnings("unchecked")
             @Override protected void onRemove() throws IgniteCheckedException {
                 if (cur == null)
                     throw new IllegalStateException("Method next() has not yet been called, or the remove() method " +
@@ -1616,7 +1660,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                     int part = cctx.affinity().partition(key);
 
-                    offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                    boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+                    if(rmv && cctx.config().isStatisticsEnabled())
+                        cctx.cache().metrics0().onOffHeapRemove();
                 }
 
                 @Override protected void onClose() throws IgniteCheckedException {
@@ -1646,7 +1693,10 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
                 int part = cctx.affinity().partition(key);
 
-                offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+                boolean rmv = offheap.removex(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
+
+                if(rmv && cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapRemove();
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
index 5f9049a..9bd6321 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManager.java
@@ -43,7 +43,14 @@ public class GridCacheTtlManager extends GridCacheManagerAdapter {
 
     /** {@inheritDoc} */
     @Override protected void start0() throws IgniteCheckedException {
-        if (cctx.kernalContext().isDaemon() || !cctx.config().isEagerTtl())
+        boolean cleanupDisabled = cctx.kernalContext().isDaemon() ||
+            !cctx.config().isEagerTtl() ||
+            CU.isAtomicsCache(cctx.name()) ||
+            CU.isMarshallerCache(cctx.name()) ||
+            CU.isUtilityCache(cctx.name()) ||
+            (cctx.kernalContext().clientNode() && cctx.config().getNearConfiguration() == null);
+
+        if (cleanupDisabled)
             return;
 
         cleanupWorker = new CleanupWorker();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 549f42f..3bd2a45 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -34,12 +35,14 @@ import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
 import javax.cache.*;
+import javax.cache.configuration.*;
 import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import java.io.*;
@@ -114,13 +117,6 @@ public class GridCacheUtils {
             }
         };
 
-    /** Not evicted partitions. */
-    private static final IgnitePredicate PART_NOT_EVICTED = new P1<GridDhtLocalPartition>() {
-        @Override public boolean apply(GridDhtLocalPartition p) {
-            return p.state() != GridDhtPartitionState.EVICTED;
-        }
-    };
-
     /** */
     private static final IgniteClosure<Integer, GridCacheVersion[]> VER_ARR_FACTORY =
         new C1<Integer, GridCacheVersion[]>() {
@@ -398,30 +394,11 @@ public class GridCacheUtils {
      * @return Partition to state transformer.
      */
     @SuppressWarnings({"unchecked"})
-    public static <K, V> IgniteClosure<GridDhtLocalPartition, GridDhtPartitionState> part2state() {
+    public static IgniteClosure<GridDhtLocalPartition, GridDhtPartitionState> part2state() {
         return PART2STATE;
     }
 
     /**
-     * @return Not evicted partitions.
-     */
-    @SuppressWarnings( {"unchecked"})
-    public static <K, V> IgnitePredicate<GridDhtLocalPartition> notEvicted() {
-        return PART_NOT_EVICTED;
-    }
-
-    /**
-     * Gets all nodes on which cache with the same name is started.
-     *
-     * @param ctx Cache context.
-     * @return All nodes on which cache with the same name is started (including nodes
-     *      that may have already left).
-     */
-    public static Collection<ClusterNode> allNodes(GridCacheContext ctx) {
-        return allNodes(ctx, AffinityTopologyVersion.NONE);
-    }
-
-    /**
      * Gets all nodes on which cache with the same name is started.
      *
      * @param ctx Cache context.
@@ -446,59 +423,6 @@ public class GridCacheUtils {
     }
 
     /**
-     * Gets alive nodes.
-     *
-     * @param ctx Cache context.
-     * @param topOrder Maximum allowed node order.
-     * @return Affinity nodes.
-     */
-    public static Collection<ClusterNode> aliveNodes(final GridCacheContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().aliveCacheNodes(ctx.namex(), topOrder);
-    }
-
-    /**
-     * Gets remote nodes on which cache with the same name is started.
-     *
-     * @param ctx Cache context.
-     * @return Remote nodes on which cache with the same name is started.
-     */
-    public static Collection<ClusterNode> remoteNodes(final GridCacheContext ctx) {
-        return remoteNodes(ctx, AffinityTopologyVersion.NONE);
-    }
-
-    /**
-     * Gets remote node with at least one cache configured.
-     *
-     * @param ctx Shared cache context.
-     * @return Collection of nodes with at least one cache configured.
-     */
-    public static Collection<ClusterNode> remoteNodes(GridCacheSharedContext ctx) {
-        return remoteNodes(ctx, AffinityTopologyVersion.NONE);
-    }
-
-    /**
-     * Gets remote nodes on which cache with the same name is started.
-     *
-     * @param ctx Cache context.
-     * @param topOrder Maximum allowed node order.
-     * @return Remote nodes on which cache with the same name is started.
-     */
-    public static Collection<ClusterNode> remoteNodes(final GridCacheContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().remoteCacheNodes(ctx.namex(), topOrder);
-    }
-
-    /**
-     * Gets alive nodes.
-     *
-     * @param ctx Cache context.
-     * @param topOrder Maximum allowed node order.
-     * @return Affinity nodes.
-     */
-    public static Collection<ClusterNode> aliveRemoteNodes(final GridCacheContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().aliveRemoteCacheNodes(ctx.namex(), topOrder);
-    }
-
-    /**
      * Gets remote nodes with at least one cache configured.
      *
      * @param ctx Cache shared context.
@@ -510,25 +434,15 @@ public class GridCacheUtils {
     }
 
     /**
-     * Gets alive nodes with at least one cache configured.
-     *
-     * @param ctx Cache context.
-     * @param topOrder Maximum allowed node order.
-     * @return Affinity nodes.
-     */
-    public static Collection<ClusterNode> aliveCacheNodes(final GridCacheSharedContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().aliveNodesWithCaches(topOrder);
-    }
-
-    /**
      * Gets alive remote nodes with at least one cache configured.
      *
      * @param ctx Cache context.
      * @param topOrder Maximum allowed node order.
      * @return Affinity nodes.
      */
-    public static Collection<ClusterNode> aliveRemoteCacheNodes(final GridCacheSharedContext ctx, AffinityTopologyVersion topOrder) {
-        return ctx.discovery().aliveRemoteNodesWithCaches(topOrder);
+    public static Collection<ClusterNode> aliveRemoteServerNodesWithCaches(final GridCacheSharedContext ctx,
+        AffinityTopologyVersion topOrder) {
+        return ctx.discovery().aliveRemoteServerNodesWithCaches(topOrder);
     }
 
     /**
@@ -577,90 +491,34 @@ public class GridCacheUtils {
     }
 
     /**
-     * Checks if given node has specified cache started.
-     *
-     * @param cacheName Cache name.
-     * @param node Node to check.
-     * @return {@code True} if given node has specified cache started.
-     */
-    public static boolean cacheNode(String cacheName, ClusterNode node) {
-        return cacheNode(cacheName, (GridCacheAttributes[])node.attribute(ATTR_CACHE));
-    }
-
-    /**
-     * Checks if given attributes relate the the node which has (or had) specified cache started.
-     *
-     * @param cacheName Cache name.
-     * @param caches Node cache attributes.
-     * @return {@code True} if given node has specified cache started.
-     */
-    public static boolean cacheNode(String cacheName, GridCacheAttributes[] caches) {
-        if (caches != null)
-            for (GridCacheAttributes attrs : caches)
-                if (F.eq(cacheName, attrs.cacheName()))
-                    return true;
-
-        return false;
-    }
-
-    /**
-     * Gets oldest alive node for specified topology version.
-     *
-     * @param cctx Cache context.
-     * @return Oldest node for the current topology version.
-     */
-    public static ClusterNode oldest(GridCacheContext cctx) {
-        return oldest(cctx, AffinityTopologyVersion.NONE);
-    }
-
-    /**
-     * Gets oldest alive node across nodes with at least one cache configured.
-     *
-     * @param ctx Cache context.
-     * @return Oldest node.
-     */
-    public static ClusterNode oldest(GridCacheSharedContext ctx) {
-        return oldest(ctx, AffinityTopologyVersion.NONE);
-    }
-
-    /**
-     * Gets oldest alive node for specified topology version.
+     * Gets oldest alive server node with at least one cache configured for specified topology version.
      *
-     * @param cctx Cache context.
-     * @param topOrder Maximum allowed node order.
-     * @return Oldest node for the given topology version.
+     * @param ctx Context.
+     * @param topVer Maximum allowed topology version.
+     * @return Oldest alive cache server node.
      */
-    public static ClusterNode oldest(GridCacheContext cctx, AffinityTopologyVersion topOrder) {
-        ClusterNode oldest = null;
+    @Nullable public static ClusterNode oldestAliveCacheServerNode(GridCacheSharedContext ctx,
+        AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> nodes = ctx.discovery().aliveServerNodesWithCaches(topVer);
 
-        for (ClusterNode n : aliveNodes(cctx, topOrder))
-            if (oldest == null || n.order() < oldest.order())
-                oldest = n;
-
-        assert oldest != null : "Failed to find oldest node for cache context [name=" + cctx.name() + ", topOrder=" + topOrder + ']';
-        assert oldest.order() <= topOrder.topologyVersion() || AffinityTopologyVersion.NONE.equals(topOrder);
+        if (nodes.isEmpty())
+            return null;
 
-        return oldest;
+        return oldest(nodes);
     }
 
     /**
-     * Gets oldest alive node with at least one cache configured for specified topology version.
-     *
-     * @param cctx Shared cache context.
-     * @param topOrder Maximum allowed node order.
+     * @param nodes Nodes.
      * @return Oldest node for the given topology version.
      */
-    public static ClusterNode oldest(GridCacheSharedContext cctx, AffinityTopologyVersion topOrder) {
+    @Nullable public static ClusterNode oldest(Collection<ClusterNode> nodes) {
         ClusterNode oldest = null;
 
-        for (ClusterNode n : aliveCacheNodes(cctx, topOrder)) {
+        for (ClusterNode n : nodes) {
             if (oldest == null || n.order() < oldest.order())
                 oldest = n;
         }
 
-        assert oldest != null : "Failed to find oldest node with caches: " + topOrder;
-        assert oldest.order() <= topOrder.topologyVersion() || AffinityTopologyVersion.NONE.equals(topOrder);
-
         return oldest;
     }
 
@@ -718,30 +576,6 @@ public class GridCacheUtils {
     }
 
     /**
-     * @return Closure that converts tx entry to key.
-     */
-    @SuppressWarnings({"unchecked"})
-    public static <K, V> IgniteClosure<IgniteTxEntry, K> tx2key() {
-        return (IgniteClosure<IgniteTxEntry, K>)tx2key;
-    }
-
-    /**
-     * @return Closure that converts tx entry collection to key collection.
-     */
-    @SuppressWarnings({"unchecked"})
-    public static <K, V> IgniteClosure<Collection<IgniteTxEntry>, Collection<K>> txCol2Key() {
-        return (IgniteClosure<Collection<IgniteTxEntry>, Collection<K>>)txCol2key;
-    }
-
-    /**
-     * @return Converts transaction entry to cache entry.
-     */
-    @SuppressWarnings( {"unchecked"})
-    public static <K, V> IgniteClosure<IgniteTxEntry, GridCacheEntryEx> tx2entry() {
-        return (IgniteClosure<IgniteTxEntry, GridCacheEntryEx>)tx2entry;
-    }
-
-    /**
      * @return Closure which converts transaction entry xid to XID version.
      */
     @SuppressWarnings( {"unchecked"})
@@ -1451,13 +1285,7 @@ public class GridCacheUtils {
     }
 
     /**
-     * @return Cache ID for utility cache.
-     */
-    public static int utilityCacheId() {
-        return cacheId(UTILITY_CACHE_NAME);
-    }
-
-    /**
+     * @param cacheName Cache name.
      * @return Cache ID.
      */
     public static int cacheId(String cacheName) {
@@ -1688,7 +1516,7 @@ public class GridCacheUtils {
     /**
      * @param aff Affinity.
      * @param n Node.
-     * @return Predicate that evaulates to {@code true} if entry is primary for node.
+     * @return Predicate that evaluates to {@code true} if entry is primary for node.
      */
     public static CacheEntryPredicate cachePrimary(
         final Affinity aff,
@@ -1790,4 +1618,76 @@ public class GridCacheUtils {
 
         return res;
     }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if given node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
+     */
+    public static boolean clientNode(ClusterNode node) {
+        Boolean clientModeAttr = node.attribute(IgniteNodeAttributes.ATTR_CLIENT_MODE);
+
+        assert clientModeAttr != null : node;
+
+        return clientModeAttr != null && clientModeAttr;
+    }
+
+    /**
+     * @param node Node.
+     * @param filter Node filter.
+     * @return {@code True} if node is not client node and pass given filter.
+     */
+    public static boolean affinityNode(ClusterNode node, IgnitePredicate<ClusterNode> filter) {
+        return !clientNode(node) && filter.apply(node);
+    }
+
+    /**
+     * Creates and starts store session listeners.
+     *
+     * @param ctx Kernal context.
+     * @param factories Factories.
+     * @return Listeners.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public static Collection<CacheStoreSessionListener> startStoreSessionListeners(GridKernalContext ctx,
+        Factory<CacheStoreSessionListener>[] factories) throws IgniteCheckedException {
+        if (factories == null)
+            return null;
+
+        Collection<CacheStoreSessionListener> lsnrs = new ArrayList<>(factories.length);
+
+        for (Factory<CacheStoreSessionListener> factory : factories) {
+            CacheStoreSessionListener lsnr = factory.create();
+
+            if (lsnr != null) {
+                ctx.resource().injectGeneric(lsnr);
+
+                if (lsnr instanceof LifecycleAware)
+                    ((LifecycleAware)lsnr).start();
+
+                lsnrs.add(lsnr);
+            }
+        }
+
+        return lsnrs;
+    }
+
+    /**
+     * Stops store session listeners.
+     *
+     * @param ctx Kernal context.
+     * @param sesLsnrs Session listeners.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public static void stopStoreSessionListeners(GridKernalContext ctx, Collection<CacheStoreSessionListener> sesLsnrs)
+        throws IgniteCheckedException {
+        if (sesLsnrs == null)
+            return;
+
+        for (CacheStoreSessionListener lsnr : sesLsnrs) {
+            if (lsnr instanceof LifecycleAware)
+                ((LifecycleAware)lsnr).stop();
+
+            ctx.resource().cleanupGeneric(lsnr);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f840015..4390993 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -699,6 +699,29 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
+        try {
+            CacheOperationContext prev = onEnter(opCtx);
+
+            try {
+                if (isAsync()) {
+                    setFuture(delegate.getAllOutTxAsync(keys));
+
+                    return null;
+                }
+                else
+                    return delegate.getAllOutTx(keys);
+            }
+            finally {
+                onLeave(prev);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw cacheException(e);
+        }
+    }
+
     /**
      * @param keys Keys.
      * @return Values map.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 5184115..d98379c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -775,6 +775,11 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public Set<K> keySet();
 
     /**
+     * @return Set of keys including internal keys.
+     */
+    public Set<K> keySetx();
+
+    /**
      * Set of keys for which this node is primary.
      * This set is dynamic and may change with grid topology changes.
      * Note that this set will contain mappings for all keys, even if their values are
@@ -1618,7 +1623,16 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * @return Value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public Map<K, V> getAllOutTx(List<K> keys) throws IgniteCheckedException;
+    public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException;
+
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys Keys to get values for.
+     * @return Future for getAllOutTx operation.
+     */
+    public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys);
 
     /**
      * Checks whether this cache is IGFS data cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
index 0186a90..0790052 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/affinity/GridCacheAffinityImpl.java
@@ -84,9 +84,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
     @Override public int[] primaryPartitions(ClusterNode n) {
         A.notNull(n, "n");
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
-
-        Set<Integer> parts = cctx.affinity().primaryPartitions(n.id(), topVer);
+        Set<Integer> parts = cctx.affinity().primaryPartitions(n.id(), topologyVersion());
 
         return U.toIntArray(parts);
     }
@@ -95,9 +93,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
     @Override public int[] backupPartitions(ClusterNode n) {
         A.notNull(n, "n");
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
-
-        Set<Integer> parts = cctx.affinity().backupPartitions(n.id(), topVer);
+        Set<Integer> parts = cctx.affinity().backupPartitions(n.id(), topologyVersion());
 
         return U.toIntArray(parts);
     }
@@ -108,7 +104,7 @@ public class GridCacheAffinityImpl<K, V> implements Affinity<K> {
 
         Collection<Integer> parts = new HashSet<>();
 
-        AffinityTopologyVersion topVer = new AffinityTopologyVersion(cctx.discovery().topologyVersion());
+        AffinityTopologyVersion topVer = topologyVersion();
 
         for (int partsCnt = partitions(), part = 0; part < partsCnt; part++) {
             for (ClusterNode affNode : cctx.affinity().nodes(part, topVer)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index fa8d192..b5c5161 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -218,7 +218,7 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
                         }
                     },
                     new QueueHeaderPredicate(),
-                    cctx.isLocal() || cctx.isReplicated(),
+                    cctx.isLocal() || (cctx.isReplicated() && cctx.affinityNode()),
                     true);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index b79f9d5..bd72764 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -327,13 +327,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     }
 
     /**
-     *
-     */
-    public void onUnlock() {
-        // No-op.
-    }
-
-    /**
      * Unlocks local lock.
      *
      * @return Removed candidate, or <tt>null</tt> if thread still holds the lock.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
index fded3c9..bd1dedf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxMapping.java
@@ -63,6 +63,9 @@ public class GridDistributedTxMapping implements Externalizable {
     /** {@code True} if mapping is for near caches, {@code false} otherwise. */
     private boolean near;
 
+    /** {@code True} if this is first mapping for optimistic tx on client node. */
+    private boolean clientFirst;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -108,6 +111,20 @@ public class GridDistributedTxMapping implements Externalizable {
     }
 
     /**
+     * @return {@code True} if this is first mapping for optimistic tx on client node.
+     */
+    public boolean clientFirst() {
+        return clientFirst;
+    }
+
+    /**
+     * @param clientFirst {@code True} if this is first mapping for optimistic tx on client node.
+     */
+    public void clientFirst(boolean clientFirst) {
+        this.clientFirst = clientFirst;
+    }
+
+    /**
      * @return {@code True} if mapping is for near caches, {@code false} otherwise.
      */
     public boolean near() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 331de4e..c3f3e7f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -210,7 +210,9 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                 removeNode(exchId.nodeId());
 
             // In case if node joins, get topology at the time of joining node.
-            ClusterNode oldest = CU.oldest(cctx, topVer);
+            ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+
+            assert oldest != null;
 
             if (log.isDebugEnabled())
                 log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
@@ -218,7 +220,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
             long updateSeq = this.updateSeq.incrementAndGet();
 
             // If this is the oldest node.
-            if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cacheId)) {
+            if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cacheId, exchId.topologyVersion())) {
                 if (node2part == null) {
                     node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 
@@ -665,7 +667,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         assert nodeId.equals(cctx.localNodeId());
 
         // In case if node joins, get topology at the time of joining node.
-        ClusterNode oldest = CU.oldest(cctx, topVer);
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
 
         // If this node became the oldest node.
         if (oldest.id().equals(cctx.localNodeId())) {
@@ -715,7 +717,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         assert nodeId != null;
         assert lock.writeLock().isHeldByCurrentThread();
 
-        ClusterNode oldest = CU.oldest(cctx, topVer);
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
 
         ClusterNode loc = cctx.localNode();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
index 303d649..7bae7f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
@@ -96,12 +96,12 @@ public class GridDhtAssignmentFetchFuture extends GridFutureAdapter<List<List<Cl
 
     /**
      * @param node Node.
-     * @param res Reponse.
+     * @param res Response.
      */
     public void onResponse(ClusterNode node, GridDhtAffinityAssignmentResponse res) {
         if (!res.topologyVersion().equals(topVer)) {
             if (log.isDebugEnabled())
-                log.debug("Received affinity assignment for wrong topolgy version (will ignore) " +
+                log.debug("Received affinity assignment for wrong topology version (will ignore) " +
                     "[node=" + node + ", res=" + res + ", topVer=" + topVer + ']');
 
             return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 10b84e2..adea9e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -54,7 +54,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     private GridDhtPartitionTopology top;
 
     /** Preloader. */
-    protected GridCachePreloader<K, V> preldr;
+    protected GridCachePreloader preldr;
 
     /** Multi tx future holder. */
     private ThreadLocal<IgniteBiTuple<IgniteUuid, GridDhtTopologyFuture>> multiTxHolder = new ThreadLocal<>();
@@ -75,7 +75,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     protected GridDhtCacheAdapter(GridCacheContext<K, V> ctx) {
         super(ctx, ctx.config().getStartSize());
 
-        top = new GridDhtPartitionTopologyImpl<>(ctx);
+        top = new GridDhtPartitionTopologyImpl(ctx);
     }
 
     /**
@@ -87,7 +87,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     protected GridDhtCacheAdapter(GridCacheContext<K, V> ctx, GridCacheConcurrentMap map) {
         super(ctx, map);
 
-        top = new GridDhtPartitionTopologyImpl<>(ctx);
+        top = new GridDhtPartitionTopologyImpl(ctx);
     }
 
     /** {@inheritDoc} */
@@ -168,17 +168,17 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     }
 
     /** {@inheritDoc} */
-    @Override public GridCachePreloader<K, V> preloader() {
+    @Override public GridCachePreloader preloader() {
         return preldr;
     }
 
     /**
      * @return DHT preloader.
      */
-    public GridDhtPreloader<K, V> dhtPreloader() {
+    public GridDhtPreloader dhtPreloader() {
         assert preldr instanceof GridDhtPreloader;
 
-        return (GridDhtPreloader<K, V>)preldr;
+        return (GridDhtPreloader)preldr;
     }
 
     /**
@@ -932,6 +932,21 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     }
 
     /**
+     * @param expVer Expected topology version.
+     * @param curVer Current topology version.
+     * @return {@code True} if cache affinity changed and operation should be remapped.
+     */
+    protected final boolean needRemap(AffinityTopologyVersion expVer, AffinityTopologyVersion curVer) {
+        if (expVer.equals(curVer))
+            return false;
+
+        Collection<ClusterNode> cacheNodes0 = ctx.discovery().cacheAffinityNodes(ctx.name(), expVer);
+        Collection<ClusterNode> cacheNodes1 = ctx.discovery().cacheAffinityNodes(ctx.name(), curVer);
+
+        return !cacheNodes0.equals(cacheNodes1);
+    }
+
+    /**
      * @param primary If {@code true} includes primary entries.
      * @param backup If {@code true} includes backup entries.
      * @return Local entries iterator.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index c9a7af8..89b85c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -292,12 +292,8 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
         return ret;
     }
 
-    /**
-     * Calls {@link GridDhtLocalPartition#onUnlock()} for this entry's partition.
-     */
+    /** {@inheritDoc} */
     @Override public void onUnlock() {
-        super.onUnlock();
-
         locPart.onUnlock();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index f6f930e..742fbfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -295,6 +295,11 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     if (info == null)
                         continue;
 
+                    boolean addReader = (!e.deleted() && k.getValue() && !skipVals);
+
+                    if (addReader)
+                        e.unswap(false);
+
                     // Register reader. If there are active transactions for this entry,
                     // then will wait for their completion before proceeding.
                     // TODO: GG-4003:
@@ -303,8 +308,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     // TODO: To fix, check that reader is contained in the list of readers once
                     // TODO: again after the returned future completes - if not, try again.
                     // TODO: Also, why is info read before transactions are complete, and not after?
-                    IgniteInternalFuture<Boolean> f = (!e.deleted() && k.getValue() && !skipVals) ?
-                        e.addReader(reader, msgId, topVer) : null;
+                    IgniteInternalFuture<Boolean> f = addReader ? e.addReader(reader, msgId, topVer) : null;
 
                     if (f != null) {
                         if (txFut == null)
@@ -317,6 +321,9 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
 
                     break;
                 }
+                catch (IgniteCheckedException err) {
+                    return new GridFinishedFuture<>(err);
+                }
                 catch (GridCacheEntryRemovedException ignore) {
                     if (log.isDebugEnabled())
                         log.debug("Got removed entry when getting a DHT value: " + e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 c57eded..bdaa552 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
@@ -47,7 +47,7 @@ import static org.apache.ignite.internal.processors.dr.GridDrType.*;
 /**
  * Cache lock future.
  */
-public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Boolean>
+public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     implements GridCacheMvccFuture<Boolean>, GridDhtFuture<Boolean>, GridCacheMappedVersion {
     /** */
     private static final long serialVersionUID = 0L;
@@ -60,7 +60,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
 
     /** Cache registry. */
     @GridToStringExclude
-    private GridCacheContext<K, V> cctx;
+    private GridCacheContext<?, ?> cctx;
 
     /** Near node ID. */
     private UUID nearNodeId;
@@ -151,7 +151,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
      * @param skipStore Skip store flag.
      */
     public GridDhtLockFuture(
-        GridCacheContext<K, V> cctx,
+        GridCacheContext<?, ?> cctx,
         UUID nearNodeId,
         GridCacheVersion nearLockVer,
         @NotNull AffinityTopologyVersion topVer,
@@ -221,7 +221,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
      * @param cacheCtx Cache context.
      * @param invalidPart Partition to retry.
      */
-    void addInvalidPartition(GridCacheContext<K, V> cacheCtx, int invalidPart) {
+    void addInvalidPartition(GridCacheContext<?, ?> cacheCtx, int invalidPart) {
         invalidParts.add(invalidPart);
 
         // Register invalid partitions with transaction.
@@ -1170,7 +1170,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
          * @param entries Entries to check.
          */
         @SuppressWarnings({"ForLoopReplaceableByForEach"})
-        private void evictReaders(GridCacheContext<K, V> cacheCtx, Collection<IgniteTxKey> keys, UUID nodeId, long msgId,
+        private void evictReaders(GridCacheContext<?, ?> cacheCtx, Collection<IgniteTxKey> keys, UUID nodeId, long msgId,
             @Nullable List<GridDhtCacheEntry> entries) {
             if (entries == null || keys == null || entries.isEmpty() || keys.isEmpty())
                 return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 073e0e7..374ab87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -41,7 +41,7 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDh
  * Partition topology.
  */
 @GridToStringExclude
-class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
+class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     /** If true, then check consistency. */
     private static final boolean CONSISTENCY_CHECK = false;
 
@@ -49,7 +49,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
     private static final boolean FULL_MAP_DEBUG = false;
 
     /** Context. */
-    private final GridCacheContext<K, V> cctx;
+    private final GridCacheContext<?, ?> cctx;
 
     /** Logger. */
     private final IgniteLogger log;
@@ -85,7 +85,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
     /**
      * @param cctx Context.
      */
-    GridDhtPartitionTopologyImpl(GridCacheContext<K, V> cctx) {
+    GridDhtPartitionTopologyImpl(GridCacheContext<?, ?> cctx) {
         assert cctx != null;
 
         this.cctx = cctx;
@@ -239,7 +239,9 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
                 removeNode(exchId.nodeId());
 
             // In case if node joins, get topology at the time of joining node.
-            ClusterNode oldest = CU.oldest(cctx.shared(), topVer);
+            ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
+
+            assert oldest != null;
 
             if (log.isDebugEnabled())
                 log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
@@ -247,7 +249,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
             long updateSeq = this.updateSeq.incrementAndGet();
 
             // If this is the oldest node.
-            if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cctx.cacheId())) {
+            if (oldest.id().equals(loc.id()) || exchFut.isCacheAdded(cctx.cacheId(), exchId.topologyVersion())) {
                 if (node2part == null) {
                     node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 
@@ -274,7 +276,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
             if (cctx.rebalanceEnabled()) {
                 for (int p = 0; p < num; p++) {
                     // If this is the first node in grid.
-                    boolean added = exchFut.isCacheAdded(cctx.cacheId());
+                    boolean added = exchFut.isCacheAdded(cctx.cacheId(), exchId.topologyVersion());
 
                     if ((oldest.id().equals(loc.id()) && oldest.id().equals(exchId.nodeId()) && exchId.isJoined()) || added) {
                         assert exchId.isJoined() || added;
@@ -604,7 +606,7 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
 
         try {
             return new GridDhtPartitionMap(cctx.nodeId(), updateSeq.get(),
-                F.viewReadOnly(locParts, CU.<K, V>part2state()), true);
+                F.viewReadOnly(locParts, CU.part2state()), true);
         }
         finally {
             lock.readLock().unlock();
@@ -660,13 +662,15 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
      * @return List of nodes for the partition.
      */
     private List<ClusterNode> nodes(int p, AffinityTopologyVersion topVer, GridDhtPartitionState state, GridDhtPartitionState... states) {
-        Collection<UUID> allIds = topVer.topologyVersion() > 0 ? F.nodeIds(CU.allNodes(cctx, topVer)) : null;
+        Collection<UUID> allIds = topVer.topologyVersion() > 0 ? F.nodeIds(CU.affinityNodes(cctx, topVer)) : null;
 
         lock.readLock().lock();
 
         try {
             assert node2part != null && node2part.valid() : "Invalid node-to-partitions map [topVer=" + topVer +
-                ", allIds=" + allIds + ", node2part=" + node2part + ']';
+                ", allIds=" + allIds +
+                ", node2part=" + node2part +
+                ", cache=" + cctx.name() + ']';
 
             Collection<UUID> nodeIds = part2node.get(p);
 
@@ -738,7 +742,11 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
 
         try {
             assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
-                ", locNodeId=" + cctx.localNode().id() + ", locName=" + cctx.gridName() + ']';
+                ", cache=" + cctx.name() +
+                ", started=" + cctx.started() +
+                ", stopping=" + stopping +
+                ", locNodeId=" + cctx.localNode().id() +
+                ", locName=" + cctx.gridName() + ']';
 
             GridDhtPartitionFullMap m = node2part;
 
@@ -756,6 +764,8 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
         if (log.isDebugEnabled())
             log.debug("Updating full partition map [exchId=" + exchId + ", parts=" + fullMapString() + ']');
 
+        assert partMap != null;
+
         lock.writeLock().lock();
 
         try {
@@ -1024,7 +1034,9 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
         assert nodeId.equals(cctx.nodeId());
 
         // In case if node joins, get topology at the time of joining node.
-        ClusterNode oldest = CU.oldest(cctx, topVer);
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
+
+        assert oldest != null;
 
         // If this node became the oldest node.
         if (oldest.id().equals(cctx.nodeId())) {
@@ -1074,7 +1086,9 @@ class GridDhtPartitionTopologyImpl<K, V> implements GridDhtPartitionTopology {
         assert nodeId != null;
         assert lock.writeLock().isHeldByCurrentThread();
 
-        ClusterNode oldest = CU.oldest(cctx, topVer);
+        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
+
+        assert oldest != null;
 
         ClusterNode loc = cctx.localNode();
 


[07/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
deleted file mode 100644
index 64d3831..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/fifo/GridCacheFifoEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,372 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.fifo;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.fifo.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheMode.*;
-
-/**
- * FIFO Eviction test.
- */
-@SuppressWarnings({"TypeMayBeWeakened"})
-public class GridCacheFifoEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<FifoEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            assert !e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e4, e5, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e3.isEvicted();
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assert !e5.isEvicted();
-
-            info(p);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            startGrid();
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] fifos = new MockEntry[keys];
-
-            for (int i = 0; i < fifos.length; i++)
-                fifos[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 5000000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(fifos.length);
-
-                MockEntry e = entry(fifos, j);
-
-                if (rmv)
-                    fifos[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            int curSize = p.getCurrentSize();
-
-            assert curSize <= max : "curSize <= max [curSize=" + curSize + ", max=" + max + ']';
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            FifoEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Object, Object> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-            info("Current cache key size " + cache.size());
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new FifoEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected FifoEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new FifoEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
deleted file mode 100644
index c623b38..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,417 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.*;
-import org.apache.ignite.cache.eviction.lru.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-
-import java.util.*;
-
-/**
- * LRU Eviction test.
- */
-@SuppressWarnings( {"TypeMayBeWeakened"})
-public class GridCacheLruEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<LruEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPolicy() throws Exception {
-        startGrid();
-
-        try {
-            MockEntry e1 = new MockEntry("1", "1");
-            MockEntry e2 = new MockEntry("2", "2");
-            MockEntry e3 = new MockEntry("3", "3");
-            MockEntry e4 = new MockEntry("4", "4");
-            MockEntry e5 = new MockEntry("5", "5");
-
-            LruEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(3);
-
-            p.onEntryAccessed(false, e1);
-
-            check(p.queue(), e1);
-
-            p.onEntryAccessed(false, e2);
-
-            check(p.queue(), e1, e2);
-
-            p.onEntryAccessed(false, e3);
-
-            check(p.queue(), e1, e2, e3);
-
-            assert !e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-
-            assertEquals(3, p.getCurrentSize());
-
-            p.onEntryAccessed(false, e4);
-
-            check(p.queue(), e2, e3, e4);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e1.isEvicted();
-            assert !e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            check(p.queue(), e3, e4, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e2.isEvicted();
-            assert !e3.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
-
-            check(p.queue(), e4, e5, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            assert e3.isEvicted();
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e1, e5);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e1);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e5, e1);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(false, e5);
-
-            assertEquals(3, p.getCurrentSize());
-
-            check(p.queue(), e4, e1, e5);
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e1);
-
-            assertEquals(2, p.getCurrentSize());
-
-            assert !e1.isEvicted();
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e4);
-
-            assertEquals(1, p.getCurrentSize());
-
-            assert !e4.isEvicted();
-            assert !e5.isEvicted();
-
-            p.onEntryAccessed(true, e5);
-
-            assertEquals(0, p.getCurrentSize());
-
-            assert !e5.isEvicted();
-
-            info(p);
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            int cnt = 11;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, new MockEntry(Integer.toString(i), Integer.toString(i)));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMiddleAccess() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 8;
-
-            p.setMaxSize(max);
-
-            MockEntry entry1 = new MockEntry("1", "1");
-            MockEntry entry2 = new MockEntry("2", "2");
-            MockEntry entry3 = new MockEntry("3", "3");
-
-            p.onEntryAccessed(false, entry1);
-            p.onEntryAccessed(false, entry2);
-            p.onEntryAccessed(false, entry3);
-
-            MockEntry[] freqUsed = new MockEntry[] {
-                new MockEntry("4", "4"),
-                new MockEntry("5", "5"),
-                new MockEntry("6", "6"),
-                new MockEntry("7", "7"),
-                new MockEntry("8", "7")
-            };
-
-            for (MockEntry e : freqUsed)
-                p.onEntryAccessed(false, e);
-
-            for (MockEntry e : freqUsed)
-                assert !e.isEvicted();
-
-            int cnt = 1001;
-
-            for (int i = 0; i < cnt; i++)
-                p.onEntryAccessed(false, entry(freqUsed, i % freqUsed.length));
-
-            info(p);
-
-            assertEquals(max, p.getCurrentSize());
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        startGrid();
-
-        try {
-            LruEvictionPolicy<String, String> p = policy();
-
-            int max = 10;
-
-            p.setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            MockEntry[] lrus = new MockEntry[keys];
-
-            for (int i = 0; i < lrus.length; i++)
-                lrus[i] = new MockEntry(Integer.toString(i));
-
-            int runs = 500000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(lrus.length);
-
-                MockEntry e = entry(lrus, j);
-
-                if (rmv)
-                    lrus[j] = new MockEntry(Integer.toString(j));
-
-                p.onEntryAccessed(rmv, e);
-            }
-
-            info(p);
-
-            assert p.getCurrentSize() <= max;
-        }
-        finally {
-            stopGrid();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            MockEntry e1 = new MockEntry("1");
-
-            MockEntry e2 = new MockEntry("2");
-
-            MockEntry e3 = new MockEntry("3");
-
-            MockEntry e4 = new MockEntry("4");
-
-            MockEntry e5 = new MockEntry("5");
-
-            LruEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPut() throws Exception {
-        mode = CacheMode.LOCAL;
-        syncCommit = true;
-        plcMax = 100;
-
-        Ignite ignite = startGrid();
-
-        try {
-            IgniteCache<Integer, Integer> cache = ignite.cache(null);
-
-            int cnt = 500;
-
-            int min = Integer.MAX_VALUE;
-
-            int minIdx = 0;
-
-            for (int i = 0; i < cnt; i++) {
-                cache.put(i, i);
-
-                int cacheSize = cache.size();
-
-                if (i > plcMax && cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-
-            min = Integer.MAX_VALUE;
-
-            minIdx = 0;
-
-            // Touch.
-            for (int i = cnt; --i > cnt - plcMax;) {
-                cache.get(i);
-
-                int cacheSize = cache.size();
-
-                if (cacheSize < min) {
-                    min = cacheSize;
-                    minIdx = i;
-                }
-            }
-
-            info("----");
-            info("Min cache size [min=" + min + ", idx=" + minIdx + ']');
-            info("Current cache size " + cache.size());
-
-            assert min >= plcMax : "Min cache size is too small: " + min;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override protected LruEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new LruEvictionPolicy<>(plcMax);
-    }
-
-    @Override protected LruEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new LruEvictionPolicy<>(nearMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int endNearPlcSize) {
-        for (int i = 0; i < gridCnt; i++)
-            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
-                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        for (int i = 0; i < gridCnt; i++)
-            assert policy(i).queue().size() <= plcMax;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
deleted file mode 100644
index e0606a7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheLruNearEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,136 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.lru.*;
-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.testframework.junits.common.*;
-
-import java.util.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheRebalanceMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * LRU near eviction tests (GG-8884).
- */
-public class GridCacheLruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Maximum size for near eviction policy. */
-    private static final int EVICTION_MAX_SIZE = 10;
-
-    /** Grid count. */
-    private static final int GRID_COUNT = 2;
-
-    /** Cache atomicity mode specified by test. */
-    private CacheAtomicityMode atomicityMode;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        CacheConfiguration cc = new CacheConfiguration();
-
-        cc.setAtomicityMode(atomicityMode);
-        cc.setCacheMode(PARTITIONED);
-        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-        cc.setRebalanceMode(SYNC);
-        cc.setStartSize(100);
-        cc.setBackups(0);
-
-        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-        nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(EVICTION_MAX_SIZE));
-        cc.setNearConfiguration(nearCfg);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkNearEvictionMaxSize() throws Exception {
-        startGridsMultiThreaded(GRID_COUNT);
-
-        try {
-            Random rand = new Random(0);
-
-            int cnt = 1000;
-
-            info("Inserting " + cnt + " keys to cache.");
-
-            try (IgniteDataStreamer<Integer, String> ldr = grid(0).dataStreamer(null)) {
-                for (int i = 0; i < cnt; i++)
-                    ldr.addData(i, Integer.toString(i));
-            }
-
-            for (int i = 0; i < GRID_COUNT; i++)
-                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                    near(i).nearSize() <= EVICTION_MAX_SIZE);
-
-            info("Getting " + cnt + " keys from cache.");
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(rand.nextInt(GRID_COUNT)).cache(null);
-
-                assertTrue(cache.get(i).equals(Integer.toString(i)));
-            }
-
-            for (int i = 0; i < GRID_COUNT; i++)
-                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                    near(i).nearSize() <= EVICTION_MAX_SIZE);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
deleted file mode 100644
index 5d4ff85..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,171 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.lru;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.eviction.lru.*;
-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.testframework.junits.common.*;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CacheRebalanceMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-
-/**
- * LRU near eviction tests for NEAR_ONLY distribution mode (GG-8884).
- */
-public class GridCacheNearOnlyLruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Grid count. */
-    private static final int GRID_COUNT = 2;
-
-    /** Maximum size for near eviction policy. */
-    private static final int EVICTION_MAX_SIZE = 10;
-
-    /** Node count. */
-    private int cnt;
-
-    /** Caching mode specified by test. */
-    private CacheMode cacheMode;
-
-    /** Cache atomicity mode specified by test. */
-    private CacheAtomicityMode atomicityMode;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        cnt = 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration c = super.getConfiguration(gridName);
-
-        if (cnt == 0)
-            c.setClientMode(true);
-
-        CacheConfiguration cc = new CacheConfiguration();
-
-        cc.setAtomicityMode(atomicityMode);
-        cc.setCacheMode(cacheMode);
-        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
-        cc.setRebalanceMode(SYNC);
-        cc.setStartSize(100);
-        cc.setBackups(0);
-
-        c.setCacheConfiguration(cc);
-
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
-
-        cnt++;
-
-        return c;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-        cacheMode = PARTITIONED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testPartitionedTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-        cacheMode = PARTITIONED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedAtomicNearEvictionMaxSize() throws Exception {
-        atomicityMode = ATOMIC;
-        cacheMode = REPLICATED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testReplicatedTransactionalNearEvictionMaxSize() throws Exception {
-        atomicityMode = TRANSACTIONAL;
-        cacheMode = REPLICATED;
-
-        checkNearEvictionMaxSize();
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    private void checkNearEvictionMaxSize() throws Exception {
-        startGrids(GRID_COUNT);
-
-        try {
-            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
-            nearCfg.setNearEvictionPolicy(new LruEvictionPolicy(EVICTION_MAX_SIZE));
-
-            grid(0).createNearCache(null, nearCfg);
-
-            int cnt = 1000;
-
-            info("Inserting " + cnt + " keys to cache.");
-
-            try (IgniteDataStreamer<Integer, String> ldr = grid(1).dataStreamer(null)) {
-                for (int i = 0; i < cnt; i++)
-                    ldr.addData(i, Integer.toString(i));
-            }
-
-            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                near(0).nearSize() <= EVICTION_MAX_SIZE);
-
-            info("Getting " + cnt + " keys from cache.");
-
-            for (int i = 0; i < cnt; i++) {
-                IgniteCache<Integer, String> cache = grid(0).cache(null);
-
-                assertTrue(cache.get(i).equals(Integer.toString(i)));
-            }
-
-            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
-                near(0).nearSize() <= EVICTION_MAX_SIZE);
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
new file mode 100644
index 0000000..3750e5a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruEvictionPolicySelfTest.java
@@ -0,0 +1,353 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.cache.eviction.*;
+import org.apache.ignite.cache.eviction.lru.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+
+/**
+ * LRU Eviction policy tests.
+ */
+public class LruEvictionPolicySelfTest extends
+    EvictionAbstractTest<LruEvictionPolicy<String, String>> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMiddleAccess() throws Exception {
+        startGrid();
+
+        try {
+            LruEvictionPolicy<String, String> p = policy();
+
+            int max = 8;
+
+            p.setMaxSize(max * MockEntry.ENTRY_SIZE);
+
+            MockEntry entry1 = new MockEntry("1", "1");
+            MockEntry entry2 = new MockEntry("2", "2");
+            MockEntry entry3 = new MockEntry("3", "3");
+
+            p.onEntryAccessed(false, entry1);
+            p.onEntryAccessed(false, entry2);
+            p.onEntryAccessed(false, entry3);
+
+            MockEntry[] freqUsed = new MockEntry[] {
+                new MockEntry("4", "4"),
+                new MockEntry("5", "5"),
+                new MockEntry("6", "6"),
+                new MockEntry("7", "7"),
+                new MockEntry("8", "7")
+            };
+
+            for (MockEntry e : freqUsed)
+                p.onEntryAccessed(false, e);
+
+            for (MockEntry e : freqUsed)
+                assert !e.isEvicted();
+
+            int cnt = 1001;
+
+            for (int i = 0; i < cnt; i++)
+                p.onEntryAccessed(false, entry(freqUsed, i % freqUsed.length));
+
+            info(p);
+
+            check(max, MockEntry.ENTRY_SIZE);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        startGrid();
+
+        try {
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            LruEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            check(p.queue(), e2, e3, e4);
+            check(MockEntry.ENTRY_SIZE, p.queue(), e2, e3, e4);
+
+            assertTrue(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assertTrue(e3.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertEquals(3, p.getCurrentSize());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5, e1);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertEquals(3, p.getCurrentSize());
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e1, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e4, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e5);
+
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            assertFalse(e5.isEvicted());
+
+            info(p);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        startGrid();
+
+        try {
+            MockEntry e1 = new MockEntry("1", "1");
+            MockEntry e2 = new MockEntry("2", "2");
+            MockEntry e3 = new MockEntry("3", "3");
+            MockEntry e4 = new MockEntry("4", "4");
+            MockEntry e5 = new MockEntry("5", "5");
+
+            LruEvictionPolicy<String, String> p = policy();
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1);
+
+            p.onEntryAccessed(false, e2);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2);
+
+            p.onEntryAccessed(false, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e1, e2, e3, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            // Batch evicted
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertTrue(e1.isEvicted());
+            assertTrue(e2.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1 = new MockEntry("1", "1"));
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(false, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5, e1);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e1, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e1.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e1);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e4, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e4);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3, e5);
+
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+
+            p.onEntryAccessed(true, e5);
+
+            check(MockEntry.ENTRY_SIZE, p.queue(), e3);
+
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(true, e3);
+
+            check(MockEntry.ENTRY_SIZE, p.queue());
+
+            info(p);
+        }
+        finally {
+            stopGrid();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected LruEvictionPolicy<String, String> createPolicy(int plcMax) {
+        LruEvictionPolicy<String, String> plc = new LruEvictionPolicy<>();
+
+        plc.setMaxSize(this.plcMax);
+        plc.setBatchSize(this.plcBatchSize);
+        plc.setMaxMemorySize(this.plcMaxMemSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected LruEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        LruEvictionPolicy<String, String> plc = new LruEvictionPolicy<>();
+
+        plc.setMaxSize(nearMax);
+        plc.setBatchSize(plcBatchSize);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkNearPolicies(int endNearPlcSize) {
+        for (int i = 0; i < gridCnt; i++)
+            for (EvictableEntry<String, String> e : nearPolicy(i).queue())
+                assert !e.isCached() : "Invalid near policy size: " + nearPolicy(i).queue();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkPolicies() {
+        for (int i = 0; i < gridCnt; i++) {
+            if (plcMaxMemSize > 0) {
+                int size = 0;
+
+                for (EvictableEntry<String, String> entry : policy(i).queue())
+                    size += ((CacheEvictableEntryImpl)entry).size();
+
+                assertEquals(size, ((LruEvictionPolicy)policy(i)).getCurrentMemorySize());
+            }
+            else
+                assertTrue(policy(i).queue().size() <= plcMax + plcBatchSize);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
new file mode 100644
index 0000000..218b817
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearEvictionPolicySelfTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
+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.testframework.junits.common.*;
+
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * LRU near eviction tests (GG-8884).
+ */
+public class LruNearEvictionPolicySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Maximum size for near eviction policy. */
+    private static final int EVICTION_MAX_SIZE = 10;
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Cache atomicity mode specified by test. */
+    private CacheAtomicityMode atomicityMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        CacheConfiguration cc = new CacheConfiguration();
+
+        cc.setAtomicityMode(atomicityMode);
+        cc.setCacheMode(PARTITIONED);
+        cc.setWriteSynchronizationMode(PRIMARY_SYNC);
+        cc.setRebalanceMode(SYNC);
+        cc.setStartSize(100);
+        cc.setBackups(0);
+
+        NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+        LruEvictionPolicy plc = new LruEvictionPolicy();
+        plc.setMaxSize(EVICTION_MAX_SIZE);
+
+        nearCfg.setNearEvictionPolicy(plc);
+        cc.setNearConfiguration(nearCfg);
+
+        c.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkNearEvictionMaxSize() throws Exception {
+        startGridsMultiThreaded(GRID_COUNT);
+
+        try {
+            Random rand = new Random(0);
+
+            int cnt = 1000;
+
+            info("Inserting " + cnt + " keys to cache.");
+
+            try (IgniteDataStreamer<Integer, String> ldr = grid(0).dataStreamer(null)) {
+                for (int i = 0; i < cnt; i++)
+                    ldr.addData(i, Integer.toString(i));
+            }
+
+            for (int i = 0; i < GRID_COUNT; i++)
+                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                    near(i).nearSize() <= EVICTION_MAX_SIZE);
+
+            info("Getting " + cnt + " keys from cache.");
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(rand.nextInt(GRID_COUNT)).cache(null);
+
+                assertTrue(cache.get(i).equals(Integer.toString(i)));
+            }
+
+            for (int i = 0; i < GRID_COUNT; i++)
+                assertTrue("Near cache size " + near(i).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                    near(i).nearSize() <= EVICTION_MAX_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
new file mode 100644
index 0000000..0d3c692
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/LruNearOnlyNearEvictionPolicySelfTest.java
@@ -0,0 +1,172 @@
+/*
+ * 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.eviction.lru;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.eviction.lru.*;
+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.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * LRU near eviction tests for NEAR_ONLY distribution mode (GG-8884).
+ */
+public class LruNearOnlyNearEvictionPolicySelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Grid count. */
+    private static final int GRID_COUNT = 2;
+
+    /** Maximum size for near eviction policy. */
+    private static final int EVICTION_MAX_SIZE = 10;
+
+    /** Node count. */
+    private int cnt;
+
+    /** Caching mode specified by test. */
+    private CacheMode cacheMode;
+
+    /** Cache atomicity mode specified by test. */
+    private CacheAtomicityMode atomicityMode;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        cnt = 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        if (cnt == 0)
+            c.setClientMode(true);
+        else {
+            CacheConfiguration cc = new CacheConfiguration();
+
+            cc.setAtomicityMode(atomicityMode);
+            cc.setCacheMode(cacheMode);
+            cc.setWriteSynchronizationMode(PRIMARY_SYNC);
+            cc.setRebalanceMode(SYNC);
+            cc.setStartSize(100);
+            cc.setBackups(0);
+
+            c.setCacheConfiguration(cc);
+        }
+
+        c.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
+
+        cnt++;
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionedAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+        cacheMode = PARTITIONED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartitionedTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+        cacheMode = PARTITIONED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedAtomicNearEvictionMaxSize() throws Exception {
+        atomicityMode = ATOMIC;
+        cacheMode = REPLICATED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReplicatedTransactionalNearEvictionMaxSize() throws Exception {
+        atomicityMode = TRANSACTIONAL;
+        cacheMode = REPLICATED;
+
+        checkNearEvictionMaxSize();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkNearEvictionMaxSize() throws Exception {
+        startGrids(GRID_COUNT);
+
+        try {
+            NearCacheConfiguration nearCfg = new NearCacheConfiguration();
+
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+            plc.setMaxSize(EVICTION_MAX_SIZE);
+
+            nearCfg.setNearEvictionPolicy(plc);
+
+            grid(0).createNearCache(null, nearCfg);
+
+            int cnt = 1000;
+
+            info("Inserting " + cnt + " keys to cache.");
+
+            try (IgniteDataStreamer<Integer, String> ldr = grid(1).dataStreamer(null)) {
+                for (int i = 0; i < cnt; i++)
+                    ldr.addData(i, Integer.toString(i));
+            }
+
+            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                near(0).nearSize() <= EVICTION_MAX_SIZE);
+
+            info("Getting " + cnt + " keys from cache.");
+
+            for (int i = 0; i < cnt; i++) {
+                IgniteCache<Integer, String> cache = grid(0).cache(null);
+
+                assertTrue(cache.get(i).equals(Integer.toString(i)));
+            }
+
+            assertTrue("Near cache size " + near(0).nearSize() + ", but eviction maximum size " + EVICTION_MAX_SIZE,
+                near(0).nearSize() <= EVICTION_MAX_SIZE);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
deleted file mode 100644
index 7088714..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/GridCacheRandomEvictionPolicySelfTest.java
+++ /dev/null
@@ -1,258 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.eviction.random;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.eviction.random.*;
-import org.apache.ignite.internal.processors.cache.eviction.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Random eviction policy test.
- */
-public class GridCacheRandomEvictionPolicySelfTest extends
-    GridCacheEvictionAbstractTest<RandomEvictionPolicy<String, String>> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testMemory() throws Exception {
-        try {
-            Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            int keys = 31;
-
-            for (int i = 0; i < keys; i++) {
-                String s = Integer.toString(i);
-
-                g.cache(null).put(s, s);
-            }
-
-            assert g.cache(null).size() <= max;
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandom() throws Exception {
-        try {
-            Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            Random rand = new Random();
-
-            int keys = 31;
-
-            String[] t = new String[keys];
-
-            for (int i = 0; i < t.length; i++)
-                t[i] = Integer.toString(i);
-
-            int runs = 10000;
-
-            for (int i = 0; i < runs; i++) {
-                boolean rmv = rand.nextBoolean();
-
-                int j = rand.nextInt(t.length);
-
-                if (rmv)
-                    g.cache(null).remove(t[j]);
-                else
-                    g.cache(null).put(t[j], t[j]);
-
-                if (i % 1000 == 0)
-                    info("Stats [cntr=" + i + ", total=" + runs + ']');
-            }
-
-            assert g.cache(null).size() <= max;
-
-            info(policy(0));
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testAllowEmptyEntries() throws Exception {
-        try {
-            startGrid();
-
-            IgniteCache<String, String> c = jcache();
-
-            MockEntry e1 = new MockEntry("1", c);
-
-            MockEntry e2 = new MockEntry("2", c);
-
-            MockEntry e3 = new MockEntry("3", c);
-
-            MockEntry e4 = new MockEntry("4", c);
-
-            MockEntry e5 = new MockEntry("5", c);
-
-            RandomEvictionPolicy<String, String> p = policy();
-
-            p.setMaxSize(10);
-
-            p.onEntryAccessed(false, e1);
-
-            assertFalse(e1.isEvicted());
-
-            p.onEntryAccessed(false, e2);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e2.isEvicted());
-
-            p.onEntryAccessed(false, e3);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-
-            p.onEntryAccessed(false, e4);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e4.isEvicted());
-
-            p.onEntryAccessed(false, e5);
-
-            assertFalse(e1.isEvicted());
-            assertFalse(e3.isEvicted());
-            assertFalse(e5.isEvicted());
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testRandomMultiThreaded() throws Exception {
-        try {
-            final Ignite g = startGrid(0);
-
-            int max = 10;
-
-            policy(0).setMaxSize(max);
-
-            final Random rand = new Random();
-
-            int keys = 31;
-
-            final String[] t = new String[keys];
-
-            for (int i = 0; i < t.length; i++)
-                t[i] = Integer.toString(i);
-
-            multithreaded(new Callable() {
-                @Nullable @Override public Object call() {
-                    int runs = 3000;
-
-                    for (int i = 0; i < runs; i++) {
-                        boolean rmv = rand.nextBoolean();
-
-                        int j = rand.nextInt(t.length);
-
-                        if (rmv)
-                            g.cache(null).remove(t[j]);
-                        else
-                            g.cache(null).put(t[j], t[j]);
-
-                        if (i != 0 && i % 1000 == 0)
-                            info("Stats [cntr=" + i + ", total=" + runs + ']');
-                    }
-
-                    return null;
-                }
-            }, 10);
-
-            assert g.cache(null).size() <= max;
-
-            info(policy(0));
-        }
-        finally {
-            stopAllGrids();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabled() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabled() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabledMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabledMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override protected RandomEvictionPolicy<String, String> createPolicy(int plcMax) {
-        return new RandomEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected RandomEvictionPolicy<String, String> createNearPolicy(int nearMax) {
-        return new RandomEvictionPolicy<>(plcMax);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkNearPolicies(int nearMax) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void checkPolicies(int plcMax) {
-        // No-op.
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicyCacheSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicyCacheSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicyCacheSizeSelfTest.java
index e9e7228..2a2004e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicyCacheSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicyCacheSizeSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.eviction.random;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.random.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.testframework.junits.common.*;
@@ -50,6 +51,7 @@ public class RandomEvictionPolicyCacheSizeSelfTest extends GridCommonAbstractTes
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         CacheConfiguration ccfg = defaultCacheConfiguration();
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
         ccfg.setNearConfiguration(null);
         ccfg.setEvictionPolicy(new RandomEvictionPolicy(PLC_MAX_SIZE));
 
@@ -67,6 +69,10 @@ public class RandomEvictionPolicyCacheSizeSelfTest extends GridCommonAbstractTes
         for (int i = 0; i < KEYS_CNT; i++)
             cache.put(i, i);
 
+        // Ensure that all entries accessed without data races and cache size will correct
+        for (int i = 0; i < KEYS_CNT; i++)
+            cache.get(i);
+
         assertEquals(PLC_MAX_SIZE * GRID_CNT, cache.size());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
new file mode 100644
index 0000000..ef34a13
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/random/RandomEvictionPolicySelfTest.java
@@ -0,0 +1,357 @@
+/*
+ * 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.eviction.random;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.eviction.random.*;
+import org.apache.ignite.internal.processors.cache.eviction.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Random eviction policy test.
+ */
+public class RandomEvictionPolicySelfTest extends
+    EvictionAbstractTest<RandomEvictionPolicy<String, String>> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMemory() throws Exception {
+        try {
+            Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            int keys = 31;
+
+            for (int i = 0; i < keys; i++) {
+                String s = Integer.toString(i);
+
+                g.cache(null).put(s, s);
+            }
+
+            assert g.cache(null).size() <= max;
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandom() throws Exception {
+        try {
+            Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            Random rand = new Random();
+
+            int keys = 31;
+
+            String[] t = new String[keys];
+
+            for (int i = 0; i < t.length; i++)
+                t[i] = Integer.toString(i);
+
+            int runs = 10000;
+
+            for (int i = 0; i < runs; i++) {
+                boolean rmv = rand.nextBoolean();
+
+                int j = rand.nextInt(t.length);
+
+                if (rmv)
+                    g.cache(null).remove(t[j]);
+                else
+                    g.cache(null).put(t[j], t[j]);
+
+                if (i % 1000 == 0)
+                    info("Stats [cntr=" + i + ", total=" + runs + ']');
+            }
+
+            assert g.cache(null).size() <= max;
+
+            info(policy(0));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAllowEmptyEntries() throws Exception {
+        try {
+            startGrid();
+
+            IgniteCache<String, String> c = jcache();
+
+            MockEntry e1 = new MockEntry("1", c);
+            MockEntry e2 = new MockEntry("2", c);
+            MockEntry e3 = new MockEntry("3", c);
+            MockEntry e4 = new MockEntry("4", c);
+            MockEntry e5 = new MockEntry("5", c);
+
+            RandomEvictionPolicy<String, String> p = policy();
+
+            p.setMaxSize(10);
+
+            p.onEntryAccessed(false, e1);
+
+            assertFalse(e1.isEvicted());
+
+            p.onEntryAccessed(false, e2);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e2.isEvicted());
+
+            p.onEntryAccessed(false, e3);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+
+            p.onEntryAccessed(false, e4);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e4.isEvicted());
+
+            p.onEntryAccessed(false, e5);
+
+            assertFalse(e1.isEvicted());
+            assertFalse(e3.isEvicted());
+            assertFalse(e5.isEvicted());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomMultiThreaded() throws Exception {
+        try {
+            final Ignite g = startGrid(0);
+
+            int max = 10;
+
+            policy(0).setMaxSize(max);
+
+            final Random rand = new Random();
+
+            int keys = 31;
+
+            final String[] t = new String[keys];
+
+            for (int i = 0; i < t.length; i++)
+                t[i] = Integer.toString(i);
+
+            multithreaded(new Callable() {
+                @Nullable @Override public Object call() {
+                    int runs = 3000;
+
+                    for (int i = 0; i < runs; i++) {
+                        boolean rmv = rand.nextBoolean();
+
+                        int j = rand.nextInt(t.length);
+
+                        if (rmv)
+                            g.cache(null).remove(t[j]);
+                        else
+                            g.cache(null).put(t[j], t[j]);
+
+                        if (i != 0 && i % 1000 == 0)
+                            info("Stats [cntr=" + i + ", total=" + runs + ']');
+                    }
+
+                    return null;
+                }
+            }, 10);
+
+            assert g.cache(null).size() <= max;
+
+            info(policy(0));
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeAllowEmptyEntries() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeMemory() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePartitionedNearDisabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePolicy() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizePut() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxMemSizeRandom() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeAllowEmptyEntries() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeAllowEmptyEntriesWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeMemory() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeMemoryWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicy() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePut() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePutWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeRandom() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizeRandomWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePartitionedNearDisabledWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void doTestPolicyWithBatch() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testMaxSizePartitionedNearDisabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabled() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearDisabledMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearDisabledBackupSyncMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabledMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testPartitionedNearEnabledBackupSyncMultiThreaded() throws Exception {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RandomEvictionPolicy<String, String> createPolicy(int plcMax) {
+        RandomEvictionPolicy<String, String> plc = new RandomEvictionPolicy<>();
+
+        plc.setMaxSize(plcMax);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected RandomEvictionPolicy<String, String> createNearPolicy(int nearMax) {
+        RandomEvictionPolicy<String, String> plc = new RandomEvictionPolicy<>();
+
+        plc.setMaxSize(plcMax);
+
+        return plc;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkNearPolicies(int nearMax) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void checkPolicies() {
+        // No-op.
+    }
+}


[04/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
new file mode 100644
index 0000000..7333020
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -0,0 +1,1196 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ * Client-based discovery tests.
+ */
+public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final AtomicInteger srvIdx = new AtomicInteger();
+
+    /** */
+    private static final AtomicInteger clientIdx = new AtomicInteger();
+
+    /** */
+    private static Collection<UUID> srvNodeIds;
+
+    /** */
+    private static Collection<UUID> clientNodeIds;
+
+    /** */
+    private static int clientsPerSrv;
+
+    /** */
+    private static CountDownLatch srvJoinedLatch;
+
+    /** */
+    private static CountDownLatch srvLeftLatch;
+
+    /** */
+    private static CountDownLatch srvFailedLatch;
+
+    /** */
+    private static CountDownLatch clientJoinedLatch;
+
+    /** */
+    private static CountDownLatch clientLeftLatch;
+
+    /** */
+    private static CountDownLatch clientFailedLatch;
+
+    /** */
+    private static CountDownLatch msgLatch;
+
+    /** */
+    private UUID nodeId;
+
+    /** */
+    private TcpDiscoveryVmIpFinder clientIpFinder;
+
+    /** */
+    private long joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT;
+
+    /** */
+    private long netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT;
+
+    /** */
+    private boolean longSockTimeouts;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TestTcpDiscoverySpi();
+
+        if (gridName.startsWith("server"))
+            disco.setIpFinder(IP_FINDER);
+        else if (gridName.startsWith("client")) {
+            cfg.setClientMode(true);
+
+            TcpDiscoveryVmIpFinder ipFinder;
+
+            if (clientIpFinder != null)
+                ipFinder = clientIpFinder;
+            else {
+                ipFinder = new TcpDiscoveryVmIpFinder();
+
+                String addr = new ArrayList<>(IP_FINDER.getRegisteredAddresses()).
+                    get((clientIdx.get() - 1) / clientsPerSrv).toString();
+
+                if (addr.startsWith("/"))
+                    addr = addr.substring(1);
+
+                ipFinder.setAddresses(Collections.singletonList(addr));
+            }
+
+            disco.setIpFinder(ipFinder);
+
+            String nodeId = cfg.getNodeId().toString();
+
+            nodeId = "cc" + nodeId.substring(2);
+
+            cfg.setNodeId(UUID.fromString(nodeId));
+        }
+        else
+            throw new IllegalArgumentException();
+
+        if (longSockTimeouts) {
+            disco.setAckTimeout(2000);
+            disco.setSocketTimeout(2000);
+        }
+
+        disco.setJoinTimeout(joinTimeout);
+        disco.setNetworkTimeout(netTimeout);
+
+        cfg.setDiscoverySpi(disco);
+
+        if (nodeId != null)
+            cfg.setNodeId(nodeId);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        Collection<InetSocketAddress> addrs = IP_FINDER.getRegisteredAddresses();
+
+        if (!F.isEmpty(addrs))
+            IP_FINDER.unregisterAddresses(addrs);
+
+        srvIdx.set(0);
+        clientIdx.set(0);
+
+        srvNodeIds = new GridConcurrentHashSet<>();
+        clientNodeIds = new GridConcurrentHashSet<>();
+
+        clientsPerSrv = 2;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllClients(true);
+        stopAllServers(true);
+
+        nodeId = null;
+        clientIpFinder = null;
+        joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT;
+        netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT;
+        longSockTimeouts = false;
+
+        assert G.allGrids().isEmpty();
+    }
+
+    /**
+     *
+     * @throws Exception
+     */
+    public void testJoinTimeout() throws Exception {
+        clientIpFinder = new TcpDiscoveryVmIpFinder();
+        joinTimeout = 1000;
+
+        try {
+            startClientNodes(1);
+
+            fail("Client cannot be start because no server nodes run");
+        }
+        catch (IgniteCheckedException e) {
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+
+            assert spiEx.getMessage().contains("Join process timed out") : spiEx.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeJoin() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvJoinedLatch = new CountDownLatch(3);
+        clientJoinedLatch = new CountDownLatch(3);
+
+        attachListeners(3, 3);
+
+        startClientNodes(1);
+
+        await(srvJoinedLatch);
+        await(clientJoinedLatch);
+
+        checkNodes(3, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeLeave() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvLeftLatch = new CountDownLatch(3);
+        clientLeftLatch = new CountDownLatch(2);
+
+        attachListeners(3, 3);
+
+        stopGrid("client-2");
+
+        await(srvLeftLatch);
+        await(clientLeftLatch);
+
+        checkNodes(3, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeFail() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvFailedLatch = new CountDownLatch(3);
+        clientFailedLatch = new CountDownLatch(2);
+
+        attachListeners(3, 3);
+
+        failClient(2);
+
+        await(srvFailedLatch);
+        await(clientFailedLatch);
+
+        checkNodes(3, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerNodeJoin() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvJoinedLatch = new CountDownLatch(3);
+        clientJoinedLatch = new CountDownLatch(3);
+
+        attachListeners(3, 3);
+
+        startServerNodes(1);
+
+        await(srvJoinedLatch);
+        await(clientJoinedLatch);
+
+        checkNodes(4, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerNodeLeave() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvLeftLatch = new CountDownLatch(2);
+        clientLeftLatch = new CountDownLatch(3);
+
+        attachListeners(3, 3);
+
+        stopGrid("server-2");
+
+        await(srvLeftLatch);
+        await(clientLeftLatch);
+
+        checkNodes(2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServerNodeFail() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvFailedLatch = new CountDownLatch(2);
+        clientFailedLatch = new CountDownLatch(3);
+
+        attachListeners(3, 3);
+
+        assert ((TcpDiscoverySpi)G.ignite("server-2").configuration().getDiscoverySpi()).clientWorkerCount() == 0;
+
+        failServer(2);
+
+        await(srvFailedLatch);
+        await(clientFailedLatch);
+
+        checkNodes(2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPing() throws Exception {
+        startServerNodes(2);
+        startClientNodes(1);
+
+        Ignite srv0 = G.ignite("server-0");
+        Ignite srv1 = G.ignite("server-1");
+        Ignite client = G.ignite("client-0");
+
+        assert ((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
+        assert ((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
+
+        assert ((IgniteEx)client).context().discovery().pingNode(srv0.cluster().localNode().id());
+        assert ((IgniteEx)client).context().discovery().pingNode(srv1.cluster().localNode().id());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPingFailedNodeFromClient() throws Exception {
+        startServerNodes(2);
+        startClientNodes(1);
+
+        Ignite srv0 = G.ignite("server-0");
+        Ignite srv1 = G.ignite("server-1");
+        Ignite client = G.ignite("client-0");
+
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        ((TcpDiscoverySpi)srv1.configuration().getDiscoverySpi()).addIncomeConnectionListener(new IgniteInClosure<Socket>() {
+            @Override public void apply(Socket sock) {
+                try {
+                    latch.await();
+                }
+                catch (InterruptedException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+        });
+
+        assert ((IgniteEx)client).context().discovery().pingNode(srv0.cluster().localNode().id());
+        assert !((IgniteEx)client).context().discovery().pingNode(srv1.cluster().localNode().id());
+
+        latch.countDown();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPingFailedClientNode() throws Exception {
+        startServerNodes(2);
+        startClientNodes(1);
+
+        Ignite srv0 = G.ignite("server-0");
+        Ignite srv1 = G.ignite("server-1");
+        Ignite client = G.ignite("client-0");
+
+        ((TcpDiscoverySpi)srv0.configuration().getDiscoverySpi()).setAckTimeout(1000);
+
+        ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi()).pauseSocketWrite();
+
+        assert !((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
+        assert !((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
+
+        ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi()).resumeAll();
+
+        assert ((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
+        assert ((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectOnRouterFail() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        setClientRouter(2, 0);
+
+        srvFailedLatch = new CountDownLatch(2);
+        clientFailedLatch = new CountDownLatch(3);
+
+        attachListeners(2, 3);
+
+        failServer(2);
+
+        await(srvFailedLatch);
+        await(clientFailedLatch);
+
+        checkNodes(2, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectOnNetworkProblem() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        setClientRouter(2, 0);
+
+        srvFailedLatch = new CountDownLatch(2);
+        clientFailedLatch = new CountDownLatch(3);
+
+        attachListeners(2, 3);
+
+        ((TcpDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi()).brakeConnection();
+
+        G.ignite("client-2").message().remoteListen(null, new MessageListener()); // Send some discovery message.
+
+        checkNodes(3, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectOneServerOneClient() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(1);
+        startClientNodes(1);
+
+        checkNodes(1, 1);
+
+        srvLeftLatch = new CountDownLatch(1);
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        ((TcpDiscoverySpi)G.ignite("client-0").configuration().getDiscoverySpi()).brakeConnection();
+
+        assertFalse(srvFailedLatch.await(2000, TimeUnit.MILLISECONDS));
+
+        assertEquals(1L, srvLeftLatch.getCount());
+
+        checkNodes(1, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetMissedMessagesOnReconnect() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(3);
+        startClientNodes(2);
+
+        checkNodes(3, 2);
+
+        clientLeftLatch = new CountDownLatch(1);
+        srvLeftLatch = new CountDownLatch(2);
+
+        attachListeners(2, 2);
+
+        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).pauseAll();
+
+        stopGrid("server-2");
+
+        await(srvLeftLatch);
+        await(srvLeftLatch);
+
+        Thread.sleep(500);
+
+        assert G.ignite("client-0").cluster().nodes().size() == 4;
+        assert G.ignite("client-1").cluster().nodes().size() == 5;
+
+        clientLeftLatch = new CountDownLatch(1);
+
+        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).resumeAll();
+
+        await(clientLeftLatch);
+
+        checkNodes(2, 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientSegmentation() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        srvFailedLatch = new CountDownLatch(2 + 2);
+        clientFailedLatch = new CountDownLatch(2 + 2);
+
+        attachListeners(2, 2);
+
+        final CountDownLatch client2StoppedLatch = new CountDownLatch(1);
+
+        IgnitionListener lsnr = new IgnitionListener() {
+            @Override public void onStateChange(@Nullable String name, IgniteState state) {
+                if (state == IgniteState.STOPPED_ON_SEGMENTATION)
+                    client2StoppedLatch.countDown();
+            }
+        };
+        G.addListener(lsnr);
+
+        final TcpDiscoverySpi disco = (TcpDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi();
+
+        try {
+            failServer(2);
+
+            await(srvFailedLatch);
+            await(clientFailedLatch);
+
+            await(client2StoppedLatch);
+
+            checkNodes(2, 2);
+        }
+        finally {
+            G.removeListener(lsnr);
+        }
+
+        assert disco.getRemoteNodes().isEmpty();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeJoinOneServer() throws Exception {
+        startServerNodes(1);
+
+        srvJoinedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        startClientNodes(1);
+
+        await(srvJoinedLatch);
+
+        checkNodes(1, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeLeaveOneServer() throws Exception {
+        startServerNodes(1);
+        startClientNodes(1);
+
+        checkNodes(1, 1);
+
+        srvLeftLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        stopGrid("client-0");
+
+        await(srvLeftLatch);
+
+        checkNodes(1, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeFailOneServer() throws Exception {
+        startServerNodes(1);
+        startClientNodes(1);
+
+        checkNodes(1, 1);
+
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        failClient(0);
+
+        await(srvFailedLatch);
+
+        checkNodes(1, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientAndRouterFail() throws Exception {
+        startServerNodes(2);
+        startClientNodes(2);
+
+        checkNodes(2, 2);
+
+        srvFailedLatch = new CountDownLatch(2);
+        clientFailedLatch = new CountDownLatch(2);
+
+        attachListeners(1, 1);
+
+        ((TcpDiscoverySpi)G.ignite("server-1").configuration().getDiscoverySpi()).addSendMessageListener(new IgniteInClosure<TcpDiscoveryAbstractMessage>() {
+            @Override public void apply(TcpDiscoveryAbstractMessage msg) {
+                try {
+                    Thread.sleep(1000000);
+                }
+                catch (InterruptedException ignored) {
+                    Thread.interrupted();
+                }
+            }
+        });
+        failClient(1);
+        failServer(1);
+
+        await(srvFailedLatch);
+        await(clientFailedLatch);
+
+        checkNodes(1, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMetrics() throws Exception {
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        attachListeners(3, 3);
+
+        assertTrue(checkMetrics(3, 3, 0));
+
+        G.ignite("client-0").compute().broadcast(F.noop());
+
+        assertTrue(GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                return checkMetrics(3, 3, 1);
+            }
+        }, 10000));
+
+        checkMetrics(3, 3, 1);
+
+        G.ignite("server-0").compute().broadcast(F.noop());
+
+        assertTrue(GridTestUtils.waitForCondition(new PA() {
+            @Override public boolean apply() {
+                return checkMetrics(3, 3, 2);
+            }
+        }, 10000));
+    }
+
+    /**
+     * @param srvCnt Number of Number of server nodes.
+     * @param clientCnt Number of client nodes.
+     * @param execJobsCnt Expected number of executed jobs.
+     * @return Whether metrics are correct.
+     */
+    private boolean checkMetrics(int srvCnt, int clientCnt, int execJobsCnt) {
+        for (int i = 0; i < srvCnt; i++) {
+            Ignite g = G.ignite("server-" + i);
+
+            for (ClusterNode n : g.cluster().nodes()) {
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
+                    return false;
+            }
+        }
+
+        for (int i = 0; i < clientCnt; i++) {
+            Ignite g = G.ignite("client-" + i);
+
+            for (ClusterNode n : g.cluster().nodes()) {
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
+                    return false;
+            }
+        }
+
+        return true;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDataExchangeFromServer() throws Exception {
+        testDataExchange("server-0");
+    }
+
+    /**
+     * TODO: IGNITE-587.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDataExchangeFromClient() throws Exception {
+        testDataExchange("client-0");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void testDataExchange(String masterName) throws Exception {
+        startServerNodes(2);
+        startClientNodes(2);
+
+        checkNodes(2, 2);
+
+        IgniteMessaging msg = grid(masterName).message();
+
+        UUID id = msg.remoteListen(null, new MessageListener());
+
+        try {
+            msgLatch = new CountDownLatch(2);
+
+            msg.send(null, "Message 1");
+
+            await(msgLatch);
+
+            startServerNodes(1);
+            startClientNodes(1);
+
+            checkNodes(3, 3);
+
+            msgLatch = new CountDownLatch(3);
+
+            msg.send(null, "Message 2");
+
+            await(msgLatch);
+        }
+        finally {
+            msg.stopRemoteListen(id);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDataExchangeFromServer2() throws Exception {
+        startServerNodes(2);
+
+        IgniteMessaging msg = grid("server-1").message();
+
+        UUID id = msg.remoteListen(null, new MessageListener());
+
+        try {
+            startClientNodes(1);
+
+            assertEquals(G.ignite("server-0").cluster().localNode().id(), ((TcpDiscoveryNode)G.ignite("client-0")
+                .cluster().localNode()).clientRouterNodeId());
+
+            checkNodes(2, 1);
+
+            msgLatch = new CountDownLatch(3);
+
+            msg.send(null, "Message");
+
+            await(msgLatch);
+        }
+        finally {
+            msg.stopRemoteListen(id);
+        }
+    }
+
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testDuplicateId() throws Exception {
+        startServerNodes(2);
+
+        nodeId = G.ignite("server-1").cluster().localNode().id();
+
+        try {
+            startGrid("client-0");
+
+            assert false;
+        }
+        catch (IgniteCheckedException e) {
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+            assert spiEx.getMessage().contains("same ID") : spiEx.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testTimeoutWaitingNodeAddedMessage() throws Exception {
+        longSockTimeouts = true;
+
+        startServerNodes(2);
+
+        final CountDownLatch cnt = new CountDownLatch(1);
+
+        ((TcpDiscoverySpi)G.ignite("server-1").configuration().getDiscoverySpi()).addSendMessageListener(
+            new IgniteInClosure<TcpDiscoveryAbstractMessage>() {
+                @Override public void apply(TcpDiscoveryAbstractMessage msg) {
+                    try {
+                        cnt.await(10, MINUTES);
+                    }
+                    catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+
+                        throw new IgniteInterruptedException(e);
+                    }
+                }
+            });
+
+        try {
+            netTimeout = 500;
+
+            startGrid("client-0");
+
+            assert false;
+        }
+        catch (IgniteCheckedException e) {
+            cnt.countDown();
+
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+            assert spiEx.getMessage().contains("Join process timed out") : spiEx.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testGridStartTime() throws Exception {
+        startServerNodes(2);
+
+        startClientNodes(2);
+
+        long startTime = -1;
+
+        for (Ignite g : G.allGrids()) {
+            IgniteEx kernal = (IgniteEx)g;
+
+            assertTrue(kernal.context().discovery().gridStartTime() > 0);
+
+            if (startTime == -1)
+                startTime = kernal.context().discovery().gridStartTime();
+            else
+                assertEquals(startTime, kernal.context().discovery().gridStartTime());
+        }
+    }
+
+    /**
+     * @param clientIdx Index.
+     * @throws Exception In case of error.
+     */
+    private void setClientRouter(int clientIdx, int srvIdx) throws Exception {
+        TcpDiscoverySpi disco =
+            (TcpDiscoverySpi)G.ignite("client-" + clientIdx).configuration().getDiscoverySpi();
+
+        TcpDiscoveryVmIpFinder ipFinder = (TcpDiscoveryVmIpFinder)disco.getIpFinder();
+
+        String addr = new ArrayList<>(IP_FINDER.getRegisteredAddresses()).get(srvIdx).toString();
+
+        if (addr.startsWith("/"))
+            addr = addr.substring(1);
+
+        ipFinder.setAddresses(Collections.singletonList(addr));
+    }
+
+    /**
+     * @param cnt Number of nodes.
+     * @throws Exception In case of error.
+     */
+    private void startServerNodes(int cnt) throws Exception {
+        for (int i = 0; i < cnt; i++) {
+            Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
+
+            srvNodeIds.add(g.cluster().localNode().id());
+        }
+    }
+
+    /**
+     * @param cnt Number of nodes.
+     * @throws Exception In case of error.
+     */
+    private void startClientNodes(int cnt) throws Exception {
+        for (int i = 0; i < cnt; i++) {
+            Ignite g = startGrid("client-" + clientIdx.getAndIncrement());
+
+            clientNodeIds.add(g.cluster().localNode().id());
+        }
+    }
+
+    /**
+     * @param idx Index.
+     */
+    private void failServer(int idx) {
+        ((TcpDiscoverySpi)G.ignite("server-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
+    }
+
+    /**
+     * @param idx Index.
+     */
+    private void failClient(int idx) {
+        ((TcpDiscoverySpi)G.ignite("client-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
+    }
+
+    /**
+     * @param srvCnt Number of server nodes.
+     * @param clientCnt Number of client nodes.
+     */
+    private void attachListeners(int srvCnt, int clientCnt) throws Exception {
+        if (srvJoinedLatch != null) {
+            for (int i = 0; i < srvCnt; i++) {
+                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Joined event fired on server: " + evt);
+
+                        srvJoinedLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_JOINED);
+            }
+        }
+
+        if (srvLeftLatch != null) {
+            for (int i = 0; i < srvCnt; i++) {
+                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Left event fired on server: " + evt);
+
+                        srvLeftLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_LEFT);
+            }
+        }
+
+        if (srvFailedLatch != null) {
+            for (int i = 0; i < srvCnt; i++) {
+                G.ignite("server-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Failed event fired on server: " + evt);
+
+                        srvFailedLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_FAILED);
+            }
+        }
+
+        if (clientJoinedLatch != null) {
+            for (int i = 0; i < clientCnt; i++) {
+                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Joined event fired on client: " + evt);
+
+                        clientJoinedLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_JOINED);
+            }
+        }
+
+        if (clientLeftLatch != null) {
+            for (int i = 0; i < clientCnt; i++) {
+                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Left event fired on client: " + evt);
+
+                        clientLeftLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_LEFT);
+            }
+        }
+
+        if (clientFailedLatch != null) {
+            for (int i = 0; i < clientCnt; i++) {
+                G.ignite("client-" + i).events().localListen(new IgnitePredicate<Event>() {
+                    @Override public boolean apply(Event evt) {
+                        info("Failed event fired on client: " + evt);
+
+                        clientFailedLatch.countDown();
+
+                        return true;
+                    }
+                }, EVT_NODE_FAILED);
+            }
+        }
+    }
+
+    /**
+     * @param srvCnt Number of server nodes.
+     * @param clientCnt Number of client nodes.
+     */
+    private void checkNodes(int srvCnt, int clientCnt) {
+        for (int i = 0; i < srvCnt; i++) {
+            Ignite g = G.ignite("server-" + i);
+
+            assertTrue(srvNodeIds.contains(g.cluster().localNode().id()));
+
+            assertFalse(g.cluster().localNode().isClient());
+
+            checkRemoteNodes(g, srvCnt + clientCnt - 1);
+        }
+
+        for (int i = 0; i < clientCnt; i++) {
+            Ignite g = G.ignite("client-" + i);
+
+            ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
+
+            assertTrue(clientNodeIds.contains(g.cluster().localNode().id()));
+
+            assertTrue(g.cluster().localNode().isClient());
+
+            checkRemoteNodes(g, srvCnt + clientCnt - 1);
+        }
+    }
+
+    /**
+     * @param ignite Grid.
+     * @param expCnt Expected nodes count.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    private void checkRemoteNodes(Ignite ignite, int expCnt) {
+        Collection<ClusterNode> nodes = ignite.cluster().forRemotes().nodes();
+
+        assertEquals(expCnt, nodes.size());
+
+        for (ClusterNode node : nodes) {
+            UUID id = node.id();
+
+            if (clientNodeIds.contains(id))
+                assertTrue(node.isClient());
+            else if (srvNodeIds.contains(id))
+                assertFalse(node.isClient());
+            else
+                assert false : "Unexpected node ID: " + id;
+        }
+    }
+
+    /**
+     * @param latch Latch.
+     * @throws InterruptedException If interrupted.
+     */
+    private void await(CountDownLatch latch) throws InterruptedException {
+        assertTrue("Latch count: " + latch.getCount(), latch.await(10000, MILLISECONDS));
+    }
+
+    /**
+     */
+    private static class MessageListener implements IgniteBiPredicate<UUID, Object> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, Object msg) {
+            X.println(">>> Received [locNodeId=" + ignite.configuration().getNodeId() + ", msg=" + msg + ']');
+
+            msgLatch.countDown();
+
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** */
+        private final Object mux = new Object();
+
+        /** */
+        private final AtomicBoolean writeLock = new AtomicBoolean();
+
+        /** */
+        private final AtomicBoolean openSockLock = new AtomicBoolean();
+
+        /**
+         * @param lock Lock.
+         */
+        private void waitFor(AtomicBoolean lock) {
+            try {
+                synchronized (mux) {
+                    while (lock.get())
+                        mux.wait();
+                }
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new RuntimeException(e);
+            }
+        }
+
+        /**
+         * @param isPause Is lock.
+         * @param locks Locks.
+         */
+        private void pauseResumeOperation(boolean isPause, AtomicBoolean... locks) {
+            synchronized (mux) {
+                for (AtomicBoolean lock : locks)
+                    lock.set(isPause);
+
+                mux.notifyAll();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg,
+            GridByteArrayOutputStream bout) throws IOException, IgniteCheckedException {
+            waitFor(writeLock);
+
+            super.writeToSocket(sock, msg, bout);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Socket openSocket(InetSocketAddress sockAddr) throws IOException {
+            waitFor(openSockLock);
+
+            return super.openSocket(sockAddr);
+        }
+
+        /**
+         *
+         */
+        public void pauseSocketWrite() {
+            pauseResumeOperation(true, writeLock);
+        }
+
+        /**
+         *
+         */
+        public void pauseAll() {
+            pauseResumeOperation(true, openSockLock, writeLock);
+
+            impl.workerThread().suspend();
+        }
+
+        /**
+         *
+         */
+        public void resumeAll() {
+            pauseResumeOperation(false, openSockLock, writeLock);
+
+            impl.workerThread().resume();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
index b5d02f0..6438268 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
@@ -20,16 +20,19 @@ package org.apache.ignite.spi.discovery.tcp;
 import org.apache.ignite.configuration.*;
 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.*;
 
 /**
  * Test for {@link TcpDiscoverySpi}.
  */
 public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
     /** */
-    private static final int TOP_SIZE = 1;
+    private static final int TOP_SIZE = 3;
 
     /** */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
@@ -41,34 +44,12 @@ public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
-        if (client) {
-            TcpDiscoveryVmIpFinder clientIpFinder = new TcpDiscoveryVmIpFinder();
-
-            String addr = new ArrayList<>(ipFinder.getRegisteredAddresses()).iterator().next().toString();
-
-            if (addr.startsWith("/"))
-                addr = addr.substring(1);
-
-            clientIpFinder.setAddresses(Arrays.asList(addr));
-
-            TcpClientDiscoverySpi discoSpi = new TcpClientDiscoverySpi();
-
-            discoSpi.setIpFinder(clientIpFinder);
-
-            cfg.setDiscoverySpi(discoSpi);
-        }
-        else {
-            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-            discoSpi.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(discoSpi);
-        }
-
-        cfg.setLocalHost("127.0.0.1");
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
 
         cfg.setCacheConfiguration();
 
+        cfg.setClientMode(client);
+
         return cfg;
     }
 
@@ -77,11 +58,16 @@ public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
         return Long.MAX_VALUE;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        client = false;
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testConcurrentStart() throws Exception {
-        for (int i = 0; i < 50; i++) {
+        for (int i = 0; i < 10; i++) {
             try {
                 startGridsMultiThreaded(TOP_SIZE);
             }
@@ -95,15 +81,28 @@ public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentStartClients() throws Exception {
-        for (int i = 0; i < 50; i++) {
+        for (int i = 0; i < 20; i++) {
             try {
                 client = false;
 
-                startGrid();
+                startGrid(0);
 
                 client = true;
 
-                startGridsMultiThreaded(TOP_SIZE);
+                final AtomicInteger gridIdx = new AtomicInteger(1);
+
+                GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                        @Nullable @Override public Object call() throws Exception {
+                            startGrid(gridIdx.getAndIncrement());
+
+                            return null;
+                        }
+                    },
+                    TOP_SIZE,
+                    "grid-starter-" + getName()
+                );
+
+                checkTopology(TOP_SIZE + 1);
             }
             finally {
                 stopAllGrids();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
index a2d8276..cfefff4 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
@@ -70,20 +70,10 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (client()) {
-            TcpClientDiscoverySpi spi = new TcpClientDiscoverySpi();
+        if (client())
+            cfg.setClientMode(true);
 
-            spi.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(spi);
-        }
-        else {
-            TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-            spi.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(spi);
-        }
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
 
         cfg.setCacheConfiguration();
 
@@ -91,8 +81,6 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
         cfg.setIncludeProperties();
 
-        cfg.setLocalHost("127.0.0.1");
-
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 5648c31..ad12753 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -175,7 +175,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
                 }
             }, 4, "grid-starter");
 
-            Collection<TcpDiscoveryNode> nodes = discoMap.get(g1.name()).ring().allNodes();
+            Collection<TcpDiscoveryNode> nodes = ((ServerImpl)discoMap.get(g1.name()).impl).ring().allNodes();
 
             ByteArrayOutputStream bos = new ByteArrayOutputStream();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
index 752e43c..04f9b41 100644
--- a/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/stream/socket/SocketStreamerSelfTest.java
@@ -58,14 +58,11 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
     /** Port. */
     private static int port;
 
-    /** Ignite. */
-    private static Ignite ignite;
-
     /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration() throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration();
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        CacheConfiguration ccfg = cacheConfiguration(cfg, null);
+        CacheConfiguration ccfg = defaultCacheConfiguration();
 
         cfg.setCacheConfiguration(ccfg);
 
@@ -81,8 +78,7 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
-        ignite = startGrids(GRID_CNT);
-        ignite.<Integer, String>getOrCreateCache(defaultCacheConfiguration());
+        startGrids(GRID_CNT);
 
         try (ServerSocket sock = new ServerSocket(0)) {
             port = sock.getLocalPort();
@@ -94,11 +90,6 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
         stopAllGrids();
     }
 
-    /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        ignite.cache(null).clear();
-    }
-
     /**
      * @throws Exception If failed.
      */
@@ -235,6 +226,12 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
     {
         SocketStreamer<Tuple, Integer, String> sockStmr = null;
 
+        Ignite ignite = grid(0);
+
+        IgniteCache<Integer, String> cache = ignite.cache(null);
+
+        cache.clear();
+
         try (IgniteDataStreamer<Integer, String> stmr = ignite.dataStreamer(null)) {
 
             stmr.allowOverwrite(true);
@@ -242,8 +239,6 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
             sockStmr = new SocketStreamer<>();
 
-            IgniteCache<Integer, String> cache = ignite.cache(null);
-
             sockStmr.setIgnite(ignite);
 
             sockStmr.setStreamer(stmr);
@@ -279,10 +274,10 @@ public class SocketStreamerSelfTest extends GridCommonAbstractTest {
 
             latch.await();
 
-            assertEquals(CNT, cache.size(CachePeekMode.PRIMARY));
-
             for (int i = 0; i < CNT; i++)
                 assertEquals(Integer.toString(i), cache.get(i));
+
+            assertEquals(CNT, cache.size(CachePeekMode.PRIMARY));
         }
         finally {
             if (sockStmr != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index bc04f90..21f9424 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -29,7 +29,7 @@ import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.swapspace.*;
+
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -447,28 +447,11 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public void writeToSwap(String spaceName, Object key, @Nullable Object val,
-        @Nullable ClassLoader ldr) {
-        /* No-op. */
-    }
-
-    /** {@inheritDoc} */
-    @Override public <T> T readFromSwap(String spaceName, SwapKey key, @Nullable ClassLoader ldr) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public int partition(String cacheName, Object key) {
         return -1;
     }
 
     /** {@inheritDoc} */
-    @Override public void removeFromSwap(String spaceName, Object key,
-        @Nullable ClassLoader ldr) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Nullable @Override public IgniteNodeValidationResult validateNode(ClusterNode node) {
         return null;
     }
@@ -484,12 +467,6 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <T> T readValueFromOffheapAndSwap(@Nullable String spaceName, Object key,
-        @Nullable ClassLoader ldr) {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public MessageFormatter messageFormatter() {
         if (formatter == null) {
             formatter = new MessageFormatter() {
@@ -524,6 +501,16 @@ public class GridSpiTestContext implements IgniteSpiContext {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+        // No-op.
+    }
+
     /**
      * @param cacheName Cache name.
      * @return Map representing cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index e25aaee..435ff46 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -1495,4 +1495,19 @@ public final class GridTestUtils {
     public static String apacheIgniteTestPath() {
         return System.getProperty("IGNITE_TEST_PATH", U.getIgniteHome() + "/target/ignite");
     }
+
+    /**
+     * {@link Class#getSimpleName()} does not return outer class name prefix for inner classes, for example,
+     * getSimpleName() returns "RegularDiscovery" instead of "GridDiscoveryManagerSelfTest$RegularDiscovery"
+     * This method return correct simple name for inner classes.
+     *
+     * @param cls Class
+     * @return Simple name with outer class prefix.
+     */
+    public static String fullSimpleName(@NotNull Class cls) {
+        if (cls.getEnclosingClass() != null)
+            return cls.getEnclosingClass().getSimpleName() + "." + cls.getSimpleName();
+        else
+            return cls.getSimpleName();
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 f3a9051..9c42920 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
@@ -455,7 +455,7 @@ public abstract class GridAbstractTest extends TestCase {
         }
 
         if (isFirstTest()) {
-            info(">>> Starting test class: " + getClass().getSimpleName() + " <<<");
+            info(">>> Starting test class: " + GridTestUtils.fullSimpleName(getClass()) + " <<<");
 
             if (startGrid) {
                 IgniteConfiguration cfg = optimize(getConfiguration());
@@ -676,8 +676,12 @@ public abstract class GridAbstractTest extends TestCase {
     protected IgniteConfiguration optimize(IgniteConfiguration cfg) throws IgniteCheckedException {
         // TODO: IGNITE-605: propose another way to avoid network overhead in tests.
         if (cfg.getLocalHost() == null) {
-            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpi)
+            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpi) {
                 cfg.setLocalHost("127.0.0.1");
+
+                if (((TcpDiscoverySpi)cfg.getDiscoverySpi()).getJoinTimeout() == 0)
+                    ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(8000);
+            }
             else
                 cfg.setLocalHost(getTestResources().getLocalHost());
         }
@@ -732,9 +736,20 @@ public abstract class GridAbstractTest extends TestCase {
      * @param cancel Cancel flag.
      */
     protected void stopAllGrids(boolean cancel) {
-        List<Ignite> ignites = G.allGrids();
+        Collection<Ignite> clients = new ArrayList<>();
+        Collection<Ignite> srvs = new ArrayList<>();
+
+        for (Ignite g : G.allGrids()) {
+            if (g.configuration().getDiscoverySpi().isClientMode())
+                clients.add(g);
+            else
+                srvs.add(g);
+        }
+
+        for (Ignite g : clients)
+            stopGrid(g.name(), cancel);
 
-        for (Ignite g : ignites)
+        for (Ignite g : srvs)
             stopGrid(g.name(), cancel);
 
         assert G.allGrids().isEmpty();
@@ -1004,17 +1019,6 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
-     * This method should be overridden by subclasses to change configuration parameters.
-     *
-     * @return Grid configuration used for starting of grid.
-     * @param rsrcs Resources.
-     * @throws Exception If failed.
-     */
-    protected IgniteConfiguration getConfiguration(IgniteTestResources rsrcs) throws Exception {
-        return getConfiguration(getTestGridName(), rsrcs);
-    }
-
-    /**
      * @return Generated unique test grid name.
      */
     public String getTestGridName() {
@@ -1201,7 +1205,7 @@ public abstract class GridAbstractTest extends TestCase {
             serializedObj.clear();
 
             if (isLastTest()) {
-                info(">>> Stopping test class: " + getClass().getSimpleName() + " <<<");
+                info(">>> Stopping test class: " + GridTestUtils.fullSimpleName(getClass()) + " <<<");
 
                 TestCounters counters = getTestCounters();
 
@@ -1389,6 +1393,22 @@ public abstract class GridAbstractTest extends TestCase {
 
     /**
      * @param obj Object that should be wrap proxy
+     * @return Created proxy.
+     */
+    protected <T> T notSerializableProxy(final T obj) {
+        Class<T> cls = (Class<T>)obj.getClass();
+
+        Class<T>[] interfaces = (Class<T>[])cls.getInterfaces();
+
+        assert interfaces.length > 0;
+
+        Class<T> lastItf = interfaces[interfaces.length - 1];
+
+        return notSerializableProxy(obj, lastItf, Arrays.copyOf(interfaces, interfaces.length - 1));
+    }
+
+    /**
+     * @param obj Object that should be wrap proxy
      * @param itfCls Interface that should be implemented by proxy
      * @param itfClses Interfaces that should be implemented by proxy (vararg parameter)
      * @return Created proxy.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
index 0709880..31cbefa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestCacheSession.java
@@ -34,6 +34,9 @@ public class TestCacheSession implements CacheStoreSession {
     /** */
     private Map<Object, Object> props;
 
+    /** */
+    private Object attachment;
+
     /**
      *
      * @param tx Transaction.
@@ -55,6 +58,21 @@ public class TestCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
+    @Override public Object attach(@Nullable Object attachment) {
+        Object prev = this.attachment;
+
+        this.attachment = attachment;
+
+        return prev;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T attachment() {
+        return (T)attachment;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> Map<K, V> properties() {
         if (props == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
index 2bbcf1b..dc876d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/TestThreadLocalCacheSession.java
@@ -54,6 +54,21 @@ public class TestThreadLocalCacheSession implements CacheStoreSession {
     }
 
     /** {@inheritDoc} */
+    @Override public Object attach(@Nullable Object attachment) {
+        TestCacheSession ses = sesHolder.get();
+
+        return ses != null ? ses.attach(attachment) : null;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Nullable @Override public <T> T attachment() {
+        TestCacheSession ses = sesHolder.get();
+
+        return ses!= null ? (T)ses.attachment() : null;
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <K, V> Map<K, V> properties() {
         TestCacheSession ses = sesHolder.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 5533897..d3535b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.local.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -383,14 +384,32 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                             int actual = owners.size();
 
                             if (affNodes.size() != owners.size() || !affNodes.containsAll(owners)) {
-                                LT.warn(log(), null, "Waiting for topology map update [grid=" + g.name() +
-                                    ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
-                                    ", affNodes=" + affNodes + ", owners=" + owners +
-                                    ", locNode=" + g.cluster().localNode().id() + ']');
+                                LT.warn(log(), null, "Waiting for topology map update [" +
+                                    "grid=" + g.name() +
+                                    ", cache=" + cfg.getName() +
+                                    ", cacheId=" + dht.context().cacheId() +
+                                    ", p=" + p +
+                                    ", affNodesCnt=" + exp +
+                                    ", ownersCnt=" + actual +
+                                    ", affNodes=" + affNodes +
+                                    ", owners=" + owners +
+                                    ", locNode=" + g.cluster().localNode() + ']');
 
                                 if (i == 0)
                                     start = System.currentTimeMillis();
 
+                                if (System.currentTimeMillis() - start > 30_000)
+                                    throw new IgniteException("Timeout of waiting for topology map update [" +
+                                        "grid=" + g.name() +
+                                        ", cache=" + cfg.getName() +
+                                        ", cacheId=" + dht.context().cacheId() +
+                                        ", p=" + p +
+                                        ", affNodesCnt=" + exp +
+                                        ", ownersCnt=" + actual +
+                                        ", affNodes=" + affNodes +
+                                        ", owners=" + owners +
+                                        ", locNode=" + g.cluster().localNode() + ']');
+
                                 Thread.sleep(200); // Busy wait.
 
                                 continue;
@@ -409,6 +428,38 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param ignite Node.
+     */
+    public void dumpCacheDebugInfo(Ignite ignite) {
+        GridKernalContext ctx = ((IgniteKernal)ignite).context();
+
+        log.error("Cache information update [node=" + ignite.name() +
+            ", client=" + ignite.configuration().isClientMode() + ']');
+
+        GridCacheSharedContext cctx = ctx.cache().context();
+
+        log.error("Pending transactions:");
+
+        for (IgniteInternalTx tx : cctx.tm().activeTransactions())
+            log.error(">>> " + tx);
+
+        log.error("Pending explicit locks:");
+
+        for (GridCacheExplicitLockSpan lockSpan : cctx.mvcc().activeExplicitLocks())
+            log.error(">>> " + lockSpan);
+
+        log.error("Pending cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().activeFutures())
+            log.error(">>> " + fut);
+
+        log.error("Pending atomic cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().atomicFutures())
+            log.error(">>> " + fut);
+    }
+
+    /**
      * @param cache Cache.
      * @return Affinity.
      */
@@ -858,4 +909,28 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
             ccfg.getAtomicWriteOrderMode() == CacheAtomicWriteOrderMode.CLOCK)
             U.sleep(50);
     }
+
+    /**
+     * @param exp Expected.
+     * @param act Actual.
+     */
+    protected void assertEqualsCollections(Collection<?> exp, Collection<?> act) {
+        if (exp.size() != act.size())
+            fail("Collections are not equal:\nExpected:\t" + exp + "\nActual:\t" + act);
+
+        Iterator<?> it1 = exp.iterator();
+        Iterator<?> it2 = act.iterator();
+
+        int idx = 0;
+
+        while (it1.hasNext()) {
+            Object item1 = it1.next();
+            Object item2 = it2.next();
+
+            if (!F.eq(item1, item2))
+                fail("Collections are not equal (position " + idx + "):\nExpected: " + exp + "\nActual:   " + act);
+
+            idx++;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 6382059..cc3abb4 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -48,11 +48,13 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTest(IgniteStartUpTestSuite.suite());
         suite.addTest(IgniteExternalizableSelfTestSuite.suite());
         suite.addTest(IgniteP2PSelfTestSuite.suite());
-        suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuit.suite());
+        suite.addTest(IgniteCacheP2pUnmarshallingErrorTestSuite.suite());
+        suite.addTest(IgniteStreamSelfTestSuite.suite());
 
         suite.addTest(new TestSuite(GridSelfTest.class));
         suite.addTest(new TestSuite(GridProjectionSelfTest.class));
         suite.addTest(new TestSuite(GridMessagingSelfTest.class));
+        suite.addTest(new TestSuite(IgniteMessagingWithClientTest.class));
         suite.addTest(new TestSuite(GridMessagingNoPeerClassLoadingSelfTest.class));
 
         if (U.isLinux() || U.isMacOs())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 4c40e75..7fb4563 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
@@ -112,6 +112,9 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteDataStructureUniqueNameTest.class));
 
+        suite.addTest(new TestSuite(IgniteClientDataStructuresTest.class));
+        suite.addTest(new TestSuite(IgniteClientDiscoveryDataStructuresTest.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
index 0852496..7f07d3d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheEvictionSelfTestSuite.java
@@ -37,15 +37,13 @@ public class IgniteCacheEvictionSelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Ignite Cache Eviction Test Suite");
 
-        suite.addTest(new TestSuite(GridCacheFifoEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheFifoBatchEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheSortedEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheSortedBatchEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheLruEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheLruNearEvictionPolicySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheNearOnlyLruNearEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(FifoEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(SortedEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruNearEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(LruNearOnlyNearEvictionPolicySelfTest.class));
+        suite.addTest(new TestSuite(RandomEvictionPolicySelfTest.class));
         suite.addTest(new TestSuite(RandomEvictionPolicyCacheSizeSelfTest.class));
-        suite.addTest(new TestSuite(GridCacheRandomEvictionPolicySelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearEvictionSelfTest.class));
         suite.addTest(new TestSuite(GridCacheAtomicNearEvictionSelfTest.class));
         suite.addTest(new TestSuite(GridCacheEvictionFilterSelfTest.class));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
index dd3ce27..c12ff66 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFailoverTestSuite.java
@@ -23,7 +23,6 @@ import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
  * Test suite.
@@ -37,15 +36,18 @@ public class IgniteCacheFailoverTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Cache Failover Test Suite");
 
         suite.addTestSuite(GridCacheAtomicInvalidPartitionHandlingSelfTest.class);
+        suite.addTestSuite(GridCacheAtomicClientInvalidPartitionHandlingSelfTest.class);
 
         suite.addTestSuite(GridCacheIncrementTransformTest.class);
 
         // Failure consistency tests.
         suite.addTestSuite(GridCacheAtomicRemoveFailureTest.class);
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderRemoveFailureTest.class);
+        suite.addTestSuite(GridCacheAtomicClientRemoveFailureTest.class);
 
         suite.addTestSuite(GridCacheDhtAtomicRemoveFailureTest.class);
         suite.addTestSuite(GridCacheDhtRemoveFailureTest.class);
+        suite.addTestSuite(GridCacheDhtClientRemoveFailureTest.class);
         suite.addTestSuite(GridCacheNearRemoveFailureTest.class);
         suite.addTestSuite(GridCacheAtomicNearRemoveFailureTest.class);
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderNearRemoveFailureTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 369e041..5fa3283 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -130,12 +130,6 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledAtomicOffHeapTieredMultiNodeFullApiSelfTest.class);
 
-        // Private cache API.
-        suite.addTestSuite(GridCacheExLocalFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExReplicatedFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExNearFullApiSelfTest.class);
-        suite.addTestSuite(GridCacheExColocatedFullApiSelfTest.class);
-
 //        Multithreaded // TODO: GG-708
 //        suite.addTestSuite(GridCacheLocalFullApiMultithreadedSelfTest.class);
 //        suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
index 1adf55f..9a0e5fc 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheMetricsSelfTestSuite.java
@@ -39,6 +39,7 @@ public class IgniteCacheMetricsSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedMetricsSelfTest.class);
         suite.addTestSuite(GridCachePartitionedMetricsSelfTest.class);
         suite.addTestSuite(GridCachePartitionedHitsAndMissesSelfTest.class);
+        suite.addTestSuite(CacheLocalOffHeapAndSwapMetricsSelfTest.class);
 
         // Atomic cache.
         suite.addTestSuite(GridCacheAtomicLocalMetricsSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheNearOnlySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheNearOnlySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheNearOnlySelfTestSuite.java
index a010c5d..7d28578 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheNearOnlySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheNearOnlySelfTestSuite.java
@@ -20,7 +20,6 @@ package org.apache.ignite.testsuites;
 import junit.framework.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.*;
 
 /**
  * Test suite for near-only cache.
@@ -33,12 +32,17 @@ public class IgniteCacheNearOnlySelfTestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Near-only cache test suite.");
 
-        suite.addTest(new TestSuite(GridCacheNearOnlySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheAtomicNearOnlySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheClientOnlySelfTest.class));
+        suite.addTest(new TestSuite(GridCacheClientOnlySelfTest.CasePartitionedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientOnlySelfTest.CasePartitionedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheClientOnlySelfTest.CaseReplicatedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientOnlySelfTest.CaseReplicatedTransactional.class));
+
         suite.addTest(new TestSuite(GridCacheNearOnlyTopologySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheReplicatedClientOnlySelfTest.class));
-        suite.addTest(new TestSuite(GridCacheReplicatedNearOnlySelfTest.class));
+
+        suite.addTest(new TestSuite(GridCacheNearOnlySelfTest.CasePartitionedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheNearOnlySelfTest.CasePartitionedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheNearOnlySelfTest.CaseReplicatedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheNearOnlySelfTest.CaseReplicatedTransactional.class));
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java
new file mode 100644
index 0000000..f54f85a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheP2pUnmarshallingErrorTestSuite.java
@@ -0,0 +1,41 @@
+/*
+ * 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.*;
+
+/**
+ * Checks behavior on exception while unmarshalling key.
+ */
+public class IgniteCacheP2pUnmarshallingErrorTestSuite extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("P2p Unmarshalling Test Suite");
+
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingNearErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingRebalanceErrorTest.class);
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingTxErrorTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTcpClientDiscoveryTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTcpClientDiscoveryTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTcpClientDiscoveryTestSuite.java
new file mode 100644
index 0000000..8c046f0
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTcpClientDiscoveryTestSuite.java
@@ -0,0 +1,47 @@
+/*
+ * 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.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+
+/**
+ * Tests a cache with TcpClientDiscovery SPI being enabled.
+ */
+public class IgniteCacheTcpClientDiscoveryTestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Cache + TcpClientDiscovery SPI test suite.");
+
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseNearPartitionedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseNearPartitionedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseNearReplicatedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseNearReplicatedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseClientPartitionedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseClientPartitionedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseClientReplicatedAtomic.class));
+        suite.addTest(new TestSuite(GridCacheClientModesTcpClientDiscoveryAbstractTest.CaseClientReplicatedTransactional.class));
+        suite.addTest(new TestSuite(GridCacheTcpClientDiscoveryMultiThreadedTest.class));
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 3fa3d9d..7e4409d 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
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import junit.framework.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.fair.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cache.store.jdbc.*;
@@ -85,6 +86,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteFairAffinityDynamicCacheSelfTest.class);
         suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class);
         suite.addTestSuite(IgniteCacheAffinitySelfTest.class);
+        suite.addTestSuite(IgniteClientNodeAffinityTest.class);
 
         // Swap tests.
         suite.addTestSuite(GridCacheSwapPreloadSelfTest.class);
@@ -109,6 +111,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
         suite.addTestSuite(DataStreamProcessorSelfTest.class);
         suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+        suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
         suite.addTestSuite(DataStreamerImplSelfTest.class);
         suite.addTestSuite(GridCacheEntryMemorySizeSelfTest.class);
         suite.addTestSuite(GridCacheClearAllSelfTest.class);
@@ -146,6 +149,8 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheTxPartitionedLocalStoreSelfTest.class);
         suite.addTestSuite(IgniteCacheSystemTransactionsSelfTest.class);
 
+        suite.addTest(IgniteCacheTcpClientDiscoveryTestSuite.suite());
+
         // Heuristic exception handling. TODO IGNITE-257
 //        suite.addTestSuite(GridCacheColocatedTxExceptionSelfTest.class);
 //        suite.addTestSuite(GridCacheReplicatedTxExceptionSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 7fa0a03..037af9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -130,15 +130,14 @@ public class IgniteCacheTestSuite2 extends TestSuite {
 
         suite.addTest(new TestSuite(GridCacheOffheapUpdateSelfTest.class));
 
-        // TODO: GG-7242, GG-7243: Enabled when fixed.
-//        suite.addTest(new TestSuite(GridCacheDhtRemoveFailureTest.class));
-//        suite.addTest(new TestSuite(GridCacheNearRemoveFailureTest.class));
-        // TODO: GG-7201: Enable when fixed.
-        //suite.addTest(new TestSuite(GridCacheDhtAtomicRemoveFailureTest.class));
-
         suite.addTest(new TestSuite(GridCacheNearPrimarySyncSelfTest.class));
         suite.addTest(new TestSuite(GridCacheColocatedPrimarySyncSelfTest.class));
 
+        suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
+        suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
+        suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
+        suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 aaf7e5b..713c5e5 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
@@ -18,6 +18,7 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
@@ -98,9 +99,11 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheTxPreloadNoWriteTest.class);
 
         suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class);
+        suite.addTestSuite(IgniteDynamicCacheWithConfigStartSelfTest.class);
         suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class);
         suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class);
         suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class);
+        suite.addTestSuite(IgniteDynamicClientCacheStartSelfTest.class);
 
         suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 
@@ -125,11 +128,16 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteExchangeFutureHistoryTest.class);
 
         suite.addTestSuite(CacheNoValueClassOnServerNodeTest.class);
+        suite.addTestSuite(IgniteSystemCacheOnClientTest.class);
 
         suite.addTestSuite(CacheRemoveAllSelfTest.class);
 
         suite.addTestSuite(CacheOffheapMapEntrySelfTest.class);
 
+        suite.addTestSuite(CacheJdbcStoreSessionListenerSelfTest.class);
+
+        suite.addTestSuite(CacheReadOnlyTransactionalClientSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index ebd3202..f0080d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@ -55,9 +55,11 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheMessageSelfTest.class);
         suite.addTestSuite(GridDeploymentManagerStopSelfTest.class);
         suite.addTestSuite(GridManagerStopSelfTest.class);
-        suite.addTestSuite(GridDiscoveryManagerAttributesSelfTest.class);
+        suite.addTestSuite(GridDiscoveryManagerAttributesSelfTest.RegularDiscovery.class);
+        suite.addTestSuite(GridDiscoveryManagerAttributesSelfTest.ClientDiscovery.class);
         suite.addTestSuite(GridDiscoveryManagerAliveCacheSelfTest.class);
-        suite.addTestSuite(GridDiscoveryManagerSelfTest.class);
+        suite.addTestSuite(GridDiscoveryManagerSelfTest.RegularDiscovery.class);
+        suite.addTestSuite(GridDiscoveryManagerSelfTest.ClientDiscovery.class);
         suite.addTestSuite(GridDiscoveryEventSelfTest.class);
         suite.addTestSuite(GridPortProcessorSelfTest.class);
         suite.addTestSuite(GridHomePathSelfTest.class);
@@ -76,6 +78,7 @@ public class IgniteKernalSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridServiceProcessorMultiNodeConfigSelfTest.class);
         suite.addTestSuite(GridServiceProcessorProxySelfTest.class);
         suite.addTestSuite(GridServiceReassignmentSelfTest.class);
+        suite.addTestSuite(GridServiceClientNodeTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 09ba42b..ea5a7ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -51,7 +51,9 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(GridTcpSpiForwardingSelfTest.class));
 
-        suite.addTest(new TestSuite(TcpClientDiscoverySelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoverySpiSelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoveryMarshallerCheckSelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoverySpiMulticastTest.class));
 
         return suite;
     }


[28/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
ignite-545: merge from sprint-6


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

Branch: refs/heads/ignite-545
Commit: 1652fd181f0f94388d4bf10bf87223c9ebef2269
Parents: cd96383
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jun 10 17:08:54 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jun 10 17:08:54 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   42 +-
 assembly/dependencies-visor-console.xml         |    3 +
 .../hibernate/CacheHibernatePersonStore.java    |  202 +-
 .../hibernate/CacheHibernateStoreExample.java   |   17 +
 .../store/jdbc/CacheJdbcPersonStore.java        |  180 +-
 .../store/jdbc/CacheJdbcStoreExample.java       |   13 +
 .../store/spring/CacheSpringPersonStore.java    |  128 +
 .../store/spring/CacheSpringStoreExample.java   |  143 +
 .../datagrid/store/spring/package-info.java     |   22 +
 .../java/org/apache/ignite/IgniteCache.java     |   16 +
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |  187 +-
 .../ignite/cache/eviction/EvictableEntry.java   |    7 +
 .../ignite/cache/eviction/EvictionPolicy.java   |    2 +
 .../cache/eviction/fifo/FifoEvictionPolicy.java |  117 +-
 .../eviction/fifo/FifoEvictionPolicyMBean.java  |   22 +
 .../cache/eviction/lru/LruEvictionPolicy.java   |  135 +-
 .../eviction/lru/LruEvictionPolicyMBean.java    |   38 +
 .../eviction/random/RandomEvictionPolicy.java   |   10 +-
 .../eviction/sorted/SortedEvictionPolicy.java   |  141 +-
 .../sorted/SortedEvictionPolicyMBean.java       |   22 +
 .../apache/ignite/cache/query/QueryMetrics.java |    6 +-
 .../apache/ignite/cache/store/CacheStore.java   |    2 +
 .../ignite/cache/store/CacheStoreSession.java   |   22 +
 .../cache/store/CacheStoreSessionListener.java  |  133 +
 .../jdbc/CacheJdbcStoreSessionListener.java     |  141 +
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   66 +-
 .../configuration/IgniteConfiguration.java      |   48 +-
 .../org/apache/ignite/igfs/IgfsUserContext.java |  119 +
 .../igfs/secondary/IgfsSecondaryFileSystem.java |    7 +
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   56 +-
 .../ignite/internal/IgniteNodeAttributes.java   |    5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   30 +-
 .../internal/MarshallerContextAdapter.java      |   36 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../internal/igfs/common/IgfsMarshaller.java    |   35 +-
 .../igfs/common/IgfsPathControlRequest.java     |   22 +
 .../internal/managers/GridManagerAdapter.java   |   68 +-
 .../checkpoint/GridCheckpointManager.java       |   52 +-
 .../managers/communication/GridIoManager.java   |  129 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   54 +
 .../discovery/GridDiscoveryManager.java         |  278 +-
 .../managers/indexing/GridIndexingManager.java  |    4 -
 .../affinity/GridAffinityAssignment.java        |   12 +
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../affinity/GridAffinityProcessor.java         |   23 +-
 .../cache/CacheEvictableEntryImpl.java          |   31 +
 .../processors/cache/CacheMetricsImpl.java      |  367 +-
 .../cache/CacheMetricsMXBeanImpl.java           |  100 +
 .../processors/cache/CacheMetricsSnapshot.java  |  380 +-
 .../cache/DynamicCacheChangeBatch.java          |   29 +-
 .../cache/DynamicCacheDescriptor.java           |   19 +
 .../processors/cache/GridCacheAdapter.java      |   50 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |   22 +-
 .../processors/cache/GridCacheEntryEx.java      |    6 +
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMapEntry.java     |   69 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   97 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |  121 +-
 .../processors/cache/GridCacheProxyImpl.java    |   26 +-
 .../cache/GridCacheSharedContext.java           |   48 +-
 .../processors/cache/GridCacheSwapManager.java  |  118 +-
 .../processors/cache/GridCacheTtlManager.java   |    9 +-
 .../processors/cache/GridCacheUtils.java        |  288 +-
 .../processors/cache/IgniteCacheProxy.java      |   23 +
 .../processors/cache/IgniteInternalCache.java   |   16 +-
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedCacheEntry.java  |    7 -
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |   10 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtCacheEntry.java      |    6 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   11 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   38 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   53 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   84 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   44 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  442 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   39 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |   31 +-
 .../cache/query/GridCacheQueryAdapter.java      |   24 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/store/CacheOsStoreManager.java        |    1 -
 .../cache/store/CacheStoreManager.java          |    7 +-
 .../store/GridCacheStoreManagerAdapter.java     |  202 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../transactions/IgniteTxLocalAdapter.java      |  170 +-
 .../cache/transactions/IgniteTxManager.java     |    3 -
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   63 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  838 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../datastreamer/DataStreamerImpl.java          |   94 +-
 .../datastructures/DataStructuresProcessor.java |  107 +-
 .../internal/processors/hadoop/HadoopJob.java   |    2 +-
 .../processors/hadoop/HadoopTaskContext.java    |   14 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |    8 +-
 .../internal/processors/igfs/IgfsImpl.java      |    8 +-
 .../processors/igfs/IgfsIpcHandler.java         |  184 +-
 .../processors/igfs/IgfsMetaManager.java        |    2 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |    9 +-
 .../internal/processors/igfs/IgfsServer.java    |    4 +-
 .../internal/processors/igfs/IgfsUtils.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |  305 +-
 .../service/GridServiceProcessor.java           |  125 +-
 .../timeout/GridSpiTimeoutObject.java           |   73 +
 .../timeout/GridTimeoutProcessor.java           |  105 +-
 .../IgniteTxRollbackCheckedException.java       |    9 +
 .../ignite/internal/util/GridJavaProcess.java   |   30 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../util/nio/GridCommunicationClient.java       |   30 +-
 .../util/nio/GridNioDelimitedBuffer.java        |    2 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |   13 +-
 .../util/nio/GridTcpCommunicationClient.java    |  554 --
 .../util/nio/GridTcpNioCommunicationClient.java |    8 -
 .../ignite/internal/visor/cache/VisorCache.java |    2 +-
 .../VisorCacheConfigurationCollectorJob.java    |    6 +-
 .../internal/visor/cache/VisorCacheMetrics.java |   19 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |   10 +-
 .../cache/VisorCacheStoreConfiguration.java     |    5 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    9 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   17 +-
 .../node/VisorNodeSuppressedErrorsTask.java     |   12 +-
 .../internal/visor/query/VisorQueryJob.java     |   13 +-
 .../internal/visor/query/VisorQueryTask.java    |    3 +-
 .../visor/util/VisorExceptionWrapper.java       |   81 +
 .../internal/visor/util/VisorTaskUtils.java     |   10 +
 .../ignite/mxbean/CacheMetricsMXBean.java       |   80 +
 .../apache/ignite/plugin/PluginProvider.java    |   26 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   70 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   57 +-
 .../ignite/spi/IgniteSpiTimeoutObject.java      |   44 +
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    3 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  438 +-
 .../tcp/TcpCommunicationSpiMBean.java           |    2 -
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4733 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  212 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5771 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../tcp/ipfinder/TcpDiscoveryIpFinder.java      |   10 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   57 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    8 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 ...cheStoreSessionListenerAbstractSelfTest.java |  315 +
 ...heStoreSessionListenerLifecycleSelfTest.java |  395 ++
 .../CacheJdbcStoreSessionListenerSelfTest.java  |  175 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 ...acheReadOnlyTransactionalClientSelfTest.java |  327 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  113 +
 .../cache/GridCacheAbstractMetricsSelfTest.java |   48 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 .../GridCacheConcurrentTxMultiNodeTest.java     |    8 +-
 ...idCacheConfigurationConsistencySelfTest.java |   14 +-
 .../GridCacheExAbstractFullApiSelfTest.java     |  103 -
 .../cache/GridCacheMemoryModeSelfTest.java      |   23 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../processors/cache/GridCacheOffHeapTest.java  |    5 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../cache/GridCacheReloadSelfTest.java          |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../processors/cache/GridCacheTestEntryEx.java  |    4 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../cache/IgniteCachePeekModesAbstractTest.java |    5 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |   62 +
 ...niteDynamicCacheWithConfigStartSelfTest.java |   97 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../IgniteCountDownLatchAbstractSelfTest.java   |  102 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../IgniteCrossCacheTxStoreSelfTest.java        |  147 +-
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   11 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   11 +-
 .../GridCacheExColocatedFullApiSelfTest.java    |   33 -
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   53 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 .../near/GridCacheExNearFullApiSelfTest.java    |   39 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 .../GridCachePartitionedEvictionSelfTest.java   |   11 +-
 .../GridCachePartitionedFullApiSelfTest.java    |   32 +
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    6 +-
 ...edOffHeapTieredMultiNodeFullApiSelfTest.java |    2 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheExReplicatedFullApiSelfTest.java   |   33 -
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 .../cache/eviction/EvictionAbstractTest.java    | 1056 ++++
 .../GridCacheBatchEvictUnswapSelfTest.java      |    5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   82 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   29 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    5 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |   11 +-
 .../eviction/GridCacheEvictionAbstractTest.java |  484 --
 .../GridCacheEvictionTouchSelfTest.java         |   22 +-
 .../cache/eviction/GridCacheMockEntry.java      |    5 +
 .../fifo/FifoEvictionPolicySelfTest.java        |  262 +
 ...ridCacheFifoBatchEvictionPolicySelfTest.java |  384 --
 .../GridCacheFifoEvictionPolicySelfTest.java    |  372 --
 .../lru/GridCacheLruEvictionPolicySelfTest.java |  417 --
 .../GridCacheLruNearEvictionPolicySelfTest.java |  136 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |  171 -
 .../eviction/lru/LruEvictionPolicySelfTest.java |  353 ++
 .../lru/LruNearEvictionPolicySelfTest.java      |  140 +
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |  172 +
 .../GridCacheRandomEvictionPolicySelfTest.java  |  258 -
 .../RandomEvictionPolicyCacheSizeSelfTest.java  |    6 +
 .../random/RandomEvictionPolicySelfTest.java    |  357 ++
 ...dCacheSortedBatchEvictionPolicySelfTest.java |  385 --
 ...acheSortedEvictionPolicyPerformanceTest.java |  135 -
 .../GridCacheSortedEvictionPolicySelfTest.java  |  373 --
 .../SortedEvictionPolicyPerformanceTest.java    |  134 +
 .../sorted/SortedEvictionPolicySelfTest.java    |  266 +
 .../IgniteCacheClientNearCacheExpiryTest.java   |  103 +
 .../IgniteCacheExpiryPolicyTestSuite.java       |    2 +
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java |  412 ++
 .../local/GridCacheExLocalFullApiSelfTest.java  |   30 -
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   96 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../DataStreamerMultiThreadedSelfTest.java      |   59 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |   97 +
 .../igfs/IgfsClientCacheSelfTest.java           |   12 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../nio/GridNioDelimitedBufferSelfTest.java     |  112 +
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    5 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    6 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    6 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../IgniteMessagingWithClientTest.java          |  166 +
 .../GridTcpCommunicationSpiAbstractTest.java    |    4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    2 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    2 -
 ...cpCommunicationSpiMultithreadedSelfTest.java |    2 +-
 .../discovery/AbstractDiscoverySelfTest.java    |   21 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |  129 +
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1196 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../stream/socket/SocketStreamerSelfTest.java   |   29 +-
 .../testframework/GridSpiTestContext.java       |   35 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/cache/TestCacheSession.java          |   18 +
 .../cache/TestThreadLocalCacheSession.java      |   15 +
 .../junits/common/GridCommonAbstractTest.java   |   83 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    4 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheEvictionSelfTestSuite.java       |   14 +-
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 -
 .../IgniteCacheMetricsSelfTestSuite.java        |    1 +
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    5 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    8 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    4 +-
 .../testsuites/IgniteStreamSelfTestSuite.java   |   39 +
 .../testsuites/IgniteUtilSelfTestSuite.java     |    2 +-
 .../gce/TcpDiscoveryGoogleStorageIpFinder.java  |   43 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   14 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |  165 +-
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    |  137 +-
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |   32 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    2 +-
 .../internal/processors/hadoop/HadoopUtils.java |  288 +-
 .../hadoop/SecondaryFileSystemProvider.java     |   57 +-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |   91 -
 .../hadoop/fs/HadoopFileSystemsUtils.java       |   17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |  204 +
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |    6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           |  170 +-
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |    2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |   33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |   19 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |   20 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |   31 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |   26 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |   52 +-
 .../hadoop/HadoopClientProtocolSelfTest.java    |    6 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   56 +-
 ...oopSecondaryFileSystemConfigurationTest.java |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   63 +-
 .../IgniteHadoopFileSystemClientSelfTest.java   |    2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |    2 +
 .../hadoop/HadoopAbstractSelfTest.java          |   14 +-
 .../hadoop/HadoopCommandLineTest.java           |   14 +-
 .../hadoop/HadoopFileSystemsTest.java           |   23 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |  183 +-
 .../hadoop/HadoopTaskExecutionSelfTest.java     |    2 +-
 .../hadoop/HadoopTasksAllVersionsTest.java      |   15 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |    5 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |    5 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |    6 +-
 .../collections/HadoopAbstractMapTest.java      |   12 +
 .../collections/HadoopSkipListSelfTest.java     |    4 +-
 .../CacheHibernateStoreSessionListener.java     |  216 +
 ...heHibernateStoreSessionListenerSelfTest.java |  228 +
 .../testsuites/IgniteHibernateTestSuite.java    |    2 +
 .../GridCacheOffheapIndexEntryEvictTest.java    |  200 +
 .../cache/GridCacheOffheapIndexGetSelfTest.java |   80 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    |   84 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    6 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |  104 +
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 ...QueryMultiThreadedOffHeapTieredSelfTest.java |   37 +
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   11 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    6 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../IgniteCacheWithIndexingTestSuite.java       |    3 +
 modules/mesos/pom.xml                           |    8 +-
 modules/spring/pom.xml                          |   14 +
 .../spring/CacheSpringStoreSessionListener.java |  207 +
 ...CacheSpringStoreSessionListenerSelfTest.java |  197 +
 .../testsuites/IgniteSpringTestSuite.java       |    3 +
 .../commands/cache/VisorCacheScanCommand.scala  |    2 +-
 scripts/git-apply-patch.sh                      |    8 +-
 scripts/git-format-patch.sh                     |   16 +-
 scripts/git-patch-functions.sh                  |   36 +-
 445 files changed, 30917 insertions(+), 16660 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 8f1730c..cd72418 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -24,12 +24,14 @@ NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 
 Ignite Release Instructions
 ===========================
-Use people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
-Execute these commands at source root:
+Use your people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
+Download https://dist.apache.org/repos/dist/release/incubator/ignite/KEYS and append you key using commands:
 
 gpg --list-sigs <keyname> >> KEYS
 gpg --armor --export <keyname> >> KEYS
 
+Upload modified KEYS file.
+
 Specify gpg profile at settings.xml. It will be used to sign sources and artifacts.
 
 <profile>
@@ -43,24 +45,38 @@ Specify gpg profile at settings.xml. It will be used to sign sources and artifac
 Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
 Maven release plugin release:prepare goal can be used to make release tag.
 
-Deploy Ignite release candidate to maven repository and dev-svn:
+Deploy Ignite release candidate to maven repository and dev-svn, make tag:
+
+   Following command deploys Ignite to maven repository, prepares sources and fabric edition binaries.
+      mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
+
+   In case you want to release both fabric and hadoop editions you have to build hadoop edition first using command
+      mvn package -P apache-release,gpg,release,scala,lgpl -Dignite.edition=hadoop -DskipTests -B
+   save /target/bin/*.zip, make "mvn clean" and restore saved files before deploying fabric.
 
-   mvn deploy -P apache-release,gpg,release,scala,lgpl,deploy-ignite-site -Dignite.edition=fabric -DskipTests -B
+   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern. Binary artifact will be
+   created inside /target/bin folder when release profile is used.
 
-   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern.
-   Sources package name is fixed. Sources package zip will be created automatically when apache-release profile used.
+   Sources artifact name is fixed. Sources artifact will be created inside /target dir when apache-release profile is used.
 
-   In case you want to release both fabric and hadoop editions you have to build hadoop first, save /target/bin/*.zip,
-   make "mvn clean" and restore them before deploy step.
+   Nexus staging (repository.apache.org) should be closed with appropriate comment contains release version and
+   release candidate number, for example "Apache Ignite 1.0.0-rc7", when mvn deploy finished.
 
-   Nexus staging (repository.apache.org) should be closed when mvn deploy finished.
-   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create new folder with name equals to released
-   version at svn root. Copy target/site folder content to svn/ignite/<version> folder and commit.
+   Checkout https://dist.apache.org/repos/dist/dev/incubator/ignite svn. Create release candidate folder with name
+   equals to release version with "-rc*" ending, for example "1.0.0-rc7", at svn root.
+   Copy /target/site folder content to svn/ignite/<rc-version> folder and commit with appropriate comment.
 
-Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>.
+   Make appropriate git tag for release candidate, for example "ignite-1.0.0-incubating-rc7".
 
-Release nexus staging and copy binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<version>
+Start vote based on https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>.
+
+Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/incubator/ignite/<rc-version>
 to https://dist.apache.org/repos/dist/release/incubator/ignite/<version> when version accepted.
+Use svn mv ^/dev/incubator/ignite/<rc-version> ^/release/incubator/ignite/<version> command for proper moving.
+
+Make appropriate git tag for released version, for example "ignite-1.0.0-incubating".
+
+Send an email to dev@ignite.incubator.apache.org contains release svn url.
 
 
 JCache TCK compliance

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/assembly/dependencies-visor-console.xml
----------------------------------------------------------------------
diff --git a/assembly/dependencies-visor-console.xml b/assembly/dependencies-visor-console.xml
index 28151da..d773769 100644
--- a/assembly/dependencies-visor-console.xml
+++ b/assembly/dependencies-visor-console.xml
@@ -90,6 +90,9 @@
                     <fileSet>
                         <directory>target/libs</directory>
                         <outputDirectory>/visor-common</outputDirectory>
+                        <excludes>
+                            <exclude>spring-jdbc-*.jar</exclude>
+                        </excludes>
                     </fileSet>
 
                     <fileSet>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
index 577301c..80a9f22 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernatePersonStore.java
@@ -21,10 +21,7 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.apache.ignite.transactions.Transaction;
 import org.hibernate.*;
-import org.hibernate.cfg.*;
-import org.jetbrains.annotations.*;
 
 import javax.cache.integration.*;
 import java.util.*;
@@ -34,99 +31,55 @@ import java.util.*;
  * and deals with maps {@link UUID} to {@link Person}.
  */
 public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
-    /** Default hibernate configuration resource path. */
-    private static final String DFLT_HIBERNATE_CFG = "/org/apache/ignite/examples/datagrid/store/hibernate" +
-        "/hibernate.cfg.xml";
-
-    /** Session attribute name. */
-    private static final String ATTR_SES = "HIBERNATE_STORE_SESSION";
-
-    /** Session factory. */
-    private SessionFactory sesFactory;
-
     /** Auto-injected store session. */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
 
-    /**
-     * Default constructor.
-     */
-    public CacheHibernatePersonStore() {
-        sesFactory = new Configuration().configure(DFLT_HIBERNATE_CFG).buildSessionFactory();
-    }
-
     /** {@inheritDoc} */
     @Override public Person load(Long key) {
-        Transaction tx = transaction();
-
-        System.out.println(">>> Store load [key=" + key + ", xid=" + (tx == null ? null : tx.xid()) + ']');
+        System.out.println(">>> Store load [key=" + key + ']');
 
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            return (Person) ses.get(Person.class, key);
+            return (Person)hibSes.get(Person.class, key);
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
-            throw new CacheLoaderException("Failed to load value from cache store with key: " + key, e);
-        }
-        finally {
-            end(ses, tx);
+            throw new CacheLoaderException("Failed to load value from cache store [key=" + key + ']', e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void write(javax.cache.Cache.Entry<? extends Long, ? extends Person> entry) {
-        Transaction tx = transaction();
-
         Long key = entry.getKey();
-
         Person val = entry.getValue();
 
-        System.out.println(">>> Store put [key=" + key + ", val=" + val + ", xid=" + (tx == null ? null : tx.xid()) + ']');
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
 
-        if (val == null) {
-            delete(key);
-
-            return;
-        }
-
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            ses.saveOrUpdate(val);
+            hibSes.saveOrUpdate(val);
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
             throw new CacheWriterException("Failed to put value to cache store [key=" + key + ", val" + val + "]", e);
         }
-        finally {
-            end(ses, tx);
-        }
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"JpaQueryApiInspection"})
     @Override public void delete(Object key) {
-        Transaction tx = transaction();
+        System.out.println(">>> Store delete [key=" + key + ']');
 
-        System.out.println(">>> Store remove [key=" + key + ", xid=" + (tx == null ? null : tx.xid()) + ']');
-
-        Session ses = session(tx);
+        Session hibSes = ses.attachment();
 
         try {
-            ses.createQuery("delete " + Person.class.getSimpleName() + " where key = :key")
-                .setParameter("key", key).setFlushMode(FlushMode.ALWAYS).executeUpdate();
+            hibSes.createQuery("delete " + Person.class.getSimpleName() + " where key = :key").
+                setParameter("key", key).
+                executeUpdate();
         }
         catch (HibernateException e) {
-            rollback(ses, tx);
-
-            throw new CacheWriterException("Failed to remove value from cache store with key: " + key, e);
-        }
-        finally {
-            end(ses, tx);
+            throw new CacheWriterException("Failed to remove value from cache store [key=" + key + ']', e);
         }
     }
 
@@ -137,18 +90,18 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
 
         final int entryCnt = (Integer)args[0];
 
-        Session ses = session(null);
+        Session hibSes = ses.attachment();
 
         try {
             int cnt = 0;
 
-            List res = ses.createCriteria(Person.class).list();
-
-            if (res != null) {
-                Iterator iter = res.iterator();
+            List list = hibSes.createCriteria(Person.class).
+                setMaxResults(entryCnt).
+                list();
 
-                while (cnt < entryCnt && iter.hasNext()) {
-                    Person person = (Person)iter.next();
+            if (list != null) {
+                for (Object obj : list) {
+                    Person person = (Person)obj;
 
                     clo.apply(person.getId(), person);
 
@@ -161,120 +114,5 @@ public class CacheHibernatePersonStore extends CacheStoreAdapter<Long, Person> {
         catch (HibernateException e) {
             throw new CacheLoaderException("Failed to load values from cache store.", e);
         }
-        finally {
-            end(ses, null);
-        }
-    }
-
-    /**
-     * Rolls back hibernate session.
-     *
-     * @param ses Hibernate session.
-     * @param tx Cache ongoing transaction.
-     */
-    private void rollback(Session ses, Transaction tx) {
-        // Rollback only if there is no cache transaction,
-        // otherwise sessionEnd() will do all required work.
-        if (tx == null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null && hTx.isActive())
-                hTx.rollback();
-        }
-    }
-
-    /**
-     * Ends hibernate session.
-     *
-     * @param ses Hibernate session.
-     * @param tx Cache ongoing transaction.
-     */
-    private void end(Session ses, @Nullable Transaction tx) {
-        // Commit only if there is no cache transaction,
-        // otherwise sessionEnd() will do all required work.
-        if (tx == null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null && hTx.isActive())
-                hTx.commit();
-
-            ses.close();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sessionEnd(boolean commit) {
-        Transaction tx = ses.transaction();
-
-        Map<String, Session> props = ses.properties();
-
-        Session ses = props.remove(ATTR_SES);
-
-        if (ses != null) {
-            org.hibernate.Transaction hTx = ses.getTransaction();
-
-            if (hTx != null) {
-                try {
-                    if (commit) {
-                        ses.flush();
-
-                        hTx.commit();
-                    }
-                    else
-                        hTx.rollback();
-
-                    System.out.println("Transaction ended [xid=" + tx.xid() + ", commit=" + commit + ']');
-                }
-                catch (HibernateException e) {
-                    throw new CacheWriterException("Failed to end transaction [xid=" + tx.xid() +
-                        ", commit=" + commit + ']', e);
-                }
-                finally {
-                    ses.close();
-                }
-            }
-        }
-    }
-
-    /**
-     * Gets Hibernate session.
-     *
-     * @param tx Cache transaction.
-     * @return Session.
-     */
-    private Session session(@Nullable Transaction tx) {
-        Session hbSes;
-
-        if (tx != null) {
-            Map<String, Session> props = ses.properties();
-
-            hbSes = props.get(ATTR_SES);
-
-            if (hbSes == null) {
-                hbSes = sesFactory.openSession();
-
-                hbSes.beginTransaction();
-
-                // Store session in session properties, so it can be accessed
-                // for other operations on the same transaction.
-                props.put(ATTR_SES, hbSes);
-
-                System.out.println("Hibernate session open [ses=" + hbSes + ", tx=" + tx.xid() + "]");
-            }
-        }
-        else {
-            hbSes = sesFactory.openSession();
-
-            hbSes.beginTransaction();
-        }
-
-        return hbSes;
-    }
-
-    /**
-     * @return Current transaction.
-     */
-    @Nullable private Transaction transaction() {
-        return ses != null ? ses.transaction() : null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
index 5a0cd0a..25760f0 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/hibernate/CacheHibernateStoreExample.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.examples.datagrid.store.hibernate;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.hibernate.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.examples.*;
 import org.apache.ignite.examples.datagrid.store.*;
@@ -37,6 +39,10 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  * start node with {@code examples/config/example-ignite.xml} configuration.
  */
 public class CacheHibernateStoreExample {
+    /** Hibernate configuration resource path. */
+    private static final String HIBERNATE_CFG =
+        "/org/apache/ignite/examples/datagrid/store/hibernate/hibernate.cfg.xml";
+
     /** Cache name. */
     private static final String CACHE_NAME = CacheHibernateStoreExample.class.getSimpleName();
 
@@ -71,6 +77,17 @@ public class CacheHibernateStoreExample {
             // Configure Hibernate store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheHibernatePersonStore.class));
 
+            // Configure Hibernate session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheHibernateStoreSessionListener lsnr = new CacheHibernateStoreSessionListener();
+
+                    lsnr.setHibernateConfigurationPath(HIBERNATE_CFG);
+
+                    return lsnr;
+                }
+            });
+
             cacheCfg.setReadThrough(true);
             cacheCfg.setWriteThrough(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
index 791f861..ed14a99 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcPersonStore.java
@@ -22,23 +22,23 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.examples.datagrid.store.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.jetbrains.annotations.*;
+import org.h2.jdbcx.*;
 
 import javax.cache.*;
 import javax.cache.integration.*;
+import javax.sql.*;
 import java.sql.*;
-import java.util.*;
 
 /**
  * Example of {@link CacheStore} implementation that uses JDBC
  * transaction with cache transactions and maps {@link Long} to {@link Person}.
- *
  */
 public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
-    /** Transaction metadata attribute name. */
-    private static final String ATTR_NAME = "SIMPLE_STORE_CONNECTION";
+    /** Data source. */
+    public static final DataSource DATA_SRC =
+        JdbcConnectionPool.create("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1", "", "");
 
-    /** Auto-injected store session. */
+    /** Store session. */
     @CacheStoreSessionResource
     private CacheStoreSession ses;
 
@@ -58,11 +58,10 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
      * @throws IgniteException If failed.
      */
     private void prepareDb() throws IgniteException {
-        try (Connection conn = openConnection(false); Statement st = conn.createStatement()) {
-            st.execute("create table if not exists PERSONS (id number unique, firstName varchar(255), " +
-                "lastName varchar(255))");
-
-            conn.commit();
+        try (Connection conn = DATA_SRC.getConnection()) {
+            conn.createStatement().execute(
+                "create table if not exists PERSONS (" +
+                "id number unique, firstName varchar(255), lastName varchar(255))");
         }
         catch (SQLException e) {
             throw new IgniteException("Failed to create database table.", e);
@@ -70,71 +69,39 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(boolean commit) {
-        Map<String, Connection> props = ses.properties();
-
-        try (Connection conn = props.remove(ATTR_NAME)) {
-            if (conn != null) {
-                if (commit)
-                    conn.commit();
-                else
-                    conn.rollback();
-            }
-
-            System.out.println(">>> Transaction ended [commit=" + commit + ']');
-        }
-        catch (SQLException e) {
-            throw new CacheWriterException("Failed to end transaction: " + ses.transaction(), e);
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public Person load(Long key) {
-        System.out.println(">>> Loading key: " + key);
+        System.out.println(">>> Store load [key=" + key + ']');
 
-        Connection conn = null;
+        Connection conn = ses.attachment();
 
-        try {
-            conn = connection();
-
-            try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id=?")) {
-                st.setString(1, key.toString());
+        try (PreparedStatement st = conn.prepareStatement("select * from PERSONS where id = ?")) {
+            st.setString(1, key.toString());
 
-                ResultSet rs = st.executeQuery();
+            ResultSet rs = st.executeQuery();
 
-                if (rs.next())
-                    return new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
-            }
+            return rs.next() ? new Person(rs.getLong(1), rs.getString(2), rs.getString(3)) : null;
         }
         catch (SQLException e) {
-            throw new CacheLoaderException("Failed to load object: " + key, e);
+            throw new CacheLoaderException("Failed to load object [key=" + key + ']', e);
         }
-        finally {
-            end(conn);
-        }
-
-        return null;
     }
 
     /** {@inheritDoc} */
     @Override public void write(Cache.Entry<? extends Long, ? extends Person> entry) {
         Long key = entry.getKey();
-
         Person val = entry.getValue();
 
-        System.out.println(">>> Putting [key=" + key + ", val=" + val +  ']');
-
-        Connection conn = null;
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
 
         try {
-            conn = connection();
+            Connection conn = ses.attachment();
 
             int updated;
 
             // Try update first. If it does not work, then try insert.
             // Some databases would allow these to be done in one 'upsert' operation.
             try (PreparedStatement st = conn.prepareStatement(
-                "update PERSONS set firstName=?, lastName=? where id=?")) {
+                "update PERSONS set firstName = ?, lastName = ? where id = ?")) {
                 st.setString(1, val.getFirstName());
                 st.setString(2, val.getLastName());
                 st.setLong(3, val.getId());
@@ -145,7 +112,7 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             // If update failed, try to insert.
             if (updated == 0) {
                 try (PreparedStatement st = conn.prepareStatement(
-                    "insert into PERSONS (id, firstName, lastName) values(?, ?, ?)")) {
+                    "insert into PERSONS (id, firstName, lastName) values (?, ?, ?)")) {
                     st.setLong(1, val.getId());
                     st.setString(2, val.getFirstName());
                     st.setString(3, val.getLastName());
@@ -155,33 +122,23 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
             }
         }
         catch (SQLException e) {
-            throw new CacheLoaderException("Failed to put object [key=" + key + ", val=" + val + ']', e);
-        }
-        finally {
-            end(conn);
+            throw new CacheWriterException("Failed to write object [key=" + key + ", val=" + val + ']', e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void delete(Object key) {
-        System.out.println(">>> Removing key: " + key);
-
-        Connection conn = null;
+        System.out.println(">>> Store delete [key=" + key + ']');
 
-        try {
-            conn = connection();
+        Connection conn = ses.attachment();
 
-            try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
-                st.setLong(1, (Long)key);
+        try (PreparedStatement st = conn.prepareStatement("delete from PERSONS where id=?")) {
+            st.setLong(1, (Long)key);
 
-                st.executeUpdate();
-            }
+            st.executeUpdate();
         }
         catch (SQLException e) {
-            throw new CacheWriterException("Failed to remove object: " + key, e);
-        }
-        finally {
-            end(conn);
+            throw new CacheWriterException("Failed to delete object [key=" + key + ']', e);
         }
     }
 
@@ -192,84 +149,27 @@ public class CacheJdbcPersonStore extends CacheStoreAdapter<Long, Person> {
 
         final int entryCnt = (Integer)args[0];
 
-        try (Connection conn = connection()) {
-            try (PreparedStatement st = conn.prepareStatement("select * from PERSONS")) {
-                try (ResultSet rs = st.executeQuery()) {
-                    int cnt = 0;
+        Connection conn = ses.attachment();
 
-                    while (cnt < entryCnt && rs.next()) {
-                        Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+        try (PreparedStatement stmt = conn.prepareStatement("select * from PERSONS limit ?")) {
+            stmt.setInt(1, entryCnt);
 
-                        clo.apply(person.getId(), person);
+            ResultSet rs = stmt.executeQuery();
 
-                        cnt++;
-                    }
+            int cnt = 0;
 
-                    System.out.println(">>> Loaded " + cnt + " values into cache.");
-                }
-            }
-        }
-        catch (SQLException e) {
-            throw new CacheLoaderException("Failed to load values from cache store.", e);
-        }
-    }
-
-    /**
-     * @return Connection.
-     * @throws SQLException In case of error.
-     */
-    private Connection connection() throws SQLException  {
-        // If there is an ongoing transaction,
-        // we must reuse the same connection.
-        if (ses.isWithinTransaction()) {
-            Map<Object, Object> props = ses.properties();
-
-            Connection conn = (Connection)props.get(ATTR_NAME);
+            while (rs.next()) {
+                Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
 
-            if (conn == null) {
-                conn = openConnection(false);
+                clo.apply(person.getId(), person);
 
-                // Store connection in session properties, so it can be accessed
-                // for other operations on the same transaction.
-                props.put(ATTR_NAME, conn);
+                cnt++;
             }
 
-            return conn;
+            System.out.println(">>> Loaded " + cnt + " values into cache.");
         }
-        // Transaction can be null in case of simple load or put operation.
-        else
-            return openConnection(true);
-    }
-
-    /**
-     * Closes allocated resources depending on transaction status.
-     *
-     * @param conn Allocated connection.
-     */
-    private void end(@Nullable Connection conn) {
-        if (!ses.isWithinTransaction() && conn != null) {
-            // Close connection right away if there is no transaction.
-            try {
-                conn.close();
-            }
-            catch (SQLException ignored) {
-                // No-op.
-            }
+        catch (SQLException e) {
+            throw new CacheLoaderException("Failed to load values from cache store.", e);
         }
     }
-
-    /**
-     * Gets connection from a pool.
-     *
-     * @param autocommit {@code true} If connection should use autocommit mode.
-     * @return Pooled connection.
-     * @throws SQLException In case of error.
-     */
-    private Connection openConnection(boolean autocommit) throws SQLException {
-        Connection conn = DriverManager.getConnection("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
-
-        conn.setAutoCommit(autocommit);
-
-        return conn;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
index 1cb73c9..637d6dc 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/jdbc/CacheJdbcStoreExample.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.examples.datagrid.store.jdbc;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.examples.*;
 import org.apache.ignite.examples.datagrid.store.*;
@@ -71,6 +73,17 @@ public class CacheJdbcStoreExample {
             // Configure JDBC store.
             cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheJdbcPersonStore.class));
 
+            // Configure JDBC session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
+
+                    lsnr.setDataSource(CacheJdbcPersonStore.DATA_SRC);
+
+                    return lsnr;
+                }
+            });
+
             cacheCfg.setReadThrough(true);
             cacheCfg.setWriteThrough(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
new file mode 100644
index 0000000..50149ba
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringPersonStore.java
@@ -0,0 +1,128 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.examples.datagrid.store.*;
+import org.apache.ignite.lang.*;
+import org.springframework.dao.*;
+import org.springframework.jdbc.core.*;
+import org.springframework.jdbc.datasource.*;
+
+import javax.cache.*;
+import javax.cache.integration.*;
+import javax.sql.*;
+import java.sql.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Example of {@link CacheStore} implementation that uses JDBC
+ * transaction with cache transactions and maps {@link Long} to {@link Person}.
+ */
+public class CacheSpringPersonStore extends CacheStoreAdapter<Long, Person> {
+    /** Data source. */
+    public static final DataSource DATA_SRC = new DriverManagerDataSource("jdbc:h2:mem:example;DB_CLOSE_DELAY=-1");
+
+    /** Spring JDBC template. */
+    private JdbcTemplate jdbcTemplate;
+
+    /**
+     * Constructor.
+     *
+     * @throws IgniteException If failed.
+     */
+    public CacheSpringPersonStore() throws IgniteException {
+        jdbcTemplate = new JdbcTemplate(DATA_SRC);
+
+        prepareDb();
+    }
+
+    /**
+     * Prepares database for example execution. This method will create a
+     * table called "PERSONS" so it can be used by store implementation.
+     *
+     * @throws IgniteException If failed.
+     */
+    private void prepareDb() throws IgniteException {
+        jdbcTemplate.update(
+            "create table if not exists PERSONS (" +
+            "id number unique, firstName varchar(255), lastName varchar(255))");
+    }
+
+    /** {@inheritDoc} */
+    @Override public Person load(Long key) {
+        System.out.println(">>> Store load [key=" + key + ']');
+
+        try {
+            return jdbcTemplate.queryForObject("select * from PERSONS where id = ?", new RowMapper<Person>() {
+                @Override public Person mapRow(ResultSet rs, int rowNum) throws SQLException {
+                    return new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+                }
+            }, key);
+        }
+        catch (EmptyResultDataAccessException ignored) {
+            return null;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(Cache.Entry<? extends Long, ? extends Person> entry) {
+        Long key = entry.getKey();
+        Person val = entry.getValue();
+
+        System.out.println(">>> Store write [key=" + key + ", val=" + val + ']');
+
+        int updated = jdbcTemplate.update("update PERSONS set firstName = ?, lastName = ? where id = ?",
+            val.getFirstName(), val.getLastName(), val.getId());
+
+        if (updated == 0) {
+            jdbcTemplate.update("insert into PERSONS (id, firstName, lastName) values (?, ?, ?)",
+                val.getId(), val.getFirstName(), val.getLastName());
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void delete(Object key) {
+        System.out.println(">>> Store delete [key=" + key + ']');
+
+        jdbcTemplate.update("delete from PERSONS where id = ?", key);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void loadCache(final IgniteBiInClosure<Long, Person> clo, Object... args) {
+        if (args == null || args.length == 0 || args[0] == null)
+            throw new CacheLoaderException("Expected entry count parameter is not provided.");
+
+        int entryCnt = (Integer)args[0];
+
+        final AtomicInteger cnt = new AtomicInteger();
+
+        jdbcTemplate.query("select * from PERSONS limit ?", new RowCallbackHandler() {
+            @Override public void processRow(ResultSet rs) throws SQLException {
+                Person person = new Person(rs.getLong(1), rs.getString(2), rs.getString(3));
+
+                clo.apply(person.getId(), person);
+
+                cnt.incrementAndGet();
+            }
+        }, entryCnt);
+
+        System.out.println(">>> Loaded " + cnt + " values into cache.");
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
new file mode 100644
index 0000000..aa624bc
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/CacheSpringStoreExample.java
@@ -0,0 +1,143 @@
+/*
+ * 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.examples.datagrid.store.spring;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.cache.store.jdbc.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.examples.*;
+import org.apache.ignite.examples.datagrid.store.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.configuration.*;
+import java.util.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Demonstrates usage of cache with underlying persistent store configured.
+ * <p>
+ * This example uses {@link CacheSpringPersonStore} as a persistent store.
+ * <p>
+ * Remote nodes can be started with {@link ExampleNodeStartup} in another JVM which will
+ * start node with {@code examples/config/example-ignite.xml} configuration.
+ */
+public class CacheSpringStoreExample {
+    /** Cache name. */
+    private static final String CACHE_NAME = CacheSpringStoreExample.class.getSimpleName();
+
+    /** Heap size required to run this example. */
+    public static final int MIN_MEMORY = 1024 * 1024 * 1024;
+
+    /** Number of entries to load. */
+    private static final int ENTRY_COUNT = 100_000;
+
+    /** Global person ID to use across entire example. */
+    private static final Long id = Math.abs(UUID.randomUUID().getLeastSignificantBits());
+
+    /**
+     * Executes example.
+     *
+     * @param args Command line arguments, none required.
+     * @throws IgniteException If example execution failed.
+     */
+    public static void main(String[] args) throws IgniteException {
+        ExamplesUtils.checkMinMemory(MIN_MEMORY);
+
+        // To start ignite with desired configuration uncomment the appropriate line.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println();
+            System.out.println(">>> Cache store example started.");
+
+            CacheConfiguration<Long, Person> cacheCfg = new CacheConfiguration<>(CACHE_NAME);
+
+            // Set atomicity as transaction, since we are showing transactions in example.
+            cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+            // Configure Spring store.
+            cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(CacheSpringPersonStore.class));
+
+            // Configure Spring session listener.
+            cacheCfg.setCacheStoreSessionListenerFactories(new Factory<CacheStoreSessionListener>() {
+                @Override public CacheStoreSessionListener create() {
+                    CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
+
+                    lsnr.setDataSource(CacheSpringPersonStore.DATA_SRC);
+
+                    return lsnr;
+                }
+            });
+
+            cacheCfg.setReadThrough(true);
+            cacheCfg.setWriteThrough(true);
+
+            try (IgniteCache<Long, Person> cache = ignite.getOrCreateCache(cacheCfg)) {
+                // Make initial cache loading from persistent store. This is a
+                // distributed operation and will call CacheStore.loadCache(...)
+                // method on all nodes in topology.
+                loadCache(cache);
+
+                // Start transaction and execute several cache operations with
+                // read/write-through to persistent store.
+                executeTransaction(cache);
+            }
+        }
+    }
+
+    /**
+     * Makes initial cache loading.
+     *
+     * @param cache Cache to load.
+     */
+    private static void loadCache(IgniteCache<Long, Person> cache) {
+        long start = System.currentTimeMillis();
+
+        // Start loading cache from persistent store on all caching nodes.
+        cache.loadCache(null, ENTRY_COUNT);
+
+        long end = System.currentTimeMillis();
+
+        System.out.println(">>> Loaded " + cache.size() + " keys with backups in " + (end - start) + "ms.");
+    }
+
+    /**
+     * Executes transaction with read/write-through to persistent store.
+     *
+     * @param cache Cache to execute transaction on.
+     */
+    private static void executeTransaction(IgniteCache<Long, Person> cache) {
+        try (Transaction tx = Ignition.ignite().transactions().txStart()) {
+            Person val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            val = cache.getAndPut(id, new Person(id, "Isaac", "Newton"));
+
+            System.out.println("Overwrote old value: " + val);
+
+            val = cache.get(id);
+
+            System.out.println("Read value: " + val);
+
+            tx.commit();
+        }
+
+        System.out.println("Read value after commit: " + cache.get(id));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
new file mode 100644
index 0000000..211239f
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/store/spring/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains Spring-based cache store implementation.
+ */
+package org.apache.ignite.examples.datagrid.store.spring;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index f9007a2..a54adc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -311,10 +311,26 @@ public interface IgniteCache<K, V> extends javax.cache.Cache<K, V>, IgniteAsyncS
     @IgniteAsyncSupported
     @Override public Map<K, V> getAll(Set<? extends K> keys);
 
+    /**
+     * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
+     * and will not lock any keys if pessimistic transaction is started by thread.
+     *
+     * @param keys The keys whose associated values are to be returned.
+     * @return A map of entries that were found for the given keys.
+     */
+    @IgniteAsyncSupported
+    public Map<K, V> getAllOutTx(Set<? extends K> keys);
+
     /** {@inheritDoc} */
     @IgniteAsyncSupported
     @Override public boolean containsKey(K key);
 
+    /**
+     * Determines if the {@link Cache} contains entries for the specified keys.
+     *
+     * @param keys Key whose presence in this cache is to be tested.
+     * @return {@code True} if this cache contains a mapping for the specified keys.
+     */
     @IgniteAsyncSupported
     public boolean containsKeys(Set<? extends K> keys);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 6ed99bd..d5d59a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -20,6 +20,8 @@ package 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.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -133,18 +135,30 @@ public class Ignition {
     }
 
     /**
-     * Sets client mode flag.
+     * Sets client mode static flag.
+     * <p>
+     * This flag used when node is started if {@link IgniteConfiguration#isClientMode()}
+     * is {@code null}. When {@link IgniteConfiguration#isClientMode()} is set this flag is ignored.
+     * It is recommended to use {@link DiscoverySpi} in client mode too.
      *
      * @param clientMode Client mode flag.
+     * @see IgniteConfiguration#isClientMode()
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public static void setClientMode(boolean clientMode) {
         IgnitionEx.setClientMode(clientMode);
     }
 
     /**
-     * Gets client mode flag.
+     * Gets client mode static flag.
+     * <p>
+     * This flag used when node is started if {@link IgniteConfiguration#isClientMode()}
+     * is {@code null}. When {@link IgniteConfiguration#isClientMode()} is set this flag is ignored.
+     * It is recommended to use {@link DiscoverySpi} in client mode too.
      *
      * @return Client mode flag.
+     * @see IgniteConfiguration#isClientMode()
+     * @see TcpDiscoverySpi#setForceServerMode(boolean)
      */
     public static boolean isClientMode() {
         return IgnitionEx.isClientMode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index 0d87326..799aace 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -30,21 +30,21 @@ public interface CacheMetrics {
     /**
      * The number of get requests that were satisfied by the cache.
      *
-     * @return the number of hits
+     * @return The number of hits.
      */
     public long getCacheHits();
 
     /**
      * This is a measure of cache efficiency.
      *
-     * @return the percentage of successful hits, as a decimal e.g 75.
+     * @return The percentage of successful hits, as a decimal e.g 75.
      */
     public float getCacheHitPercentage();
 
     /**
      * A miss is a get request that is not satisfied.
      *
-     * @return the number of misses
+     * @return The number of misses.
      */
     public long getCacheMisses();
 
@@ -52,7 +52,7 @@ public interface CacheMetrics {
      * Returns the percentage of cache accesses that did not find a requested entry
      * in the cache.
      *
-     * @return the percentage of accesses that failed to find anything
+     * @return The percentage of accesses that failed to find anything.
      */
     public float getCacheMissPercentage();
 
@@ -60,14 +60,14 @@ public interface CacheMetrics {
      * The total number of requests to the cache. This will be equal to the sum of
      * the hits and misses.
      *
-     * @return the number of gets
+     * @return The number of gets.
      */
     public long getCacheGets();
 
     /**
      * The total number of puts to the cache.
      *
-     * @return the number of puts
+     * @return The number of puts.
      */
     public long getCachePuts();
 
@@ -75,7 +75,7 @@ public interface CacheMetrics {
      * The total number of removals from the cache. This does not include evictions,
      * where the cache itself initiates the removal to make space.
      *
-     * @return the number of removals
+     * @return The number of removals.
      */
     public long getCacheRemovals();
 
@@ -84,28 +84,28 @@ public interface CacheMetrics {
      * initiated by the cache itself to free up space. An eviction is not treated as
      * a removal and does not appear in the removal counts.
      *
-     * @return the number of evictions
+     * @return The number of evictions.
      */
     public long getCacheEvictions();
 
     /**
      * The mean time to execute gets.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageGetTime();
 
     /**
      * The mean time to execute puts.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAveragePutTime();
 
     /**
      * The mean time to execute removes.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageRemoveTime();
 
@@ -113,7 +113,7 @@ public interface CacheMetrics {
     /**
      * The mean time to execute tx commit.
      *
-     * @return the time in µs
+     * @return The time in µs.
      */
     public float getAverageTxCommitTime();
 
@@ -124,7 +124,6 @@ public interface CacheMetrics {
      */
     public float getAverageTxRollbackTime();
 
-
     /**
      * Gets total number of transaction commits.
      *
@@ -154,6 +153,62 @@ public interface CacheMetrics {
     public long getOverflowSize();
 
     /**
+     * The total number of get requests to the off-heap memory.
+     *
+     * @return The number of gets.
+     */
+    public long getOffHeapGets();
+
+    /**
+     * The total number of put requests to the off-heap memory.
+     *
+     * @return The number of puts.
+     */
+    public long getOffHeapPuts();
+
+    /**
+     * The total number of removals from the off-heap memory. This does not include evictions.
+     *
+     * @return The number of removals.
+     */
+    public long getOffHeapRemovals();
+
+    /**
+     * The total number of evictions from the off-heap memory.
+     *
+     * @return The number of evictions.
+     */
+    public long getOffHeapEvictions();
+
+    /**
+     * The number of get requests that were satisfied by the off-heap memory.
+     *
+     * @return The off-heap hits number.
+     */
+    public long getOffHeapHits();
+
+    /**
+     * Gets the percentage of hits on off-heap memory.
+     *
+     * @return The percentage of hits on off-heap memory.
+     */
+    public float getOffHeapHitPercentage();
+
+    /**
+     * A miss is a get request that is not satisfied by off-heap memory.
+     *
+     * @return The off-heap misses number.
+     */
+    public long getOffHeapMisses();
+
+    /**
+     * Gets the percentage of misses on off-heap memory.
+     *
+     * @return The percentage of misses on off-heap memory.
+     */
+    public float getOffHeapMissPercentage();
+
+    /**
      * Gets number of entries stored in off-heap memory.
      *
      * @return Number of entries stored in off-heap memory.
@@ -161,6 +216,20 @@ public interface CacheMetrics {
     public long getOffHeapEntriesCount();
 
     /**
+     * Gets number of primary entries stored in off-heap memory.
+     *
+     * @return Number of primary entries stored in off-heap memory.
+     */
+    public long getOffHeapPrimaryEntriesCount();
+
+    /**
+     * Gets number of backup entries stored in off-heap memory.
+     *
+     * @return Number of backup entries stored in off-heap memory.
+     */
+    public long getOffHeapBackupEntriesCount();
+
+    /**
      * Gets memory size allocated in off-heap.
      *
      * @return Memory size allocated in off-heap.
@@ -168,6 +237,76 @@ public interface CacheMetrics {
     public long getOffHeapAllocatedSize();
 
     /**
+     * Gets off-heap memory maximum size.
+     *
+     * @return Off-heap memory maximum size.
+     */
+    public long getOffHeapMaxSize();
+
+    /**
+     * The total number of get requests to the swap.
+     *
+     * @return The number of gets.
+     */
+    public long getSwapGets();
+
+    /**
+     * The total number of put requests to the swap.
+     *
+     * @return The number of puts.
+     */
+    public long getSwapPuts();
+
+    /**
+     * The total number of removals from the swap.
+     *
+     * @return The number of removals.
+     */
+    public long getSwapRemovals();
+
+    /**
+     * The number of get requests that were satisfied by the swap.
+     *
+     * @return The swap hits number.
+     */
+    public long getSwapHits();
+
+    /**
+     * A miss is a get request that is not satisfied by swap.
+     *
+     * @return The swap misses number.
+     */
+    public long getSwapMisses();
+
+    /**
+     * Gets number of entries stored in swap.
+     *
+     * @return Number of entries stored in swap.
+     */
+    public long getSwapEntriesCount();
+
+    /**
+     * Gets size of swap.
+     *
+     * @return Size of swap.
+     */
+    public long getSwapSize();
+
+    /**
+     * Gets the percentage of hits on swap.
+     *
+     * @return The percentage of hits on swap.
+     */
+    public float getSwapHitPercentage();
+
+    /**
+     * Gets the percentage of misses on swap.
+     *
+     * @return The percentage of misses on swap.
+     */
+    public float getSwapMissPercentage();
+
+    /**
      * Gets number of non-{@code null} values in the cache.
      *
      * @return Number of non-{@code null} values in the cache.
@@ -184,7 +323,7 @@ public interface CacheMetrics {
     /**
      * Returns {@code true} if this cache is empty.
      *
-     * @return {@code true} if this cache is empty.
+     * @return {@code True} if this cache is empty.
      */
     public boolean isEmpty();
 
@@ -294,7 +433,7 @@ public interface CacheMetrics {
     public int getTxDhtRolledbackVersionsSize();
 
     /**
-     * Returns {@code True} if write-behind is enabled.
+     * Returns {@code true} if write-behind is enabled.
      *
      * @return {@code True} if write-behind is enabled.
      */
@@ -372,16 +511,16 @@ public interface CacheMetrics {
     /**
      * Determines the required type of keys for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the key type,
-     * or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the key type,
+     * or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getKeyType();
 
     /**
      * Determines the required type of values for this {@link Cache}, if any.
      *
-     * @return the fully qualified class name of the value type,
-     *         or "java.lang.Object" if the type is undefined.
+     * @return The fully qualified class name of the value type,
+     *         or {@code "java.lang.Object"} if the type is undefined.
      */
     public String getValueType();
 
@@ -407,7 +546,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code true}.
      *
-     * @return true if the cache is store by value
+     * @return {@code True} if the cache is store by value.
      */
     public boolean isStoreByValue();
 
@@ -416,7 +555,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if statistics collection is enabled
+     * @return {@code True} if statistics collection is enabled.
      */
     public boolean isStatisticsEnabled();
 
@@ -425,7 +564,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}.
      *
-     * @return true if management is enabled
+     * @return {@code true} if management is enabled.
      */
     public boolean isManagementEnabled();
 
@@ -434,7 +573,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in
+     * @return {@code True} when a {@link Cache} is in
      *         "read-through" mode.
      * @see CacheLoader
      */
@@ -448,7 +587,7 @@ public interface CacheMetrics {
      * <p>
      * The default value is {@code false}
      *
-     * @return {@code true} when a {@link Cache} is in "write-through" mode.
+     * @return {@code True} when a {@link Cache} is in "write-through" mode.
      * @see CacheWriter
      */
     public boolean isWriteThrough();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
index d87109f..9f1889a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictableEntry.java
@@ -46,6 +46,13 @@ public interface EvictableEntry<K, V> extends Cache.Entry<K, V> {
     public boolean isCached();
 
     /**
+     * Returns entry size in bytes.
+     *
+     * @return entry size in bytes.
+     */
+    public int size();
+
+    /**
      * Gets metadata added by eviction policy.
      *
      * @return Metadata value or {@code null}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
index f409e9b..07c269d 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/eviction/EvictionPolicy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.cache.eviction;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.cache.eviction.random.*;
+import org.apache.ignite.cache.eviction.sorted.*;
 
 /**
  * Pluggable cache eviction policy. Usually, implementations will internally order
@@ -32,6 +33,7 @@ import org.apache.ignite.cache.eviction.random.*;
  * <li>{@link LruEvictionPolicy}</li>
  * <li>{@link RandomEvictionPolicy}</li>
  * <li>{@link FifoEvictionPolicy}</li>
+ * <li>{@link SortedEvictionPolicy}</li>
  * </ul>
  * <p>
  * The eviction policy thread-safety is ensured by Ignition. Implementations of this interface should


[22/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 4b8db00..9f18c98 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
@@ -44,6 +44,8 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 import java.util.concurrent.locks.*;
 
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.*;
 
 /**
@@ -117,8 +119,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     private GridFutureAdapter<Boolean> initFut;
 
     /** Topology snapshot. */
-    private AtomicReference<GridDiscoveryTopologySnapshot> topSnapshot =
-        new AtomicReference<>();
+    private AtomicReference<GridDiscoveryTopologySnapshot> topSnapshot = new AtomicReference<>();
 
     /** Last committed cache version before next topology version use. */
     private AtomicReference<GridCacheVersion> lastVer = new AtomicReference<>();
@@ -146,8 +147,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** Dynamic cache change requests. */
     private Collection<DynamicCacheChangeRequest> reqs;
 
+    /** Cache validation results. */
     private volatile Map<Integer, Boolean> cacheValidRes;
 
+    /** Skip preload flag. */
+    private boolean skipPreload;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -200,6 +205,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param cctx Cache context.
      * @param busyLock Busy lock.
      * @param exchId Exchange ID.
+     * @param reqs Cache change requests.
      */
     public GridDhtPartitionsExchangeFuture(
         GridCacheSharedContext cctx,
@@ -221,16 +227,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
         log = cctx.logger(getClass());
 
-        // Grab all nodes with order of equal or less than last joined node.
-        oldestNode.set(CU.oldest(cctx, exchId.topologyVersion()));
-
-        assert oldestNode.get() != null;
-
         initFut = new GridFutureAdapter<>();
 
         if (log.isDebugEnabled())
-            log.debug("Creating exchange future [localNode=" + cctx.localNodeId() +
-                ", fut=" + this + ']');
+            log.debug("Creating exchange future [localNode=" + cctx.localNodeId() + ", fut=" + this + ']');
+    }
+
+    /**
+     * @param reqs Cache change requests.
+     */
+    public void cacheChangeRequests(Collection<DynamicCacheChangeRequest> reqs) {
+        this.reqs = reqs;
     }
 
     /** {@inheritDoc} */
@@ -250,6 +257,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @return Skip preload flag.
+     */
+    public boolean skipPreload() {
+        return skipPreload;
+    }
+
+    /**
      * @return Dummy flag.
      */
     public boolean dummy() {
@@ -279,9 +293,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /**
      * @param cacheId Cache ID to check.
+     * @param topVer Topology version.
      * @return {@code True} if cache was added during this exchange.
      */
-    public boolean isCacheAdded(int cacheId) {
+    public boolean isCacheAdded(int cacheId, AffinityTopologyVersion topVer) {
         if (!F.isEmpty(reqs)) {
             for (DynamicCacheChangeRequest req : reqs) {
                 if (req.start() && !req.clientStartOnly()) {
@@ -291,7 +306,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             }
         }
 
-        return false;
+        GridCacheContext<?, ?> cacheCtx = cctx.cacheContext(cacheId);
+
+        return cacheCtx != null && F.eq(cacheCtx.startTopologyVersion(), topVer);
     }
 
     /**
@@ -312,7 +329,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
-     * Rechecks topology.
+     * @param cacheCtx Cache context.
+     * @throws IgniteCheckedException If failed.
      */
     private void initTopology(GridCacheContext cacheCtx) throws IgniteCheckedException {
         if (stopping(cacheCtx.cacheId()))
@@ -330,8 +348,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     exchId + ']');
 
             // Fetch affinity assignment from remote node.
-            GridDhtAssignmentFetchFuture fetchFut =
-                new GridDhtAssignmentFetchFuture(cacheCtx, exchId.topologyVersion(), CU.affinityNodes(cacheCtx));
+            GridDhtAssignmentFetchFuture fetchFut = new GridDhtAssignmentFetchFuture(cacheCtx,
+                exchId.topologyVersion(),
+                CU.affinityNodes(cacheCtx, exchId.topologyVersion()));
 
             fetchFut.init();
 
@@ -341,11 +360,21 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 log.debug("Fetched affinity from remote node, initializing affinity assignment [locNodeId=" +
                     cctx.localNodeId() + ", topVer=" + exchId.topologyVersion() + ']');
 
+            if (affAssignment == null) {
+                affAssignment = new ArrayList<>(cacheCtx.affinity().partitions());
+
+                List<ClusterNode> empty = Collections.emptyList();
+
+                for (int i = 0; i < cacheCtx.affinity().partitions(); i++)
+                    affAssignment.add(empty);
+            }
+
             cacheCtx.affinity().initializeAffinity(exchId.topologyVersion(), affAssignment);
         }
     }
 
     /**
+     * @param cacheCtx Cache context.
      * @return {@code True} if local node can calculate affinity on it's own for this partition map exchange.
      */
     private boolean canCalculateAffinity(GridCacheContext cacheCtx) {
@@ -391,20 +420,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
-     * @return Exchange id.
-     */
-    GridDhtPartitionExchangeId key() {
-        return exchId;
-    }
-
-    /**
-     * @return Oldest node.
-     */
-    ClusterNode oldestNode() {
-        return oldestNode.get();
-    }
-
-    /**
      * @return Exchange ID.
      */
     public GridDhtPartitionExchangeId exchangeId() {
@@ -412,13 +427,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
-     * @return Init future.
-     */
-    IgniteInternalFuture<?> initFuture() {
-        return initFut;
-    }
-
-    /**
      * @return {@code true} if entered to busy state.
      */
     private boolean enterBusy() {
@@ -444,7 +452,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @throws IgniteInterruptedCheckedException If interrupted.
      */
     public void init() throws IgniteInterruptedCheckedException {
-        assert oldestNode.get() != null;
+        if (isDone())
+            return;
 
         if (init.compareAndSet(false, true)) {
             if (isDone())
@@ -455,10 +464,118 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 // will return corresponding nodes.
                 U.await(evtLatch);
 
+                assert discoEvt != null : this;
+                assert !dummy && !forcePreload : this;
+
+                ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, exchId.topologyVersion());
+
+                oldestNode.set(oldest);
+
                 startCaches();
 
+                // True if client node joined or failed.
+                boolean clientNodeEvt;
+
+                if (F.isEmpty(reqs)) {
+                    int type = discoEvt.type();
+
+                    assert type == EVT_NODE_JOINED || type == EVT_NODE_LEFT || type == EVT_NODE_FAILED : discoEvt;
+
+                    clientNodeEvt = CU.clientNode(discoEvt.eventNode());
+                }
+                else {
+                    assert discoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT : discoEvt;
+
+                    boolean clientOnlyStart = true;
+
+                    for (DynamicCacheChangeRequest req : reqs) {
+                        if (!req.clientStartOnly()) {
+                            clientOnlyStart = false;
+
+                            break;
+                        }
+                    }
+
+                    clientNodeEvt = clientOnlyStart;
+                }
+
+                if (clientNodeEvt) {
+                    ClusterNode node = discoEvt.eventNode();
+
+                    // Client need to initialize affinity for local join event or for stated client caches.
+                    if (!node.isLocal()) {
+                        for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                            if (cacheCtx.isLocal())
+                                continue;
+
+                            GridDhtPartitionTopology top = cacheCtx.topology();
+
+                            top.updateTopologyVersion(exchId, this, -1, stopping(cacheCtx.cacheId()));
+
+                            if (cacheCtx.affinity().affinityTopologyVersion() == AffinityTopologyVersion.NONE) {
+                                initTopology(cacheCtx);
+
+                                top.beforeExchange(this);
+                            }
+                            else
+                                cacheCtx.affinity().clientEventTopologyChange(discoEvt, exchId.topologyVersion());
+                        }
+
+                        if (exchId.isLeft())
+                            cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
+
+                        onDone(exchId.topologyVersion());
+
+                        skipPreload = cctx.kernalContext().clientNode();
+
+                        return;
+                    }
+                }
+
+                if (cctx.kernalContext().clientNode()) {
+                    skipPreload = true;
+
+                    for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                        if (cacheCtx.isLocal())
+                            continue;
+
+                        GridDhtPartitionTopology top = cacheCtx.topology();
+
+                        top.updateTopologyVersion(exchId, this, -1, stopping(cacheCtx.cacheId()));
+                    }
+
+                    for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                        if (cacheCtx.isLocal())
+                            continue;
+
+                        initTopology(cacheCtx);
+                    }
+
+                    if (oldestNode.get() != 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();
+                    }
+                    else
+                        onDone(exchId.topologyVersion());
+
+                    return;
+                }
+
+                assert oldestNode.get() != null;
+
                 for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                    if (isCacheAdded(cacheCtx.cacheId())) {
+                    if (isCacheAdded(cacheCtx.cacheId(), exchId.topologyVersion())) {
                         if (cacheCtx.discovery().cacheAffinityNodes(cacheCtx.name(), topologyVersion()).isEmpty())
                             U.quietAndWarn(log, "No server nodes found for cache client: " + cacheCtx.namex());
                     }
@@ -468,8 +585,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                 List<String> cachesWithoutNodes = null;
 
-                for (String name : cctx.cache().cacheNames()) {
-                    if (exchId.isLeft()) {
+                if (exchId.isLeft()) {
+                    for (String name : cctx.cache().cacheNames()) {
                         if (cctx.discovery().cacheAffinityNodes(name, topologyVersion()).isEmpty()) {
                             if (cachesWithoutNodes == null)
                                 cachesWithoutNodes = new ArrayList<>();
@@ -505,7 +622,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 }
 
                 if (cachesWithoutNodes != null) {
-                    StringBuilder sb = new StringBuilder("All server nodes for the following caches have left the cluster: ");
+                    StringBuilder sb =
+                        new StringBuilder("All server nodes for the following caches have left the cluster: ");
 
                     for (int i = 0; i < cachesWithoutNodes.size(); i++) {
                         String cache = cachesWithoutNodes.get(i);
@@ -537,7 +655,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 }
 
                 // Grab all alive remote nodes with order of equal or less than last joined node.
-                rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteCacheNodes(cctx,
+                rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteServerNodesWithCaches(cctx,
                     exchId.topologyVersion()));
 
                 rmtIds = Collections.unmodifiableSet(new HashSet<>(F.nodeIds(rmtNodes)));
@@ -591,6 +709,28 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 if (exchId.isLeft())
                     cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
 
+                IgniteInternalFuture<?> locksFut = cctx.mvcc().finishLocks(exchId.topologyVersion());
+
+                while (true) {
+                    try {
+                        locksFut.get(2 * cctx.gridConfig().getNetworkTimeout(), TimeUnit.MILLISECONDS);
+
+                        break;
+                    }
+                    catch (IgniteFutureTimeoutCheckedException ignored) {
+                        U.warn(log, "Failed to wait for locks release future. " +
+                            "Dumping pending objects that might be the cause: " + cctx.localNodeId());
+
+                        U.warn(log, "Locked entries:");
+
+                        Map<IgniteTxKey, Collection<GridCacheMvccCandidate>> locks =
+                            cctx.mvcc().unfinishedLocks(exchId.topologyVersion());
+
+                        for (Map.Entry<IgniteTxKey, Collection<GridCacheMvccCandidate>> e : locks.entrySet())
+                            U.warn(log, "Locked entry [key=" + e.getKey() + ", mvcc=" + e.getValue() + ']');
+                    }
+                }
+
                 for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                     if (cacheCtx.isLocal())
                         continue;
@@ -650,36 +790,25 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             if (log.isDebugEnabled())
                 log.debug("Initialized future: " + this);
 
-            if (canSkipExchange())
-                onDone(exchId.topologyVersion());
+            // If this node is not oldest.
+            if (!oldestNode.get().id().equals(cctx.localNodeId()))
+                sendPartitions();
             else {
-                // If this node is not oldest.
-                if (!oldestNode.get().id().equals(cctx.localNodeId()))
-                    sendPartitions();
-                else {
-                    boolean allReceived = allReceived();
+                boolean allReceived = allReceived();
 
-                    if (allReceived && replied.compareAndSet(false, true)) {
-                        if (spreadPartitions())
-                            onDone(exchId.topologyVersion());
-                    }
+                if (allReceived && replied.compareAndSet(false, true)) {
+                    if (spreadPartitions())
+                        onDone(exchId.topologyVersion());
                 }
-
-                scheduleRecheck();
             }
+
+            scheduleRecheck();
         }
         else
             assert false : "Skipped init future: " + this;
     }
 
     /**
-     * @return {@code True} if no distributed exchange is needed.
-     */
-    private boolean canSkipExchange() {
-        return false; // TODO ignite-23;
-    }
-
-    /**
      *
      */
     private void dumpPendingObjects() {
@@ -755,7 +884,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @throws IgniteCheckedException If failed.
      */
     private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) throws IgniteCheckedException {
-        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id, cctx.versions().last());
+        GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
+            cctx.kernalContext().clientNode(),
+            cctx.versions().last());
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
             if (!cacheCtx.isLocal())
@@ -780,8 +911,14 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             id.topologyVersion());
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (!cacheCtx.isLocal())
-                m.addFullPartitionsMap(cacheCtx.cacheId(), cacheCtx.topology().partitionMap(true));
+            if (!cacheCtx.isLocal()) {
+                AffinityTopologyVersion startTopVer = cacheCtx.startTopologyVersion();
+
+                boolean ready = startTopVer == null || startTopVer.compareTo(id.topologyVersion()) <= 0;
+
+                if (ready)
+                    m.addFullPartitionsMap(cacheCtx.cacheId(), cacheCtx.topology().partitionMap(true));
+            }
         }
 
         // It is important that client topologies be added after contexts.
@@ -839,14 +976,18 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /** {@inheritDoc} */
     @Override public boolean onDone(AffinityTopologyVersion res, Throwable err) {
-        Map<Integer, Boolean> m = new HashMap<>();
+        Map<Integer, Boolean> m = null;
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-            if (cacheCtx.config().getTopologyValidator() != null && !CU.isSystemCache(cacheCtx.name()))
+            if (cacheCtx.config().getTopologyValidator() != null && !CU.isSystemCache(cacheCtx.name())) {
+                if (m == null)
+                    m = new HashMap<>();
+
                 m.put(cacheCtx.cacheId(), cacheCtx.config().getTopologyValidator().validate(discoEvt.topologyNodes()));
+            }
         }
 
-        cacheValidRes = m;
+        cacheValidRes = m != null ? m : Collections.<Integer, Boolean>emptyMap();
 
         cctx.cache().onExchangeDone(exchId.topologyVersion(), reqs, err);
 
@@ -864,8 +1005,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             if (timeoutObj != null)
                 cctx.kernalContext().timeout().removeTimeoutObject(timeoutObj);
 
-            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                if (exchId.event() == EventType.EVT_NODE_FAILED || exchId.event() == EventType.EVT_NODE_LEFT)
+            if (exchId.isLeft()) {
+                for (GridCacheContext cacheCtx : cctx.cacheContexts())
                     cacheCtx.config().getAffinity().removeNode(exchId.nodeId());
             }
 
@@ -1018,39 +1159,39 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             return;
         }
 
-        ClusterNode curOldest = oldestNode.get();
+        if (log.isDebugEnabled())
+            log.debug("Received full partition map from node [nodeId=" + nodeId + ", msg=" + msg + ']');
 
-        if (!nodeId.equals(curOldest.id())) {
-            if (log.isDebugEnabled())
-                log.debug("Received full partition map from unexpected node [oldest=" + curOldest.id() +
-                    ", unexpectedNodeId=" + nodeId + ']');
+        assert exchId.topologyVersion().equals(msg.topologyVersion());
 
-            ClusterNode sender = cctx.discovery().node(nodeId);
+        initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+            @Override public void apply(IgniteInternalFuture<Boolean> t) {
+                ClusterNode curOldest = oldestNode.get();
 
-            if (sender == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Sender node left grid, will ignore message from unexpected node [nodeId=" + nodeId +
-                        ", exchId=" + msg.exchangeId() + ']');
+                if (!nodeId.equals(curOldest.id())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Received full partition map from unexpected node [oldest=" + curOldest.id() +
+                            ", unexpectedNodeId=" + nodeId + ']');
 
-                return;
-            }
+                    ClusterNode snd = cctx.discovery().node(nodeId);
 
-            // Will process message later if sender node becomes oldest node.
-            if (sender.order() > curOldest.order())
-                fullMsgs.put(nodeId, msg);
+                    if (snd == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Sender node left grid, will ignore message from unexpected node [nodeId=" + nodeId +
+                                ", exchId=" + msg.exchangeId() + ']');
 
-            return;
-        }
+                        return;
+                    }
 
-        assert msg.exchangeId().equals(exchId);
+                    // Will process message later if sender node becomes oldest node.
+                    if (snd.order() > curOldest.order())
+                        fullMsgs.put(nodeId, msg);
 
-        if (log.isDebugEnabled())
-            log.debug("Received full partition map from node [nodeId=" + nodeId + ", msg=" + msg + ']');
+                    return;
+                }
 
-        assert exchId.topologyVersion().equals(msg.topologyVersion());
+                assert msg.exchangeId().equals(exchId);
 
-        initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
-            @Override public void apply(IgniteInternalFuture<Boolean> t) {
                 assert msg.lastVersion() != null;
 
                 cctx.versions().onReceived(nodeId, msg.lastVersion());
@@ -1075,8 +1216,12 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
             if (cacheCtx != null)
                 cacheCtx.topology().update(exchId, entry.getValue());
-            else if (CU.oldest(cctx).isLocal())
-                cctx.exchange().clientTopology(cacheId, this).update(exchId, entry.getValue());
+            else {
+                ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+
+                if (oldest != null && oldest.isLocal())
+                    cctx.exchange().clientTopology(cacheId, this).update(exchId, entry.getValue());
+            }
         }
     }
 
@@ -1135,40 +1280,42 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                             boolean set = false;
 
-                            ClusterNode newOldest = CU.oldest(cctx, exchId.topologyVersion());
-
-                            // If local node is now oldest.
-                            if (newOldest.id().equals(cctx.localNodeId())) {
-                                synchronized (mux) {
-                                    if (oldestNode.compareAndSet(oldest, newOldest)) {
-                                        // If local node is just joining.
-                                        if (exchId.nodeId().equals(cctx.localNodeId())) {
-                                            try {
-                                                for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
-                                                    if (!cacheCtx.isLocal())
-                                                        cacheCtx.topology().beforeExchange(
-                                                            GridDhtPartitionsExchangeFuture.this);
+                            ClusterNode newOldest = CU.oldestAliveCacheServerNode(cctx, exchId.topologyVersion());
+
+                            if (newOldest != null) {
+                                // If local node is now oldest.
+                                if (newOldest.id().equals(cctx.localNodeId())) {
+                                    synchronized (mux) {
+                                        if (oldestNode.compareAndSet(oldest, newOldest)) {
+                                            // If local node is just joining.
+                                            if (exchId.nodeId().equals(cctx.localNodeId())) {
+                                                try {
+                                                    for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                                                        if (!cacheCtx.isLocal())
+                                                            cacheCtx.topology().beforeExchange(
+                                                                GridDhtPartitionsExchangeFuture.this);
+                                                    }
                                                 }
-                                            }
-                                            catch (IgniteCheckedException e) {
-                                                onDone(e);
+                                                catch (IgniteCheckedException e) {
+                                                    onDone(e);
 
-                                                return;
+                                                    return;
+                                                }
                                             }
-                                        }
 
-                                        set = true;
+                                            set = true;
+                                        }
                                     }
                                 }
-                            }
-                            else {
-                                synchronized (mux) {
-                                    set = oldestNode.compareAndSet(oldest, newOldest);
-                                }
+                                else {
+                                    synchronized (mux) {
+                                        set = oldestNode.compareAndSet(oldest, newOldest);
+                                    }
 
-                                if (set && log.isDebugEnabled())
-                                    log.debug("Reassigned oldest node [this=" + cctx.localNodeId() +
-                                        ", old=" + oldest.id() + ", new=" + newOldest.id() + ']');
+                                    if (set && log.isDebugEnabled())
+                                        log.debug("Reassigned oldest node [this=" + cctx.localNodeId() +
+                                            ", old=" + oldest.id() + ", new=" + newOldest.id() + ']');
+                                }
                             }
 
                             if (set) {
@@ -1190,9 +1337,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                             assert rmtNodes != null;
 
-                            for (Iterator<ClusterNode> it = rmtNodes.iterator(); it.hasNext(); )
+                            for (Iterator<ClusterNode> it = rmtNodes.iterator(); it.hasNext(); ) {
                                 if (it.next().id().equals(nodeId))
                                     it.remove();
+                            }
 
                             if (allReceived() && ready.get() && replied.compareAndSet(false, true))
                                 if (spreadPartitions())
@@ -1254,30 +1402,34 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             GridTimeoutObject timeoutObj = new GridTimeoutObjectAdapter(
                 cctx.gridConfig().getNetworkTimeout() * Math.max(1, cctx.gridConfig().getCacheConfiguration().length)) {
                 @Override public void onTimeout() {
-                    if (isDone())
-                        return;
-
-                    if (!enterBusy())
-                        return;
+                    cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                        @Override public void run() {
+                            if (isDone())
+                                return;
+
+                            if (!enterBusy())
+                                return;
+
+                            try {
+                                U.warn(log,
+                                    "Retrying preload partition exchange due to timeout [done=" + isDone() +
+                                        ", dummy=" + dummy + ", exchId=" + exchId + ", rcvdIds=" + F.id8s(rcvdIds) +
+                                        ", rmtIds=" + F.id8s(rmtIds) + ", remaining=" + F.id8s(remaining()) +
+                                        ", init=" + init + ", initFut=" + initFut.isDone() +
+                                        ", ready=" + ready + ", replied=" + replied + ", added=" + added +
+                                        ", oldest=" + U.id8(oldestNode.get().id()) + ", oldestOrder=" +
+                                        oldestNode.get().order() + ", evtLatch=" + evtLatch.getCount() +
+                                        ", locNodeOrder=" + cctx.localNode().order() +
+                                        ", locNodeId=" + cctx.localNode().id() + ']',
+                                    "Retrying preload partition exchange due to timeout.");
 
-                    try {
-                        U.warn(log,
-                            "Retrying preload partition exchange due to timeout [done=" + isDone() +
-                                ", dummy=" + dummy + ", exchId=" + exchId + ", rcvdIds=" + F.id8s(rcvdIds) +
-                                ", rmtIds=" + F.id8s(rmtIds) + ", remaining=" + F.id8s(remaining()) +
-                                ", init=" + init + ", initFut=" + initFut.isDone() +
-                                ", ready=" + ready + ", replied=" + replied + ", added=" + added +
-                                ", oldest=" + U.id8(oldestNode.get().id()) + ", oldestOrder=" +
-                                oldestNode.get().order() + ", evtLatch=" + evtLatch.getCount() +
-                                ", locNodeOrder=" + cctx.localNode().order() +
-                                ", locNodeId=" + cctx.localNode().id() + ']',
-                            "Retrying preload partition exchange due to timeout.");
-
-                        recheck();
-                    }
-                    finally {
-                        leaveBusy();
-                    }
+                                recheck();
+                            }
+                            finally {
+                                leaveBusy();
+                            }
+                        }
+                    });
                 }
             };
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
index 8256274..73794ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsFullMessage.java
@@ -59,8 +59,10 @@ public class GridDhtPartitionsFullMessage extends GridDhtPartitionsAbstractMessa
     /**
      * @param id Exchange ID.
      * @param lastVer Last version.
+     * @param topVer Topology version.
      */
-    public GridDhtPartitionsFullMessage(@Nullable GridDhtPartitionExchangeId id, @Nullable GridCacheVersion lastVer,
+    public GridDhtPartitionsFullMessage(@Nullable GridDhtPartitionExchangeId id,
+        @Nullable GridCacheVersion lastVer,
         @NotNull AffinityTopologyVersion topVer) {
         super(id, lastVer);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
index 66140cd..713a80b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
@@ -45,6 +45,9 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     /** Serialized partitions. */
     private byte[] partsBytes;
 
+    /** */
+    private boolean client;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -54,10 +57,22 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
     /**
      * @param exchId Exchange ID.
+     * @param client Client message flag.
      * @param lastVer Last version.
      */
-    public GridDhtPartitionsSingleMessage(GridDhtPartitionExchangeId exchId, @Nullable GridCacheVersion lastVer) {
+    public GridDhtPartitionsSingleMessage(GridDhtPartitionExchangeId exchId,
+        boolean client,
+        @Nullable GridCacheVersion lastVer) {
         super(exchId, lastVer);
+
+        this.client = client;
+    }
+
+    /**
+     * @return {@code True} if sent from client node.
+     */
+    public boolean client() {
+        return client;
     }
 
     /**
@@ -110,6 +125,12 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
         switch (writer.state()) {
             case 5:
+                if (!writer.writeBoolean("client", client))
+                    return false;
+
+                writer.incrementState();
+
+            case 6:
                 if (!writer.writeByteArray("partsBytes", partsBytes))
                     return false;
 
@@ -132,6 +153,14 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
         switch (reader.state()) {
             case 5:
+                client = reader.readBoolean("client");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 6:
                 partsBytes = reader.readByteArray("partsBytes");
 
                 if (!reader.isLastRead())
@@ -151,7 +180,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 6;
+        return 7;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index d6373f0..51010ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
@@ -46,7 +47,7 @@ import static org.apache.ignite.internal.util.GridConcurrentFactory.*;
 /**
  * DHT cache preloader.
  */
-public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
+public class GridDhtPreloader extends GridCachePreloaderAdapter {
     /** Default preload resend timeout. */
     public static final long DFLT_PRELOAD_RESEND_TIMEOUT = 1500;
 
@@ -57,13 +58,13 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     private final GridAtomicLong topVer = new GridAtomicLong();
 
     /** Force key futures. */
-    private final ConcurrentMap<IgniteUuid, GridDhtForceKeysFuture<K, V>> forceKeyFuts = newMap();
+    private final ConcurrentMap<IgniteUuid, GridDhtForceKeysFuture<?, ?>> forceKeyFuts = newMap();
 
     /** Partition suppliers. */
-    private GridDhtPartitionSupplyPool<K, V> supplyPool;
+    private GridDhtPartitionSupplyPool supplyPool;
 
     /** Partition demanders. */
-    private GridDhtPartitionDemandPool<K, V> demandPool;
+    private GridDhtPartitionDemandPool demandPool;
 
     /** Start future. */
     private final GridFutureAdapter<Object> startFut;
@@ -92,7 +93,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
 
                 assert !loc.id().equals(n.id());
 
-                for (GridDhtForceKeysFuture<K, V> f : forceKeyFuts.values())
+                for (GridDhtForceKeysFuture<?, ?> f : forceKeyFuts.values())
                     f.onDiscoveryEvent(e);
 
                 assert e.type() != EVT_NODE_JOINED || n.order() > loc.order() : "Node joined with smaller-than-local " +
@@ -117,7 +118,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     /**
      * @param cctx Cache context.
      */
-    public GridDhtPreloader(GridCacheContext<K, V> cctx) {
+    public GridDhtPreloader(GridCacheContext<?, ?> cctx) {
         super(cctx);
 
         top = cctx.dht().topology();
@@ -158,8 +159,8 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
                 }
             });
 
-        supplyPool = new GridDhtPartitionSupplyPool<>(cctx, busyLock);
-        demandPool = new GridDhtPartitionDemandPool<>(cctx, busyLock);
+        supplyPool = new GridDhtPartitionSupplyPool(cctx, busyLock);
+        demandPool = new GridDhtPartitionDemandPool(cctx, busyLock);
 
         cctx.events().addListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED);
     }
@@ -227,12 +228,14 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
 
             final long start = U.currentTimeMillis();
 
-            if (cctx.config().getRebalanceDelay() >= 0) {
-                U.log(log, "Starting rebalancing in " + cctx.config().getRebalanceMode() + " mode: " + cctx.name());
+            final CacheConfiguration cfg = cctx.config();
+
+            if (cfg.getRebalanceDelay() >= 0) {
+                U.log(log, "Starting rebalancing in " + cfg.getRebalanceMode() + " mode: " + cctx.name());
 
                 demandPool.syncFuture().listen(new CI1<Object>() {
                     @Override public void apply(Object t) {
-                        U.log(log, "Completed rebalancing in " + cctx.config().getRebalanceMode() + " mode " +
+                        U.log(log, "Completed rebalancing in " + cfg.getRebalanceMode() + " mode " +
                             "[cache=" + cctx.name() + ", time=" + (U.currentTimeMillis() - start) + " ms]");
                     }
                 });
@@ -253,12 +256,12 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public GridDhtPreloaderAssignments<K, V> assign(GridDhtPartitionsExchangeFuture exchFut) {
+    @Override public GridDhtPreloaderAssignments assign(GridDhtPartitionsExchangeFuture exchFut) {
         return demandPool.assign(exchFut);
     }
 
     /** {@inheritDoc} */
-    @Override public void addAssignments(GridDhtPreloaderAssignments<K, V> assignments, boolean forcePreload) {
+    @Override public void addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload) {
         demandPool.addAssignments(assignments, forcePreload);
     }
 
@@ -271,7 +274,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> syncFuture() {
-        return demandPool.syncFuture();
+        return cctx.kernalContext().clientNode() ? startFut : demandPool.syncFuture();
     }
 
     /**
@@ -406,7 +409,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
             return;
 
         try {
-            GridDhtForceKeysFuture<K, V> f = forceKeyFuts.get(msg.futureId());
+            GridDhtForceKeysFuture<?, ?> f = forceKeyFuts.get(msg.futureId());
 
             if (f != null)
                 f.onResult(node.id(), msg);
@@ -491,7 +494,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
      */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
     @Override public GridDhtFuture<Object> request(Collection<KeyCacheObject> keys, AffinityTopologyVersion topVer) {
-        final GridDhtForceKeysFuture<K, V> fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this);
+        final GridDhtForceKeysFuture<?, ?> fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this);
 
         IgniteInternalFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);
 
@@ -543,7 +546,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
      *
      * @param fut Future to add.
      */
-    void addFuture(GridDhtForceKeysFuture<K, V> fut) {
+    void addFuture(GridDhtForceKeysFuture<?, ?> fut) {
         forceKeyFuts.put(fut.futureId(), fut);
     }
 
@@ -552,7 +555,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
      *
      * @param fut Future to remove.
      */
-    void remoteFuture(GridDhtForceKeysFuture<K, V> fut) {
+    void remoteFuture(GridDhtForceKeysFuture<?, ?> fut) {
         forceKeyFuts.remove(fut.futureId(), fut);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
index 369fc68..2f6ef6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloaderAssignments.java
@@ -27,8 +27,7 @@ import java.util.concurrent.*;
 /**
  * Partition to node assignments.
  */
-public class GridDhtPreloaderAssignments<K, V> extends
-    ConcurrentHashMap<ClusterNode, GridDhtPartitionDemandMessage> {
+public class GridDhtPreloaderAssignments extends ConcurrentHashMap<ClusterNode, GridDhtPartitionDemandMessage> {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index ba3357d..041f83a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -433,6 +433,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public V tryPutIfAbsent(K key, V val) throws IgniteCheckedException {
+        return dht.tryPutIfAbsent(key, val);
+    }
+
+    /** {@inheritDoc} */
     @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
         return dht.getAndReplace(key, val);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 8258b14..351d6cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -95,7 +95,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     }
 
     /** {@inheritDoc} */
-    @Override public GridCachePreloader<K, V> preloader() {
+    @Override public GridCachePreloader preloader() {
         return dht().preloader();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index fc178e3..74438bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -274,7 +274,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
         if (affNodes.isEmpty()) {
             assert !cctx.affinityNode();
 
-            onDone(new ClusterTopologyCheckedException("Failed to map keys for near-only cache (all partition " +
+            onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for near-only cache (all partition " +
                 "nodes left the grid)."));
 
             return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 0ffb4e5..3d28018 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -45,7 +45,7 @@ import static org.apache.ignite.events.EventType.*;
 /**
  * Cache lock future.
  */
-public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<Boolean>
+public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean>
     implements GridCacheMvccFuture<Boolean> {
     /** */
     private static final long serialVersionUID = 0L;
@@ -58,7 +58,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
     /** Cache registry. */
     @GridToStringExclude
-    private GridCacheContext<K, V> cctx;
+    private GridCacheContext<?, ?> cctx;
 
     /** Lock owner thread. */
     @GridToStringInclude
@@ -135,7 +135,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * @param skipStore skipStore
      */
     public GridNearLockFuture(
-        GridCacheContext<K, V> cctx,
+        GridCacheContext<?, ?> cctx,
         Collection<KeyCacheObject> keys,
         @Nullable GridNearTxLocal tx,
         boolean read,
@@ -184,15 +184,14 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * @return Participating nodes.
      */
     @Override public Collection<? extends ClusterNode> nodes() {
-        return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
-                    if (isMini(f))
-                        return ((MiniFuture)f).node();
+        return F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+            @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
+                if (isMini(f))
+                    return ((MiniFuture)f).node();
 
-                    return cctx.discovery().localNode();
-                }
-            });
+                return cctx.discovery().localNode();
+            }
+        });
     }
 
     /** {@inheritDoc} */
@@ -350,13 +349,14 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * Undoes all locks.
      *
      * @param dist If {@code true}, then remove locks from remote nodes as well.
+     * @param rollback {@code True} if should rollback tx.
      */
-    private void undoLocks(boolean dist) {
+    private void undoLocks(boolean dist, boolean rollback) {
         // Transactions will undo during rollback.
         if (dist && tx == null)
             cctx.nearTx().removeLocks(lockVer, keys);
         else {
-            if (tx != null) {
+            if (rollback && tx != null) {
                 if (tx.setRollbackOnly()) {
                     if (log.isDebugEnabled())
                         log.debug("Marked transaction as rollback only because locks could not be acquired: " + tx);
@@ -397,7 +397,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * @param dist {@code True} if need to distribute lock release.
      */
     private void onFailed(boolean dist) {
-        undoLocks(dist);
+        undoLocks(dist, true);
 
         complete(false);
     }
@@ -607,7 +607,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                 ", fut=" + this + ']');
 
         if (!success)
-            undoLocks(distribute);
+            undoLocks(distribute, true);
 
         if (tx != null)
             cctx.tm().txContext(tx);
@@ -682,7 +682,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             // Continue mapping on the same topology version as it was before.
             this.topVer.compareAndSet(null, topVer);
 
-            map(keys);
+            map(keys, false);
 
             markInitialized();
 
@@ -690,14 +690,16 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
         }
 
         // Must get topology snapshot and map on that version.
-        mapOnTopology();
+        mapOnTopology(false);
     }
 
     /**
      * Acquires topology future and checks it completeness under the read lock. If it is not complete,
      * will asynchronously wait for it's completeness and then try again.
+     *
+     * @param remap Remap flag.
      */
-    void mapOnTopology() {
+    void mapOnTopology(final boolean remap) {
         // We must acquire topology snapshot from the topology version future.
         cctx.topology().readLock();
 
@@ -721,19 +723,27 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
                 AffinityTopologyVersion topVer = fut.topologyVersion();
 
-                if (tx != null)
-                    tx.topologyVersion(topVer);
+                if (remap) {
+                    if (tx != null)
+                        tx.onRemap(topVer);
 
-                this.topVer.compareAndSet(null, topVer);
+                    this.topVer.set(topVer);
+                }
+                else {
+                    if (tx != null)
+                        tx.topologyVersion(topVer);
+
+                    this.topVer.compareAndSet(null, topVer);
+                }
 
-                map(keys);
+                map(keys, remap);
 
                 markInitialized();
             }
             else {
                 fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                        mapOnTopology();
+                        mapOnTopology(remap);
                     }
                 });
             }
@@ -749,14 +759,15 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * groups belonging to one primary node and locks for these groups are acquired sequentially.
      *
      * @param keys Keys.
+     * @param remap Remap flag.
      */
-    private void map(Iterable<KeyCacheObject> keys) {
+    private void map(Iterable<KeyCacheObject> keys, boolean remap) {
         try {
             AffinityTopologyVersion topVer = this.topVer.get();
 
             assert topVer != null;
 
-            assert topVer.topologyVersion() > 0;
+            assert topVer.topologyVersion() > 0 : topVer;
 
             if (CU.affinityNodes(cctx, topVer).isEmpty()) {
                 onDone(new ClusterTopologyServerNotFoundException("Failed to map keys for near-only cache (all " +
@@ -765,8 +776,11 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                 return;
             }
 
-            ConcurrentLinkedDeque8<GridNearLockMapping> mappings =
-                new ConcurrentLinkedDeque8<>();
+            boolean clientNode = cctx.kernalContext().clientNode();
+
+            assert !remap || (clientNode && (tx == null || !tx.hasRemoteLocks()));
+
+            ConcurrentLinkedDeque8<GridNearLockMapping> mappings = new ConcurrentLinkedDeque8<>();
 
             // Assign keys to primary nodes.
             GridNearLockMapping map = null;
@@ -795,6 +809,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             if (log.isDebugEnabled())
                 log.debug("Starting (re)map for mappings [mappings=" + mappings + ", fut=" + this + ']');
 
+            boolean first = true;
+
             // Create mini futures.
             for (Iterator<GridNearLockMapping> iter = mappings.iterator(); iter.hasNext(); ) {
                 GridNearLockMapping mapping = iter.next();
@@ -872,6 +888,14 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
                                 if (!cand.reentry()) {
                                     if (req == null) {
+                                        boolean clientFirst = false;
+
+                                        if (first) {
+                                            clientFirst = clientNode && (tx == null || !tx.hasRemoteLocks());
+
+                                            first = false;
+                                        }
+
                                         req = new GridNearLockRequest(
                                             cctx.cacheId(),
                                             topVer,
@@ -893,7 +917,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                                             inTx() ? tx.subjectId() : null,
                                             inTx() ? tx.taskNameHash() : 0,
                                             read ? accessTtl : -1L,
-                                            skipStore);
+                                            skipStore,
+                                            clientFirst);
 
                                         mapping.request(req);
                                     }
@@ -1197,7 +1222,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
     /**
      * @return DHT cache.
      */
-    private GridDhtTransactionalCacheAdapter<K, V> dht() {
+    private GridDhtTransactionalCacheAdapter<?, ?> dht() {
         return cctx.nearTx().dht();
     }
 
@@ -1356,110 +1381,146 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                     return;
                 }
 
-                int i = 0;
+                if (res.clientRemapVersion() != null) {
+                    assert cctx.kernalContext().clientNode();
 
-                AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer.get();
+                    IgniteInternalFuture<?> affFut =
+                        cctx.shared().exchange().affinityReadyFuture(res.clientRemapVersion());
 
-                for (KeyCacheObject k : keys) {
-                    while (true) {
-                        GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
+                    if (affFut != null && !affFut.isDone()) {
+                        affFut.listen(new CI1<IgniteInternalFuture<?>>() {
+                            @Override public void apply(IgniteInternalFuture<?> fut) {
+                                remap();
+                            }
+                        });
+                    }
+                    else
+                        remap();
+                }
+                else {
+                    int i = 0;
 
-                        try {
-                            if (res.dhtVersion(i) == null) {
-                                onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
-                                    "(will fail the lock): " + res));
+                    AffinityTopologyVersion topVer = GridNearLockFuture.this.topVer.get();
 
-                                return;
-                            }
+                    for (KeyCacheObject k : keys) {
+                        while (true) {
+                            GridNearCacheEntry entry = cctx.near().entryExx(k, topVer);
 
-                            IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
+                            try {
+                                if (res.dhtVersion(i) == null) {
+                                    onDone(new IgniteCheckedException("Failed to receive DHT version from remote node " +
+                                        "(will fail the lock): " + res));
 
-                            CacheObject oldVal = entry.rawGet();
-                            boolean hasOldVal = false;
-                            CacheObject newVal = res.value(i);
+                                    return;
+                                }
 
-                            boolean readRecordable = false;
+                                IgniteBiTuple<GridCacheVersion, CacheObject> oldValTup = valMap.get(entry.key());
 
-                            if (retval) {
-                                readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
+                                CacheObject oldVal = entry.rawGet();
+                                boolean hasOldVal = false;
+                                CacheObject newVal = res.value(i);
 
-                                if (readRecordable)
-                                    hasOldVal = entry.hasValue();
-                            }
+                                boolean readRecordable = false;
 
-                            GridCacheVersion dhtVer = res.dhtVersion(i);
-                            GridCacheVersion mappedVer = res.mappedVersion(i);
+                                if (retval) {
+                                    readRecordable = cctx.events().isRecordable(EVT_CACHE_OBJECT_READ);
+
+                                    if (readRecordable)
+                                        hasOldVal = entry.hasValue();
+                                }
 
-                            if (newVal == null) {
-                                if (oldValTup != null) {
-                                    if (oldValTup.get1().equals(dhtVer))
-                                        newVal = oldValTup.get2();
+                                GridCacheVersion dhtVer = res.dhtVersion(i);
+                                GridCacheVersion mappedVer = res.mappedVersion(i);
 
-                                    oldVal = oldValTup.get2();
+                                if (newVal == null) {
+                                    if (oldValTup != null) {
+                                        if (oldValTup.get1().equals(dhtVer))
+                                            newVal = oldValTup.get2();
+
+                                        oldVal = oldValTup.get2();
+                                    }
                                 }
-                            }
 
-                            // Lock is held at this point, so we can set the
-                            // returned value if any.
-                            entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
+                                // Lock is held at this point, so we can set the
+                                // returned value if any.
+                                entry.resetFromPrimary(newVal, lockVer, dhtVer, node.id(), topVer);
 
-                            if (inTx() && implicitTx() && tx.onePhaseCommit()) {
-                                boolean pass = res.filterResult(i);
+                                if (inTx()) {
+                                    tx.hasRemoteLocks(true);
 
-                                tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
-                            }
+                                    if (implicitTx() && tx.onePhaseCommit()) {
+                                        boolean pass = res.filterResult(i);
 
-                            entry.readyNearLock(lockVer, mappedVer, res.committedVersions(), res.rolledbackVersions(),
-                                res.pending());
-
-                            if (retval) {
-                                if (readRecordable)
-                                    cctx.events().addEvent(
-                                        entry.partition(),
-                                        entry.key(),
-                                        tx,
-                                        null,
-                                        EVT_CACHE_OBJECT_READ,
-                                        newVal,
-                                        newVal != null,
-                                        oldVal,
-                                        hasOldVal,
-                                        CU.subjectId(tx, cctx.shared()),
-                                        null,
-                                        inTx() ? tx.resolveTaskName() : null);
-
-                                if (cctx.cache().configuration().isStatisticsEnabled())
-                                    cctx.cache().metrics0().onRead(false);
-                            }
+                                        tx.entry(cctx.txKey(k)).filters(pass ? CU.empty0() : CU.alwaysFalse0Arr());
+                                    }
+                                }
 
-                            if (log.isDebugEnabled())
-                                log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
+                                entry.readyNearLock(lockVer,
+                                    mappedVer,
+                                    res.committedVersions(),
+                                    res.rolledbackVersions(),
+                                    res.pending());
+
+                                if (retval) {
+                                    if (readRecordable)
+                                        cctx.events().addEvent(
+                                            entry.partition(),
+                                            entry.key(),
+                                            tx,
+                                            null,
+                                            EVT_CACHE_OBJECT_READ,
+                                            newVal,
+                                            newVal != null,
+                                            oldVal,
+                                            hasOldVal,
+                                            CU.subjectId(tx, cctx.shared()),
+                                            null,
+                                            inTx() ? tx.resolveTaskName() : null);
+
+                                    if (cctx.cache().configuration().isStatisticsEnabled())
+                                        cctx.cache().metrics0().onRead(false);
+                                }
 
-                            break; // Inner while loop.
-                        }
-                        catch (GridCacheEntryRemovedException ignored) {
-                            if (log.isDebugEnabled())
-                                log.debug("Failed to add candidates because entry was removed (will renew).");
+                                if (log.isDebugEnabled())
+                                    log.debug("Processed response for entry [res=" + res + ", entry=" + entry + ']');
 
-                            // Replace old entry with new one.
-                            entries.set(i, (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
+                                break; // Inner while loop.
+                            }
+                            catch (GridCacheEntryRemovedException ignored) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Failed to add candidates because entry was removed (will renew).");
+
+                                // Replace old entry with new one.
+                                entries.set(i, (GridDistributedCacheEntry)cctx.cache().entryEx(entry.key()));
+                            }
                         }
+
+                        i++;
                     }
 
-                    i++;
-                }
+                    try {
+                        proceedMapping(mappings);
+                    }
+                    catch (IgniteCheckedException e) {
+                        onDone(e);
+                    }
 
-                try {
-                    proceedMapping(mappings);
-                }
-                catch (IgniteCheckedException e) {
-                    onDone(e);
+                    onDone(true);
                 }
-
-                onDone(true);
             }
         }
 
+        /**
+         *
+         */
+        private void remap() {
+            undoLocks(false, false);
+
+            mapOnTopology(true);
+
+            onDone(true);
+        }
+
         /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(MiniFuture.class, this, "node", node.id(), "super", super.toString());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
index e71dd65..81184a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockRequest.java
@@ -80,6 +80,9 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
     /** Flag indicating whether cache operation requires a previous value. */
     private boolean retVal;
 
+    /** {@code True} if first lock request for lock operation sent from client node. */
+    private boolean firstClientReq;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -98,6 +101,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
      * @param implicitTx Flag to indicate that transaction is implicit.
      * @param implicitSingleTx Implicit-transaction-with-one-key flag.
      * @param isRead Indicates whether implicit lock is for read or write operation.
+     * @param retVal Return value flag.
      * @param isolation Transaction isolation.
      * @param isInvalidate Invalidation flag.
      * @param timeout Lock timeout.
@@ -108,6 +112,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
      * @param taskNameHash Task name hash code.
      * @param accessTtl TTL for read operation.
      * @param skipStore Skip store flag.
+     * @param firstClientReq {@code True} if first lock request for lock operation sent from client node.
      */
     public GridNearLockRequest(
         int cacheId,
@@ -130,7 +135,8 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         @Nullable UUID subjId,
         int taskNameHash,
         long accessTtl,
-        boolean skipStore
+        boolean skipStore,
+        boolean firstClientReq
     ) {
         super(
             cacheId,
@@ -158,11 +164,19 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
         this.taskNameHash = taskNameHash;
         this.accessTtl = accessTtl;
         this.retVal = retVal;
+        this.firstClientReq = firstClientReq;
 
         dhtVers = new GridCacheVersion[keyCnt];
     }
 
     /**
+     * @return {@code True} if first lock request for lock operation sent from client node.
+     */
+    public boolean firstClientRequest() {
+        return firstClientReq;
+    }
+
+    /**
      * @return Topology version.
      */
     @Override public AffinityTopologyVersion topologyVersion() {
@@ -368,60 +382,66 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 writer.incrementState();
 
             case 24:
-                if (!writer.writeBoolean("hasTransforms", hasTransforms))
+                if (!writer.writeBoolean("firstClientReq", firstClientReq))
                     return false;
 
                 writer.incrementState();
 
             case 25:
-                if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx))
+                if (!writer.writeBoolean("hasTransforms", hasTransforms))
                     return false;
 
                 writer.incrementState();
 
             case 26:
-                if (!writer.writeBoolean("implicitTx", implicitTx))
+                if (!writer.writeBoolean("implicitSingleTx", implicitSingleTx))
                     return false;
 
                 writer.incrementState();
 
             case 27:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeBoolean("implicitTx", implicitTx))
                     return false;
 
                 writer.incrementState();
 
             case 28:
-                if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit))
+                if (!writer.writeIgniteUuid("miniId", miniId))
                     return false;
 
                 writer.incrementState();
 
             case 29:
-                if (!writer.writeBoolean("retVal", retVal))
+                if (!writer.writeBoolean("onePhaseCommit", onePhaseCommit))
                     return false;
 
                 writer.incrementState();
 
             case 30:
-                if (!writer.writeUuid("subjId", subjId))
+                if (!writer.writeBoolean("retVal", retVal))
                     return false;
 
                 writer.incrementState();
 
             case 31:
-                if (!writer.writeBoolean("syncCommit", syncCommit))
+                if (!writer.writeUuid("subjId", subjId))
                     return false;
 
                 writer.incrementState();
 
             case 32:
-                if (!writer.writeInt("taskNameHash", taskNameHash))
+                if (!writer.writeBoolean("syncCommit", syncCommit))
                     return false;
 
                 writer.incrementState();
 
             case 33:
+                if (!writer.writeInt("taskNameHash", taskNameHash))
+                    return false;
+
+                writer.incrementState();
+
+            case 34:
                 if (!writer.writeMessage("topVer", topVer))
                     return false;
 
@@ -468,7 +488,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 24:
-                hasTransforms = reader.readBoolean("hasTransforms");
+                firstClientReq = reader.readBoolean("firstClientReq");
 
                 if (!reader.isLastRead())
                     return false;
@@ -476,7 +496,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 25:
-                implicitSingleTx = reader.readBoolean("implicitSingleTx");
+                hasTransforms = reader.readBoolean("hasTransforms");
 
                 if (!reader.isLastRead())
                     return false;
@@ -484,7 +504,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 26:
-                implicitTx = reader.readBoolean("implicitTx");
+                implicitSingleTx = reader.readBoolean("implicitSingleTx");
 
                 if (!reader.isLastRead())
                     return false;
@@ -492,7 +512,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 27:
-                miniId = reader.readIgniteUuid("miniId");
+                implicitTx = reader.readBoolean("implicitTx");
 
                 if (!reader.isLastRead())
                     return false;
@@ -500,7 +520,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 28:
-                onePhaseCommit = reader.readBoolean("onePhaseCommit");
+                miniId = reader.readIgniteUuid("miniId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -508,7 +528,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 29:
-                retVal = reader.readBoolean("retVal");
+                onePhaseCommit = reader.readBoolean("onePhaseCommit");
 
                 if (!reader.isLastRead())
                     return false;
@@ -516,7 +536,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 30:
-                subjId = reader.readUuid("subjId");
+                retVal = reader.readBoolean("retVal");
 
                 if (!reader.isLastRead())
                     return false;
@@ -524,7 +544,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 31:
-                syncCommit = reader.readBoolean("syncCommit");
+                subjId = reader.readUuid("subjId");
 
                 if (!reader.isLastRead())
                     return false;
@@ -532,7 +552,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 32:
-                taskNameHash = reader.readInt("taskNameHash");
+                syncCommit = reader.readBoolean("syncCommit");
 
                 if (!reader.isLastRead())
                     return false;
@@ -540,6 +560,14 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
                 reader.incrementState();
 
             case 33:
+                taskNameHash = reader.readInt("taskNameHash");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 34:
                 topVer = reader.readMessage("topVer");
 
                 if (!reader.isLastRead())
@@ -559,7 +587,7 @@ public class GridNearLockRequest extends GridDistributedLockRequest {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 34;
+        return 35;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
index 20928de..f324198 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockResponse.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -58,6 +59,9 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
     /** Filter evaluation results for fast-commit transactions. */
     private boolean[] filterRes;
 
+    /** {@code True} if client node should remap lock request. */
+    private AffinityTopologyVersion clientRemapVer;
+
     /**
      * Empty constructor (required by {@link Externalizable}).
      */
@@ -73,6 +77,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
      * @param filterRes {@code True} if need to allocate array for filter evaluation results.
      * @param cnt Count.
      * @param err Error.
+     * @param clientRemapVer {@code True} if client node should remap lock request.
      */
     public GridNearLockResponse(
         int cacheId,
@@ -81,13 +86,15 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
         IgniteUuid miniId,
         boolean filterRes,
         int cnt,
-        Throwable err
+        Throwable err,
+        AffinityTopologyVersion clientRemapVer
     ) {
         super(cacheId, lockVer, futId, cnt, err);
 
         assert miniId != null;
 
         this.miniId = miniId;
+        this.clientRemapVer = clientRemapVer;
 
         dhtVers = new GridCacheVersion[cnt];
         mappedVers = new GridCacheVersion[cnt];
@@ -97,6 +104,13 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
     }
 
     /**
+     * @return {@code True} if client node should remap lock request.
+     */
+    @Nullable public AffinityTopologyVersion clientRemapVersion() {
+        return clientRemapVer;
+    }
+
+    /**
      * Gets pending versions that are less than {@link #version()}.
      *
      * @return Pending versions.
@@ -192,30 +206,36 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
         switch (writer.state()) {
             case 11:
-                if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG))
+                if (!writer.writeMessage("clientRemapVer", clientRemapVer))
                     return false;
 
                 writer.incrementState();
 
             case 12:
-                if (!writer.writeBooleanArray("filterRes", filterRes))
+                if (!writer.writeObjectArray("dhtVers", dhtVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 13:
-                if (!writer.writeObjectArray("mappedVers", mappedVers, MessageCollectionItemType.MSG))
+                if (!writer.writeBooleanArray("filterRes", filterRes))
                     return false;
 
                 writer.incrementState();
 
             case 14:
-                if (!writer.writeIgniteUuid("miniId", miniId))
+                if (!writer.writeObjectArray("mappedVers", mappedVers, MessageCollectionItemType.MSG))
                     return false;
 
                 writer.incrementState();
 
             case 15:
+                if (!writer.writeIgniteUuid("miniId", miniId))
+                    return false;
+
+                writer.incrementState();
+
+            case 16:
                 if (!writer.writeCollection("pending", pending, MessageCollectionItemType.MSG))
                     return false;
 
@@ -238,7 +258,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
         switch (reader.state()) {
             case 11:
-                dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
+                clientRemapVer = reader.readMessage("clientRemapVer");
 
                 if (!reader.isLastRead())
                     return false;
@@ -246,7 +266,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
                 reader.incrementState();
 
             case 12:
-                filterRes = reader.readBooleanArray("filterRes");
+                dhtVers = reader.readObjectArray("dhtVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -254,7 +274,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
                 reader.incrementState();
 
             case 13:
-                mappedVers = reader.readObjectArray("mappedVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
+                filterRes = reader.readBooleanArray("filterRes");
 
                 if (!reader.isLastRead())
                     return false;
@@ -262,7 +282,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
                 reader.incrementState();
 
             case 14:
-                miniId = reader.readIgniteUuid("miniId");
+                mappedVers = reader.readObjectArray("mappedVers", MessageCollectionItemType.MSG, GridCacheVersion.class);
 
                 if (!reader.isLastRead())
                     return false;
@@ -270,6 +290,14 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
                 reader.incrementState();
 
             case 15:
+                miniId = reader.readIgniteUuid("miniId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 16:
                 pending = reader.readCollection("pending", MessageCollectionItemType.MSG);
 
                 if (!reader.isLastRead())
@@ -289,7 +317,7 @@ public class GridNearLockResponse extends GridDistributedLockResponse {
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 16;
+        return 17;
     }
 
     /** {@inheritDoc} */


[12/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
new file mode 100644
index 0000000..adac0b2
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerAbstractSelfTest.java
@@ -0,0 +1,315 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.io.*;
+import java.sql.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ * Tests for store session listeners.
+ */
+public abstract class CacheStoreSessionListenerAbstractSelfTest extends GridCommonAbstractTest implements Serializable {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    protected static final String URL = "jdbc:h2:mem:example;DB_CLOSE_DELAY=-1";
+
+    /** */
+    protected static final AtomicInteger loadCacheCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger loadCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger writeCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger deleteCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicInteger reuseCnt = new AtomicInteger();
+
+    /** */
+    protected static final AtomicBoolean write = new AtomicBoolean();
+
+    /** */
+    protected static final AtomicBoolean fail = new AtomicBoolean();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table1");
+            conn.createStatement().executeUpdate("DROP TABLE IF EXISTS Table2");
+
+            conn.createStatement().executeUpdate("CREATE TABLE Table1 (id INT AUTO_INCREMENT, key INT, value INT)");
+            conn.createStatement().executeUpdate("CREATE TABLE Table2 (id INT AUTO_INCREMENT, key INT, value INT)");
+        }
+
+        loadCacheCnt.set(0);
+        loadCnt.set(0);
+        writeCnt.set(0);
+        deleteCnt.set(0);
+        reuseCnt.set(0);
+
+        write.set(false);
+        fail.set(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicCache() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.ATOMIC);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            cache.loadCache(null);
+            cache.get(1);
+            cache.put(1, 1);
+            cache.remove(1);
+        }
+
+        assertEquals(3, loadCacheCnt.get());
+        assertEquals(1, loadCnt.get());
+        assertEquals(1, writeCnt.get());
+        assertEquals(1, deleteCnt.get());
+        assertEquals(0, reuseCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTransactionalCache() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.TRANSACTIONAL);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            cache.loadCache(null);
+            cache.get(1);
+            cache.put(1, 1);
+            cache.remove(1);
+        }
+
+        assertEquals(3, loadCacheCnt.get());
+        assertEquals(1, loadCnt.get());
+        assertEquals(1, writeCnt.get());
+        assertEquals(1, deleteCnt.get());
+        assertEquals(0, reuseCnt.get());
+
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransaction() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg = cacheConfiguration(null, CacheAtomicityMode.TRANSACTIONAL);
+
+        try (IgniteCache<Integer, Integer> cache = ignite(0).createCache(cfg)) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache.put(1, 1);
+                cache.put(2, 2);
+                cache.remove(3);
+                cache.remove(4);
+
+                tx.commit();
+            }
+        }
+
+        assertEquals(2, writeCnt.get());
+        assertEquals(2, deleteCnt.get());
+        assertEquals(3, reuseCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCrossCacheTransaction() throws Exception {
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+                cache1.remove(3);
+                cache2.remove(4);
+
+                tx.commit();
+            }
+        }
+
+        assertEquals(2, writeCnt.get());
+        assertEquals(2, deleteCnt.get());
+        assertEquals(3, reuseCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCommit() throws Exception {
+        write.set(true);
+
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+
+                tx.commit();
+            }
+        }
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            checkTable(conn, 1, false);
+            checkTable(conn, 2, false);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRollback() throws Exception {
+        write.set(true);
+        fail.set(true);
+
+        CacheConfiguration<Integer, Integer> cfg1 = cacheConfiguration("cache1", CacheAtomicityMode.TRANSACTIONAL);
+        CacheConfiguration<Integer, Integer> cfg2 = cacheConfiguration("cache2", CacheAtomicityMode.TRANSACTIONAL);
+
+        try (
+            IgniteCache<Integer, Integer> cache1 = ignite(0).createCache(cfg1);
+            IgniteCache<Integer, Integer> cache2 = ignite(0).createCache(cfg2)
+        ) {
+            try (Transaction tx = ignite(0).transactions().txStart()) {
+                cache1.put(1, 1);
+                cache2.put(2, 2);
+
+                tx.commit();
+
+                assert false : "Exception was not thrown.";
+            }
+            catch (IgniteException e) {
+                CacheWriterException we = X.cause(e, CacheWriterException.class);
+
+                assertNotNull(we);
+
+                assertEquals("Expected failure.", we.getMessage());
+            }
+        }
+
+        try (Connection conn = DriverManager.getConnection(URL)) {
+            checkTable(conn, 1, true);
+            checkTable(conn, 2, true);
+        }
+    }
+
+    /**
+     * @param conn Connection.
+     * @param idx Table index.
+     * @param empty If table expected to be empty.
+     * @throws Exception In case of error.
+     */
+    private void checkTable(Connection conn, int idx, boolean empty) throws Exception {
+        ResultSet rs = conn.createStatement().executeQuery("SELECT key, value FROM Table" + idx);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            int key = rs.getInt(1);
+            int val = rs.getInt(2);
+
+            assertEquals(idx, key);
+            assertEquals(idx, val);
+
+            cnt++;
+        }
+
+        assertEquals(empty ? 0 : 1, cnt);
+    }
+
+    /**
+     * @param name Cache name.
+     * @param atomicity Atomicity mode.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name, CacheAtomicityMode atomicity) {
+        CacheConfiguration<Integer, Integer> cfg = new CacheConfiguration<>();
+
+        cfg.setName(name);
+        cfg.setAtomicityMode(atomicity);
+        cfg.setCacheStoreFactory(storeFactory());
+        cfg.setCacheStoreSessionListenerFactories(sessionListenerFactory());
+        cfg.setReadThrough(true);
+        cfg.setWriteThrough(true);
+        cfg.setLoadPreviousValue(true);
+
+        return cfg;
+    }
+
+    /**
+     * @return Store factory.
+     */
+    protected abstract Factory<? extends CacheStore<Integer, Integer>> storeFactory();
+
+    /**
+     * @return Session listener factory.
+     */
+    protected abstract Factory<CacheStoreSessionListener> sessionListenerFactory();
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
new file mode 100644
index 0000000..814c8a5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/CacheStoreSessionListenerLifecycleSelfTest.java
@@ -0,0 +1,395 @@
+/*
+ * 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.cache.store;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ * Store session listeners test.
+ */
+public class CacheStoreSessionListenerLifecycleSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final Queue<String> evts = new ConcurrentLinkedDeque<>();
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheStoreSessionListenerFactories(
+            new SessionListenerFactory("Shared 1"),
+            new SessionListenerFactory("Shared 2")
+        );
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        evts.clear();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoCaches() throws Exception {
+        try {
+            startGrid();
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList("Shared 1 START", "Shared 2 START", "Shared 1 STOP", "Shared 2 STOP"),
+            evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                CacheConfiguration<Integer, Integer> cacheCfg = cacheConfiguration("cache-" + i);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+
+            // Put to cache-0.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "Shared 1 SESSION START cache-0",
+            "Shared 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-0",
+            "Shared 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPartialOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+                if (i == 0) {
+                    cacheCfg.setCacheStoreSessionListenerFactories(
+                        new SessionListenerFactory(name + " 1"),
+                        new SessionListenerFactory(name + " 2")
+                    );
+                }
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "Shared 1 SESSION START cache-1",
+            "Shared 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "Shared 1 SESSION END cache-1",
+            "Shared 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOverride() throws Exception {
+        try {
+            Ignite ignite = startGrid();
+
+            for (int i = 0; i < 2; i++) {
+                String name = "cache-" + i;
+
+                CacheConfiguration cacheCfg = cacheConfiguration(name);
+
+                cacheCfg.setCacheStoreSessionListenerFactories(new SessionListenerFactory(name + " 1"), new SessionListenerFactory(name + " 2"));
+
+                ignite.createCache(cacheCfg);
+            }
+
+            ignite.cache("cache-0").put(1, 1);
+            ignite.cache("cache-1").put(1, 1);
+
+            try (Transaction tx = ignite.transactions().txStart()) {
+                ignite.cache("cache-0").put(2, 2);
+                ignite.cache("cache-0").put(3, 3);
+                ignite.cache("cache-1").put(2, 2);
+                ignite.cache("cache-1").put(3, 3);
+
+                tx.commit();
+            }
+        }
+        finally {
+            stopGrid();
+        }
+
+        assertEqualsCollections(Arrays.asList(
+            "Shared 1 START",
+            "Shared 2 START",
+            "cache-0 1 START",
+            "cache-0 2 START",
+            "cache-1 1 START",
+            "cache-1 2 START",
+
+            // Put to cache-0.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+
+            // Put to cache-1.
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            // Transaction.
+            "cache-0 1 SESSION START cache-0",
+            "cache-0 2 SESSION START cache-0",
+            "cache-1 1 SESSION START cache-1",
+            "cache-1 2 SESSION START cache-1",
+            "cache-0 1 SESSION END cache-0",
+            "cache-0 2 SESSION END cache-0",
+            "cache-1 1 SESSION END cache-1",
+            "cache-1 2 SESSION END cache-1",
+
+            "cache-0 1 STOP",
+            "cache-0 2 STOP",
+            "cache-1 1 STOP",
+            "cache-1 2 STOP",
+            "Shared 1 STOP",
+            "Shared 2 STOP"
+        ), evts);
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     */
+    private CacheConfiguration<Integer, Integer> cacheConfiguration(String name) {
+        CacheConfiguration<Integer, Integer> cacheCfg = new CacheConfiguration<>(name);
+
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setCacheStoreFactory(FactoryBuilder.factoryOf(Store.class));
+        cacheCfg.setWriteThrough(true);
+
+        return cacheCfg;
+    }
+
+    /**
+     */
+    private static class SessionListener implements CacheStoreSessionListener, LifecycleAware {
+        /** */
+        private final String name;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListener(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void start() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " START");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void stop() throws IgniteException {
+            assertNotNull(ignite);
+
+            evts.add(name + " STOP");
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionStart(CacheStoreSession ses) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION START " + ses.cacheName());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+            assertNotNull(ignite);
+
+            evts.add(name + " SESSION END " + ses.cacheName());
+        }
+    }
+
+    /**
+     */
+    private static class SessionListenerFactory implements Factory<CacheStoreSessionListener> {
+        /** */
+        private String name;
+
+        /**
+         * @param name Name.
+         */
+        private SessionListenerFactory(String name) {
+            this.name = name;
+        }
+
+        @Override public CacheStoreSessionListener create() {
+            return new SessionListener(name);
+        }
+    }
+
+    /**
+     */
+    public static class Store extends CacheStoreAdapter<Integer, Integer> {
+        public Store() {
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
new file mode 100644
index 0000000..64af249
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/CacheJdbcStoreSessionListenerSelfTest.java
@@ -0,0 +1,175 @@
+/*
+ * 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.cache.store.jdbc;
+
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.h2.jdbcx.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.integration.*;
+import java.sql.*;
+import java.util.*;
+
+/**
+ * Tests for {@link CacheJdbcStoreSessionListener}.
+ */
+public class CacheJdbcStoreSessionListenerSelfTest extends CacheStoreSessionListenerAbstractSelfTest {
+    /** {@inheritDoc} */
+    @Override protected Factory<? extends CacheStore<Integer, Integer>> storeFactory() {
+        return new Factory<CacheStore<Integer, Integer>>() {
+            @Override public CacheStore<Integer, Integer> create() {
+                return new Store();
+            }
+        };
+    }
+
+    /** {@inheritDoc} */
+    @Override protected Factory<CacheStoreSessionListener> sessionListenerFactory() {
+        return new Factory<CacheStoreSessionListener>() {
+            @Override public CacheStoreSessionListener create() {
+                CacheJdbcStoreSessionListener lsnr = new CacheJdbcStoreSessionListener();
+
+                lsnr.setDataSource(JdbcConnectionPool.create(URL, "", ""));
+
+                return lsnr;
+            }
+        };
+    }
+
+    /**
+     */
+    private static class Store extends CacheStoreAdapter<Integer, Integer> {
+        /** */
+        private static String SES_CONN_KEY = "ses_conn";
+
+        /** */
+        @CacheStoreSessionResource
+        private CacheStoreSession ses;
+
+        /** {@inheritDoc} */
+        @Override public void loadCache(IgniteBiInClosure<Integer, Integer> clo, Object... args) {
+            loadCacheCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            loadCnt.incrementAndGet();
+
+            checkConnection();
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void write(Cache.Entry<? extends Integer, ? extends Integer> entry)
+            throws CacheWriterException {
+            writeCnt.incrementAndGet();
+
+            checkConnection();
+
+            if (write.get()) {
+                Connection conn = ses.attachment();
+
+                try {
+                    String table;
+
+                    switch (ses.cacheName()) {
+                        case "cache1":
+                            table = "Table1";
+
+                            break;
+
+                        case "cache2":
+                            if (fail.get())
+                                throw new CacheWriterException("Expected failure.");
+
+                            table = "Table2";
+
+                            break;
+
+                        default:
+                            throw new CacheWriterException("Wring cache: " + ses.cacheName());
+                    }
+
+                    PreparedStatement stmt = conn.prepareStatement(
+                        "INSERT INTO " + table + " (key, value) VALUES (?, ?)");
+
+                    stmt.setInt(1, entry.getKey());
+                    stmt.setInt(2, entry.getValue());
+
+                    stmt.executeUpdate();
+                }
+                catch (SQLException e) {
+                    throw new CacheWriterException(e);
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void delete(Object key) throws CacheWriterException {
+            deleteCnt.incrementAndGet();
+
+            checkConnection();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sessionEnd(boolean commit) {
+            assertNull(ses.attachment());
+        }
+
+        /**
+         */
+        private void checkConnection() {
+            Connection conn = ses.attachment();
+
+            assertNotNull(conn);
+
+            try {
+                assertFalse(conn.isClosed());
+                assertFalse(conn.getAutoCommit());
+            }
+            catch (SQLException e) {
+                throw new RuntimeException(e);
+            }
+
+            verifySameInstance(conn);
+        }
+
+        /**
+         * @param conn Connection.
+         */
+        private void verifySameInstance(Connection conn) {
+            Map<String, Connection> props = ses.properties();
+
+            Connection sesConn = props.get(SES_CONN_KEY);
+
+            if (sesConn == null)
+                props.put(SES_CONN_KEY, conn);
+            else {
+                assertSame(conn, sesConn);
+
+                reuseCnt.incrementAndGet();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
index 6735021..7bc8dd6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
@@ -47,6 +47,7 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest {
 
         disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
         disco.setIpFinder(IP_FINDER);
+        disco.setForceServerMode(true);
 
         cfg.setDiscoverySpi(disco);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
index 59a4f5d..d90bafe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
@@ -33,7 +33,7 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.events.EventType.*;
 
 /**
@@ -163,8 +163,6 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest {
                 private AtomicInteger cnt = new AtomicInteger();
 
                 @Override public boolean apply(Event evt) {
-                    assert evt.type() == EVT_NODE_LEFT;
-
                     evts.put(cnt.getAndIncrement(), ((DiscoveryEvent) evt).topologyNodes());
 
                     latch.countDown();
@@ -228,7 +226,8 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest {
                 private AtomicInteger cnt = new AtomicInteger();
 
                 @Override public boolean apply(Event evt) {
-                    assert evt.type() == EVT_NODE_JOINED || evt.type() == EVT_NODE_LEFT;
+                    assert evt.type() == EVT_NODE_JOINED
+                        || evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
 
                     evts.put(cnt.getAndIncrement(), ((DiscoveryEvent) evt).topologyNodes());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 46512fe..a116d54 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -33,6 +33,9 @@ import org.apache.ignite.testframework.junits.common.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
 /**
  * Fail fast test.
  */
@@ -76,7 +79,7 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
                     return true;
                 }
-            }, EventType.EVT_NODE_FAILED);
+            }, EVT_NODE_FAILED);
         }
 
         Ignite ignite1 = ignite(0);
@@ -98,7 +101,7 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         failNode(ignite1);
 
-        assert failLatch.await(500, TimeUnit.MILLISECONDS);
+        assert failLatch.await(1000, MILLISECONDS);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
index 0171290..cb1341c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
@@ -20,9 +20,13 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.util.typedef.*;
 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.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -38,6 +42,9 @@ import static org.apache.ignite.events.EventType.*;
  */
 @SuppressWarnings("deprecation")
 public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest implements Externalizable {
+    /** VM ip finder for TCP discovery. */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /** Waiting timeout. */
     private static final int WAIT_TIMEOUT = 30000;
 
@@ -87,6 +94,15 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder));
+
+        return cfg;
+    }
+
     /**
      * @param startGrid Start grid flag.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
index 0a961d4..04cd3f0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionForCachesSelfTest.java
@@ -52,9 +52,14 @@ public class GridProjectionForCachesSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(discoverySpi());
 
-        cfg.setCacheConfiguration(
-            cacheConfiguration(null, new AttributeFilter(getTestGridName(0)), false),
-            cacheConfiguration(CACHE_NAME, new AttributeFilter(getTestGridName(2), getTestGridName(3)), true));
+        List<CacheConfiguration> ccfgs = new ArrayList<>();
+
+        if (gridName.equals(getTestGridName(0)))
+            ccfgs.add(cacheConfiguration(null, new AttributeFilter(getTestGridName(0)), false));
+        else if (gridName.equals(getTestGridName(2)) || gridName.equals(getTestGridName(3)))
+            ccfgs.add(cacheConfiguration(CACHE_NAME, new AttributeFilter(getTestGridName(2), getTestGridName(3)), true));
+
+        cfg.setCacheConfiguration(ccfgs.toArray(new CacheConfiguration[ccfgs.size()]));
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index 284aa0c..96d5f1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -27,7 +27,6 @@ import org.apache.ignite.testframework.junits.common.*;
 
 import java.io.*;
 import java.util.*;
-import java.util.concurrent.atomic.*;
 
 /**
  * Test grids starting with non compatible release types.
@@ -36,49 +35,56 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
-    /** Counter. */
-    private static final AtomicInteger cnt = new AtomicInteger();
-
     /** */
-    private String firstNodeVer;
+    private String nodeVer;
 
     /** */
-    private String secondNodeVer;
+    private boolean clientMode;
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        final int idx = cnt.getAndIncrement();
+        if (clientMode)
+            cfg.setClientMode(true);
 
-        // Override node attributes in discovery spi.
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi() {
-            @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
                 super.setNodeAttributes(attrs, ver);
 
-                if (idx % 2 == 0)
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, firstNodeVer);
-                else
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, secondNodeVer);
+                attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
             }
         };
 
-        discoSpi.setIpFinder(IP_FINDER);
+        discoSpi.setIpFinder(IP_FINDER).setForceServerMode(true);
 
         cfg.setDiscoverySpi(discoSpi);
 
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        clientMode = false;
+
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testOsEditionDoesNotSupportRollingUpdates() throws Exception {
-        firstNodeVer = "1.0.0";
-        secondNodeVer = "1.0.1";
+        nodeVer = "1.0.0";
+
+        startGrid(0);
 
         try {
-            startGrids(2);
+            nodeVer = "1.0.1";
+
+            startGrid(1);
+
+            fail("Exception has not been thrown.");
         }
         catch (IgniteCheckedException e) {
             StringWriter errors = new StringWriter();
@@ -87,17 +93,36 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
 
             String stackTrace = errors.toString();
 
-            assertTrue(
-                "Caught exception does not contain specified string.",
-                stackTrace.contains("Local node and remote node have different version numbers")
-            );
-
-            return;
+            if (!stackTrace.contains("Local node and remote node have different version numbers"))
+                throw e;
         }
-        finally {
-            stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOsEditionDoesNotSupportRollingUpdatesClientMode() throws Exception {
+        nodeVer = "1.0.0";
+
+        startGrid(0);
+
+        try {
+            nodeVer = "1.0.1";
+            clientMode = true;
+
+            startGrid(1);
+
+            fail("Exception has not been thrown.");
         }
+        catch (IgniteCheckedException e) {
+            StringWriter errors = new StringWriter();
+
+            e.printStackTrace(new PrintWriter(errors));
 
-        fail("Exception has not been thrown.");
+            String stackTrace = errors.toString();
+
+            if (!stackTrace.contains("Local node and remote node have different version numbers"))
+                throw e;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
index b4dce6c..7f5ee54 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridSelfTest.java
@@ -118,8 +118,8 @@ public class GridSelfTest extends GridProjectionAbstractTest {
 
         g.message().remoteListen(null, new MessagingListenActor<String>() {
             @Override protected void receive(UUID nodeId, String rcvMsg) throws Throwable {
-                assert locNodeId.equals(nodeId);
-                assert msg.equals(rcvMsg);
+                assertEquals(locNodeId, nodeId);
+                assertEquals(msg, rcvMsg);
 
                 stop(rcvMsg);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index b36c6f1..f2afb07 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -52,7 +52,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     private static final int TMP_NODES_CNT = 3;
 
     /** */
-    private static final int ITERATIONS = 20;
+    private static final int ITERATIONS = 10;
 
     /** */
     private int gridCntr;
@@ -64,6 +64,9 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     private volatile CountDownLatch latch;
 
     /** */
+    private boolean clientMode;
+
+    /** */
     private final IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
         @Override public boolean apply(Event evt) {
             assertNotNull("Topology lost nodes before stopTempNodes() was called.", latch);
@@ -88,7 +91,15 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
 
         TcpDiscoverySpi disc = new TcpDiscoverySpi();
 
+        if (clientMode && ((gridName.charAt(gridName.length() - 1) - '0') & 1) != 0)
+            cfg.setClientMode(true);
+        else
+            disc.setMaxMissedClientHeartbeats(50);
+
+        disc.setHeartbeatFrequency(500);
         disc.setIpFinder(IP_FINDER);
+        disc.setAckTimeout(1000);
+        disc.setSocketTimeout(1000);
 
         cfg.setCacheConfiguration(cCfg);
         cfg.setDiscoverySpi(disc);
@@ -118,7 +129,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     /**
      * @throws Exception If failed.
      */
-    public void testAlives() throws Exception {
+    private void doTestAlive() throws Exception {
         for (int i = 0; i < ITERATIONS; i++) {
             info("Performing iteration: " + i);
 
@@ -141,6 +152,24 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testAlives() throws Exception {
+        clientMode = false;
+
+        doTestAlive();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAlivesClient() throws Exception {
+        clientMode = true;
+
+        doTestAlive();
+    }
+
+    /**
      * Waits while topology on all nodes became equals to the expected size.
      *
      * @param nodesCnt Expected nodes count.
@@ -149,6 +178,8 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     @SuppressWarnings("BusyWait")
     private void awaitDiscovery(long nodesCnt) throws InterruptedException {
         for (Ignite g : alive) {
+            ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
+
             while (g.cluster().nodes().size() != nodesCnt)
                 Thread.sleep(10);
         }
@@ -187,7 +218,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
                     });
 
                 assertTrue(
-                    currTop.contains(GridCacheUtils.oldest(k.internalCache().context(), new AffinityTopologyVersion(currVer))));
+                    currTop.contains(GridCacheUtils.oldestAliveCacheServerNode(k.context().cache().context(), new AffinityTopologyVersion(currVer))));
             }
         }
     }
@@ -213,23 +244,28 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
      * Stops temporary nodes.
      */
     private void stopTempNodes() {
-        int rmv = 0;
+        Collection<Ignite> toRmv = new ArrayList<>(alive.subList(0, TMP_NODES_CNT));
 
-        Collection<Ignite> toRmv = new ArrayList<>(TMP_NODES_CNT);
+        alive.removeAll(toRmv);
+
+        // Remove listeners to avoid receiving events from stopping nodes.
+        for (Ignite g : toRmv)
+            g.events().stopLocalListen(lsnr, EventType.EVT_NODE_LEFT, EventType.EVT_NODE_FAILED);
 
-        for (Iterator<Ignite> iter = alive.iterator(); iter.hasNext() && rmv < TMP_NODES_CNT;) {
-            toRmv.add(iter.next());
+        for (Iterator<Ignite> itr = toRmv.iterator(); itr.hasNext(); ) {
+            Ignite g = itr.next();
 
-            iter.remove();
+            if (g.cluster().localNode().isClient()) {
+                G.stop(g.name(), false);
 
-            rmv++;
+                itr.remove();
+            }
         }
 
-        // Remove listeners to avoid receiving events from stopping nodes.
-        for (Ignite g : toRmv)
-            g.events().stopLocalListen(lsnr, EventType.EVT_NODE_LEFT, EventType.EVT_NODE_FAILED);
+        for (Ignite g : toRmv) {
+            assert !g.cluster().localNode().isClient();
 
-        for (Ignite g : toRmv)
             G.stop(g.name(), false);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index 538ea39..e76c615 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -29,7 +29,7 @@ import static org.apache.ignite.configuration.DeploymentMode.*;
 /**
  * Tests for node attributes consistency checks.
  */
-public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTest {
+public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTest {
     /** */
     private static final String PREFER_IPV4 = "java.net.preferIPv4Stack";
 
@@ -37,7 +37,7 @@ public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTe
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** */
-    private static DeploymentMode mode = SHARED;
+    private static DeploymentMode mode;
 
     /** */
     private static boolean p2pEnabled;
@@ -46,18 +46,34 @@ public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTe
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disc = new TcpDiscoverySpi();
-
-        disc.setIpFinder(IP_FINDER);
+        if (gridName.equals(getTestGridName(1)))
+            cfg.setClientMode(true);
 
         cfg.setIncludeProperties(PREFER_IPV4);
         cfg.setDeploymentMode(mode);
         cfg.setPeerClassLoadingEnabled(p2pEnabled);
-        cfg.setDiscoverySpi(disc);
+
+        TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi();
+
+        discoverySpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoverySpi);
 
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        mode = SHARED;
+
+        p2pEnabled = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -83,44 +99,35 @@ public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTe
      * @throws Exception If failed.
      */
     public void testPreferIpV4StackDifferentValues() throws Exception {
-        try {
-            System.setProperty(PREFER_IPV4, "true");
+        System.setProperty(PREFER_IPV4, "true");
 
-            for (int i = 0; i < 2; i++) {
-                Ignite g = startGrid(i);
+        for (int i = 0; i < 2; i++) {
+            Ignite g = startGrid(i);
 
-                assert "true".equals(g.cluster().localNode().attribute(PREFER_IPV4));
-            }
+            assert "true".equals(g.cluster().localNode().attribute(PREFER_IPV4));
+        }
 
-            System.setProperty(PREFER_IPV4, "false");
+        System.setProperty(PREFER_IPV4, "false");
 
-            startGrid(2);
-        }
-        finally {
-            stopAllGrids();
-        }
+        startGrid(2);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testDifferentDeploymentModes() throws Exception {
-        try {
-            startGrid(1);
+        startGrid(0);
 
-            mode = CONTINUOUS;
+        mode = CONTINUOUS;
 
-            try {
-                startGrid(2);
+        try {
+            startGrid(1);
 
-                fail();
-            }
-            catch (IgniteCheckedException e) {
-                assertTrue(e.getCause().getMessage().startsWith("Remote node has deployment mode different from"));
-            }
+            fail();
         }
-        finally {
-            stopAllGrids();
+        catch (IgniteCheckedException e) {
+            if (!e.getCause().getMessage().startsWith("Remote node has deployment mode different from"))
+                throw e;
         }
     }
 
@@ -128,23 +135,18 @@ public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTe
      * @throws Exception If failed.
      */
     public void testDifferentPeerClassLoadingEnabledFlag() throws Exception {
-        try {
-            startGrid(1);
+        startGrid(0);
 
-            p2pEnabled = true;
+        p2pEnabled = true;
 
-            try {
-                startGrid(2);
+        try {
+            startGrid(1);
 
-                fail();
-            }
-            catch (IgniteCheckedException e) {
-                assertTrue(e.getCause().getMessage().startsWith("Remote node has peer class loading enabled flag " +
-                    "different from"));
-            }
+            fail();
         }
-        finally {
-            stopAllGrids();
+        catch (IgniteCheckedException e) {
+            if (!e.getCause().getMessage().startsWith("Remote node has peer class loading enabled flag different from"))
+                throw e;
         }
     }
 
@@ -153,19 +155,35 @@ public class GridDiscoveryManagerAttributesSelfTest extends GridCommonAbstractTe
      * @throws Exception If failed.
      */
     private void testPreferIpV4Stack(boolean preferIpV4) throws Exception {
-        try {
-            String val = String.valueOf(preferIpV4);
+        String val = String.valueOf(preferIpV4);
 
-            System.setProperty(PREFER_IPV4, val);
+        System.setProperty(PREFER_IPV4, val);
 
-            for (int i = 0; i < 2; i++) {
-                Ignite g = startGrid(i);
+        for (int i = 0; i < 2; i++) {
+            Ignite g = startGrid(i);
 
-                assert val.equals(g.cluster().localNode().attribute(PREFER_IPV4));
-            }
+            assert val.equals(g.cluster().localNode().attribute(PREFER_IPV4));
         }
-        finally {
-            stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    public static class RegularDiscovery extends GridDiscoveryManagerAttributesSelfTest {
+        /** {@inheritDoc} */
+        @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+            IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+            return cfg;
         }
     }
+
+    /**
+     *
+     */
+    public static class ClientDiscovery extends GridDiscoveryManagerAttributesSelfTest {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index 6f7c935..65aec49 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -33,7 +33,7 @@ import static org.apache.ignite.cache.CacheMode.*;
 /**
  *
  */
-public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
+public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
     /** */
     private static final String CACHE_NAME = "cache";
 
@@ -50,12 +50,6 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disc = new TcpDiscoverySpi();
-
-        disc.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(disc);
-
         CacheConfiguration ccfg1 = defaultCacheConfiguration();
 
         ccfg1.setName(CACHE_NAME);
@@ -66,14 +60,21 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
 
         if (gridName.equals(getTestGridName(1)))
             cfg.setClientMode(true);
+        else {
+            ccfg1.setNearConfiguration(null);
+            ccfg2.setNearConfiguration(null);
 
-        ccfg1.setNearConfiguration(null);
-        ccfg2.setNearConfiguration(null);
+            ccfg1.setCacheMode(PARTITIONED);
+            ccfg2.setCacheMode(PARTITIONED);
+
+            cfg.setCacheConfiguration(ccfg1, ccfg2);
+        }
 
-        ccfg1.setCacheMode(PARTITIONED);
-        ccfg2.setCacheMode(PARTITIONED);
+        TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi();
 
-        cfg.setCacheConfiguration(ccfg1, ccfg2);
+        discoverySpi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(discoverySpi);
 
         return cfg;
     }
@@ -186,4 +187,25 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
         assertTrue(g0.context().discovery().hasNearCache(null, four));
         assertFalse(g0.context().discovery().hasNearCache(null, five));
     }
+
+    /**
+     *
+     */
+    public static class RegularDiscovery extends GridDiscoveryManagerSelfTest {
+        /** {@inheritDoc} */
+        @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+            IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+            return cfg;
+        }
+    }
+
+    /**
+     *
+     */
+    public static class ClientDiscovery extends GridDiscoveryManagerSelfTest {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
index e7fab8e..7da3728 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
@@ -54,6 +54,7 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
new file mode 100644
index 0000000..f2c38e1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheReadOnlyTransactionalClientSelfTest.java
@@ -0,0 +1,327 @@
+/*
+ * 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.cache.store.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import javax.cache.configuration.*;
+import javax.cache.processor.*;
+
+import static org.apache.ignite.IgniteSystemProperties.*;
+
+/**
+ * Tests for read-only transactional cache client.
+ */
+public class CacheReadOnlyTransactionalClientSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_NAME = "test-cache";
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean nearEnabled;
+
+    /** */
+    private Factory<CacheStore> factory;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration cc = new CacheConfiguration();
+
+        cc.setName(CACHE_NAME);
+        cc.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
+        cc.setCacheStoreFactory(factory);
+
+        if (client && nearEnabled)
+            cc.setNearConfiguration(new NearCacheConfiguration());
+
+        cfg.setCacheConfiguration(cc);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        client = false;
+        factory = new Factory1();
+
+        startGrids(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopGrid();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCorrectStore() throws Exception {
+        client = true;
+        nearEnabled = false;
+        factory = new Factory1();
+
+        Ignite ignite = startGrid();
+
+        IgniteCache cache = ignite.cache(CACHE_NAME);
+
+        cache.get(0);
+        cache.getAll(F.asSet(0, 1));
+        cache.getAndPut(0, 0);
+        cache.getAndPutIfAbsent(0, 0);
+        cache.getAndRemove(0);
+        cache.getAndReplace(0, 0);
+        cache.put(0, 0);
+        cache.putAll(F.asMap(0, 0, 1, 1));
+        cache.putIfAbsent(0, 0);
+        cache.remove(0);
+        cache.remove(0, 0);
+        cache.removeAll(F.asSet(0, 1));
+        cache.removeAll();
+        cache.invoke(0, new EP());
+        cache.invokeAll(F.asSet(0, 1), new EP());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInvalidStore() throws Exception {
+        client = true;
+        nearEnabled = false;
+        factory = new Factory2();
+
+        try {
+            startGrid();
+
+            assert false : "Exception was not thrown.";
+        }
+        catch (Exception e) {
+            assert e.getMessage().startsWith("Store factory mismatch") : e.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDisabledConsistencyCheck() throws Exception {
+        client = false;
+        nearEnabled = false;
+        factory = new Factory2();
+
+        System.setProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK, "true");
+
+        startGrid("client-1");
+
+        factory = new Factory1();
+
+        System.clearProperty(IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK);
+
+        startGrid("client-2");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoStoreNearDisabled() throws Exception {
+        nearEnabled = false;
+
+        doTestNoStore();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoStoreNearEnabled() throws Exception {
+        nearEnabled = true;
+
+        doTestNoStore();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void doTestNoStore() throws Exception {
+        client = true;
+        factory = null;
+
+        Ignite ignite = startGrid();
+
+        IgniteCache cache = ignite.cache(CACHE_NAME);
+
+        cache.get(0);
+        cache.getAll(F.asSet(0, 1));
+
+        try {
+            cache.getAndPut(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.getAndPutIfAbsent(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.getAndRemove(0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.getAndReplace(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.put(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.putAll(F.asMap(0, 0, 1, 1));
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.putIfAbsent(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.remove(0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.remove(0, 0);
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.removeAll(F.asSet(0, 1));
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.removeAll();
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.invoke(0, new EP());
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+
+        try {
+            cache.invokeAll(F.asSet(0, 1), new EP());
+        }
+        catch (CacheException e) {
+            assert e.getMessage().startsWith("Updates are not allowed for transactional cache: " + CACHE_NAME + ".") :
+                e.getMessage();
+        }
+    }
+
+    /**
+     */
+    private static class Factory1 implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return null;
+        }
+    }
+
+    /**
+     */
+    private static class Factory2 implements Factory<CacheStore> {
+        /** {@inheritDoc} */
+        @Override public CacheStore create() {
+            return null;
+        }
+    }
+
+    /**
+     */
+    private static class EP implements CacheEntryProcessor {
+        @Override public Object process(MutableEntry entry, Object... arguments) {
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
index f5de96f..1d4d2f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.*;
 public class CacheRemoveAllSelfTest extends GridCacheAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return 60000;
+        return 2 * 60 * 1000;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 5d9ad35..6b7d1b8 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,7 +74,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         discoSpi.setSocketTimeout(10_000);
         discoSpi.setAckTimeout(10_000);
-        discoSpi.setNetworkTimeout(10_000);
+        discoSpi.setNetworkTimeout(60_000);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 70d8f9c..9bfbd15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 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.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -130,6 +131,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES)
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 
@@ -3913,6 +3916,33 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception If failed.
      */
+    public void testIterator() throws Exception {
+        IgniteCache<Integer, Integer> cache = grid(0).cache(null);
+
+        final int KEYS = 1000;
+
+        for (int i = 0; i < KEYS; i++)
+            cache.put(i, i);
+
+        // Try to initialize readers in case when near cache is enabled.
+        for (int i = 0; i < gridCount(); i++) {
+            cache = grid(i).cache(null);
+
+            for (int k = 0; k < KEYS; k++)
+                assertEquals((Object)k, cache.get(k));
+        }
+
+        int cnt = 0;
+
+        for (Cache.Entry e : cache)
+            cnt++;
+
+        assertEquals(KEYS, cnt);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testIgniteCacheIterator() throws Exception {
         IgniteCache<String, Integer> cache = jcache(0);
 
@@ -4863,6 +4893,89 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTx() throws Exception {
+        checkGetOutTx(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetOutTxAsync() throws Exception {
+        checkGetOutTx(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkGetOutTx(boolean async) throws Exception {
+        final AtomicInteger lockEvtCnt = new AtomicInteger();
+
+        IgnitePredicate<Event> lsnr = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                lockEvtCnt.incrementAndGet();
+
+                return true;
+            }
+        };
+
+        try {
+            IgniteCache<String, Integer> cache = grid(0).cache(null);
+
+            List<String> keys = primaryKeysForCache(cache, 2);
+
+            assertEquals(2, keys.size());
+
+            cache.put(keys.get(0), 0);
+            cache.put(keys.get(1), 1);
+
+            grid(0).events().localListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+
+            if (async)
+                cache = cache.withAsync();
+
+            try (Transaction tx = transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Integer val0 = cache.get(keys.get(0));
+
+                if (async)
+                    val0 = cache.<Integer>future().get();
+
+                assertEquals(0, val0.intValue());
+
+                Map<String, Integer> allOutTx = cache.getAllOutTx(F.asSet(keys.get(1)));
+
+                if (async)
+                    allOutTx = cache.<Map<String, Integer>>future().get();
+
+                assertEquals(1, allOutTx.size());
+
+                assertTrue(allOutTx.containsKey(keys.get(1)));
+
+                assertEquals(1, allOutTx.get(keys.get(1)).intValue());
+            }
+
+            assertTrue(GridTestUtils.waitForCondition(new PA() {
+                @Override public boolean apply() {
+                    info("Lock event count: " + lockEvtCnt.get());
+                    if (atomicityMode() == ATOMIC)
+                        return lockEvtCnt.get() == 0;
+
+                    if (cacheMode() == PARTITIONED && nearEnabled()) {
+                        if (!grid(0).configuration().isClientMode())
+                            return lockEvtCnt.get() == 4;
+                    }
+
+                    return lockEvtCnt.get() == 2;
+                }
+            }, 15000));
+        }
+        finally {
+            grid(0).events().stopLocalListen(lsnr, EVT_CACHE_OBJECT_LOCKED, EVT_CACHE_OBJECT_UNLOCKED);
+        }
+    }
+
+    /**
      * Sets given value, returns old value.
      */
     public static final class SetValueProcessor implements EntryProcessor<String, Integer, Integer> {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
index 1821e12..bb1732e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
@@ -38,7 +38,7 @@ import static java.util.concurrent.TimeUnit.*;
  */
 public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstractSelfTest {
     /** */
-    private static final int KEY_CNT = 50;
+    private static final int KEY_CNT = 500;
 
     /** {@inheritDoc} */
     @Override protected boolean swapEnabled() {
@@ -163,24 +163,18 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         IgniteCache<Object, Object> cacheAsync = cache.withAsync();
 
-        cache.put(1, 1);
-        cache.put(2, 2);
+        for (int i = 0; i < KEY_CNT; i++)
+            cache.put(i, i);
 
         assertEquals(cache.metrics().getAverageRemoveTime(), 0.0, 0.0);
 
-        cacheAsync.getAndRemove(1);
-
-        IgniteFuture<Object> fut = cacheAsync.future();
-
-        assertEquals(1, (int)fut.get());
-
-        assert cache.metrics().getAverageRemoveTime() > 0;
-
-        cacheAsync.getAndRemove(2);
+        for (int i = 0; i < KEY_CNT; i++) {
+            cacheAsync.getAndRemove(i);
 
-        fut = cacheAsync.future();
+            IgniteFuture<Object> fut = cacheAsync.future();
 
-        assertEquals(2, (int)fut.get());
+            fut.get();
+        }
 
         assert cache.metrics().getAverageRemoveTime() > 0;
     }
@@ -221,18 +215,13 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
     public void testRemoveAvgTime() throws Exception {
         IgniteCache<Integer, Integer> cache = grid(0).cache(null);
 
-        cache.put(1, 1);
-        cache.put(2, 2);
+        for (int i = 0; i < KEY_CNT; i++)
+            cache.put(i, i);
 
         assertEquals(cache.metrics().getAverageRemoveTime(), 0.0, 0.0);
 
-        cache.remove(1);
-
-        float avgRmvTime = cache.metrics().getAverageRemoveTime();
-
-        assert avgRmvTime > 0;
-
-        cache.remove(2);
+        for (int i = 0; i < KEY_CNT; i++)
+            cache.remove(i);
 
         assert cache.metrics().getAverageRemoveTime() > 0;
     }
@@ -378,17 +367,12 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
         assertEquals(0.0, cache.metrics().getAveragePutTime(), 0.0);
         assertEquals(0, cache.metrics().getCachePuts());
 
-        cache.put(1, 1);
-
-        float avgPutTime = cache.metrics().getAveragePutTime();
-
-        assert avgPutTime >= 0;
-
-        assertEquals(1, cache.metrics().getCachePuts());
+        for (int i = 0; i < KEY_CNT; i++)
+            cache.put(i, i);
 
-        cache.put(2, 2);
+        assert cache.metrics().getAveragePutTime() > 0;
 
-        assert cache.metrics().getAveragePutTime() >= 0;
+        assertEquals(KEY_CNT, cache.metrics().getCachePuts());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index c6ede61..2b6a6b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -18,10 +18,12 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.*;
 import org.jsr166.*;
 
@@ -71,6 +73,18 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
     private String sizePropVal;
 
     /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
+        if (testClientNode() && getTestGridName(0).equals(gridName))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return GRID_CNT;
     }
@@ -106,9 +120,18 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
     }
 
     /**
+     * @return {@code True} if test updates from client node.
+     */
+    protected boolean testClientNode() {
+        return false;
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testPutAndRemove() throws Exception {
+        assertEquals(testClientNode(), (boolean)grid(0).configuration().isClientMode());
+
         final IgniteCache<Integer, Integer> sndCache0 = grid(0).cache(null);
 
         final AtomicBoolean stop = new AtomicBoolean();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 efd0185..eeb9f45 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
@@ -58,7 +58,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
     protected static final Map<Object, Object> map = new ConcurrentHashMap8<>();
 
     /** VM ip finder for TCP discovery. */
-    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /**
      * @return Grids count to start.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
index da2b81c..db4061a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
@@ -60,6 +60,7 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 8a1ae78..bba4ad9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -26,7 +26,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.*;
@@ -38,6 +37,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
+
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -110,7 +110,11 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
             CacheConfiguration cc = defaultCacheConfiguration();
 
             cc.setCacheMode(mode);
-            cc.setEvictionPolicy(new LruEvictionPolicy(1000));
+
+            LruEvictionPolicy plc = new LruEvictionPolicy();
+            plc.setMaxSize(1000);
+
+            cc.setEvictionPolicy(plc);
             cc.setEvictSynchronized(false);
             cc.setSwapEnabled(false);
             cc.setWriteSynchronizationMode(FULL_SYNC);


[02/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
index 7dca049..f23c62c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsOutProc.java
@@ -81,6 +81,9 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
     /** IGFS name. */
     private final String igfs;
 
+    /** The user this out proc is performing on behalf of. */
+    private final String userName;
+
     /** Client log. */
     private final Log log;
 
@@ -100,8 +103,8 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    public HadoopIgfsOutProc(String host, int port, String grid, String igfs, Log log) throws IOException {
-        this(host, port, grid, igfs, false, log);
+    public HadoopIgfsOutProc(String host, int port, String grid, String igfs, Log log, String user) throws IOException {
+        this(host, port, grid, igfs, false, log, user);
     }
 
     /**
@@ -113,8 +116,8 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    public HadoopIgfsOutProc(int port, String grid, String igfs, Log log) throws IOException {
-        this(null, port, grid, igfs, true, log);
+    public HadoopIgfsOutProc(int port, String grid, String igfs, Log log, String user) throws IOException {
+        this(null, port, grid, igfs, true, log, user);
     }
 
     /**
@@ -128,7 +131,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
      * @param log Client logger.
      * @throws IOException If failed.
      */
-    private HadoopIgfsOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log)
+    private HadoopIgfsOutProc(String host, int port, String grid, String igfs, boolean shmem, Log log, String user)
         throws IOException {
         assert host != null && !shmem || host == null && shmem :
             "Invalid arguments [host=" + host + ", port=" + port + ", shmem=" + shmem + ']';
@@ -138,6 +141,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         this.grid = grid;
         this.igfs = igfs;
         this.log = log;
+        this.userName = IgfsUtils.fixUserName(user);
 
         io = HadoopIgfsIpcIo.get(log, endpoint);
 
@@ -173,6 +177,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(INFO);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_RES).get();
     }
@@ -184,6 +189,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(UPDATE);
         msg.path(path);
         msg.properties(props);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_RES).get();
     }
@@ -196,6 +202,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.accessTime(accessTime);
         msg.modificationTime(modificationTime);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -207,6 +214,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(RENAME);
         msg.path(src);
         msg.destinationPath(dest);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -218,6 +226,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(DELETE);
         msg.path(path);
         msg.flag(recursive);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -231,6 +240,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.start(start);
         msg.length(len);
+        msg.userName(userName);
 
         return io.send(msg).chain(BLOCK_LOCATION_COL_RES).get();
     }
@@ -241,6 +251,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(PATH_SUMMARY);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(SUMMARY_RES).get();
     }
@@ -252,6 +263,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(MAKE_DIRECTORIES);
         msg.path(path);
         msg.properties(props);
+        msg.userName(userName);
 
         return io.send(msg).chain(BOOL_RES).get();
     }
@@ -262,6 +274,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(LIST_FILES);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(FILE_COL_RES).get();
     }
@@ -272,6 +285,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
 
         msg.command(LIST_PATHS);
         msg.path(path);
+        msg.userName(userName);
 
         return io.send(msg).chain(PATH_COL_RES).get();
     }
@@ -288,6 +302,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.command(OPEN_READ);
         msg.path(path);
         msg.flag(false);
+        msg.userName(userName);
 
         IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
 
@@ -303,6 +318,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.flag(true);
         msg.sequentialReadsBeforePrefetch(seqReadsBeforePrefetch);
+        msg.userName(userName);
 
         IgfsInputStreamDescriptor rmtDesc = io.send(msg).chain(STREAM_DESCRIPTOR_RES).get();
 
@@ -321,6 +337,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.properties(props);
         msg.replication(replication);
         msg.blockSize(blockSize);
+        msg.userName(userName);
 
         Long streamId = io.send(msg).chain(LONG_RES).get();
 
@@ -336,6 +353,7 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
         msg.path(path);
         msg.flag(create);
         msg.properties(props);
+        msg.userName(userName);
 
         Long streamId = io.send(msg).chain(LONG_RES).get();
 
@@ -471,4 +489,9 @@ public class HadoopIgfsOutProc implements HadoopIgfsEx, HadoopIgfsIpcIoListener
             }
         };
     }
+
+    /** {@inheritDoc} */
+    @Override public String user() {
+        return userName;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
index 1dada21..7d0db49 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsWrapper.java
@@ -55,6 +55,9 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
     /** Logger. */
     private final Log log;
 
+    /** The user name this wrapper works on behalf of. */
+    private final String userName;
+
     /**
      * Constructor.
      *
@@ -63,13 +66,15 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
      * @param conf Configuration.
      * @param log Current logger.
      */
-    public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log) throws IOException {
+    public HadoopIgfsWrapper(String authority, String logDir, Configuration conf, Log log, String user)
+            throws IOException {
         try {
             this.authority = authority;
             this.endpoint = new HadoopIgfsEndpoint(authority);
             this.logDir = logDir;
             this.conf = conf;
             this.log = log;
+            this.userName = user;
         }
         catch (IgniteCheckedException e) {
             throw new IOException("Failed to parse endpoint: " + authority, e);
@@ -362,13 +367,14 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopIgfsInProc(igfs, log);
+                    hadoop = new HadoopIgfsInProc(igfs, log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
                 catch (IOException | IgniteCheckedException e) {
                     if (e instanceof HadoopIgfsCommunicationException)
-                        hadoop.close(true);
+                        if (hadoop != null)
+                            hadoop.close(true);
 
                     if (log.isDebugEnabled())
                         log.debug("Failed to connect to in-proc IGFS, fallback to IPC mode.", e);
@@ -384,7 +390,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
                 HadoopIgfsEx hadoop = null;
 
                 try {
-                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                    hadoop = new HadoopIgfsOutProc(endpoint.port(), endpoint.grid(), endpoint.igfs(), log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
@@ -409,7 +415,7 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
 
                 try {
                     hadoop = new HadoopIgfsOutProc(LOCALHOST, endpoint.port(), endpoint.grid(), endpoint.igfs(),
-                        log);
+                        log, userName);
 
                     curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
                 }
@@ -430,7 +436,8 @@ public class HadoopIgfsWrapper implements HadoopIgfs {
             HadoopIgfsEx hadoop = null;
 
             try {
-                hadoop = new HadoopIgfsOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(), log);
+                hadoop = new HadoopIgfsOutProc(endpoint.host(), endpoint.port(), endpoint.grid(), endpoint.igfs(),
+                    log, userName);
 
                 curDelegate = new Delegate(hadoop, hadoop.handshake(logDir));
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
index 2e04ac1..b170125 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/HadoopRunnableTask.java
@@ -99,6 +99,22 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
 
     /** {@inheritDoc} */
     @Override public Void call() throws IgniteCheckedException {
+        ctx = job.getTaskContext(info);
+
+        return ctx.runAsJobOwner(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                call0();
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Implements actual task running.
+     * @throws IgniteCheckedException
+     */
+    void call0() throws IgniteCheckedException {
         execStartTs = U.currentTimeMillis();
 
         Throwable err = null;
@@ -108,8 +124,6 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
         HadoopPerformanceCounter perfCntr = null;
 
         try {
-            ctx = job.getTaskContext(info);
-
             perfCntr = HadoopPerformanceCounter.getCounter(ctx.counters(), nodeId);
 
             perfCntr.onTaskSubmit(info, submitTs);
@@ -156,8 +170,6 @@ public abstract class HadoopRunnableTask implements Callable<Void> {
             if (ctx != null)
                 ctx.cleanupTaskEnvironment();
         }
-
-        return null;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 d265ca8..d754039 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
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.hadoop.v2;
 
 import org.apache.hadoop.fs.*;
-import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
 import org.apache.hadoop.mapred.JobID;
@@ -68,7 +67,7 @@ public class HadoopV2Job implements HadoopJob {
         new ConcurrentHashMap8<>();
 
     /** Pooling task context class and thus class loading environment. */
-    private final Queue<Class<?>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
+    private final Queue<Class<? extends HadoopTaskContext>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
 
     /** All created contexts. */
     private final Queue<Class<?>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
@@ -93,12 +92,7 @@ public class HadoopV2Job implements HadoopJob {
 
         hadoopJobID = new JobID(jobId.globalId().toString(), jobId.localId());
 
-        HadoopClassLoader clsLdr = (HadoopClassLoader)getClass().getClassLoader();
-
-        // Before create JobConf instance we should set new context class loader.
-        Thread.currentThread().setContextClassLoader(clsLdr);
-
-        jobConf = new JobConf();
+        jobConf = HadoopUtils.safeCreateJobConf();
 
         HadoopFileSystemsUtils.setupFileSystems(jobConf);
 
@@ -139,7 +133,9 @@ public class HadoopV2Job implements HadoopJob {
 
             Path jobDir = new Path(jobDirPath);
 
-            try (FileSystem fs = FileSystem.get(jobDir.toUri(), jobConf)) {
+            try {
+                FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf, true);
+
                 JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
                     jobDir);
 
@@ -197,7 +193,7 @@ public class HadoopV2Job implements HadoopJob {
         if (old != null)
             return old.get();
 
-        Class<?> cls = taskCtxClsPool.poll();
+        Class<? extends HadoopTaskContext> cls = taskCtxClsPool.poll();
 
         try {
             if (cls == null) {
@@ -205,9 +201,9 @@ 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-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
+                    "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
 
-                cls = ldr.loadClass(HadoopV2TaskContext.class.getName());
+                cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 
                 fullCtxClsQueue.add(cls);
             }
@@ -325,7 +321,14 @@ public class HadoopV2Job implements HadoopJob {
 
     /** {@inheritDoc} */
     @Override public void cleanupStagingDirectory() {
-        if (rsrcMgr != null)
-            rsrcMgr.cleanupStagingDirectory();
+        rsrcMgr.cleanupStagingDirectory();
+    }
+
+    /**
+     * Getter for job configuration.
+     * @return The job configuration.
+     */
+    public JobConf jobConf() {
+        return jobConf;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 6f6bfa1..2f64e77 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
@@ -40,6 +40,9 @@ import java.util.*;
  * files are needed to be placed on local files system.
  */
 public class HadoopV2JobResourceManager {
+    /** File type Fs disable caching property name. */
+    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME = HadoopUtils.disableFsCachePropertyName("file");
+
     /** Hadoop job context. */
     private final JobContextImpl ctx;
 
@@ -84,7 +87,7 @@ public class HadoopV2JobResourceManager {
         try {
             cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP, dir.getAbsolutePath());
 
-            if(!cfg.getBoolean("fs.file.impl.disable.cache", false))
+            if (!cfg.getBoolean(FILE_DISABLE_CACHING_PROPERTY_NAME, false))
                 FileSystem.getLocal(cfg).setWorkingDirectory(new Path(dir.getAbsolutePath()));
         }
         finally {
@@ -112,15 +115,17 @@ public class HadoopV2JobResourceManager {
                 stagingDir = new Path(new URI(mrDir));
 
                 if (download) {
-                    FileSystem fs = FileSystem.get(stagingDir.toUri(), cfg);
+                    FileSystem fs = HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), cfg, true);
 
                     if (!fs.exists(stagingDir))
-                        throw new IgniteCheckedException("Failed to find map-reduce submission directory (does not exist): " +
-                            stagingDir);
+                        throw new IgniteCheckedException("Failed to find map-reduce submission " +
+                            "directory (does not exist): " + stagingDir);
 
                     if (!FileUtil.copy(fs, stagingDir, jobLocDir, false, cfg))
-                        throw new IgniteCheckedException("Failed to copy job submission directory contents to local file system " +
-                            "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath() + ", jobId=" + jobId + ']');
+                        throw new IgniteCheckedException("Failed to copy job submission directory "
+                            + "contents to local file system "
+                            + "[path=" + stagingDir + ", locDir=" + jobLocDir.getAbsolutePath()
+                            + ", jobId=" + jobId + ']');
                 }
 
                 File jarJobFile = new File(jobLocDir, "job.jar");
@@ -144,7 +149,8 @@ public class HadoopV2JobResourceManager {
                 }
             }
             else if (!jobLocDir.mkdirs())
-                throw new IgniteCheckedException("Failed to create local job directory: " + jobLocDir.getAbsolutePath());
+                throw new IgniteCheckedException("Failed to create local job directory: "
+                    + jobLocDir.getAbsolutePath());
 
             setLocalFSWorkingDirectory(jobLocDir);
         }
@@ -204,14 +210,14 @@ public class HadoopV2JobResourceManager {
 
             FileSystem dstFs = FileSystem.getLocal(cfg);
 
-            FileSystem srcFs = srcPath.getFileSystem(cfg);
+            FileSystem srcFs = HadoopUtils.fileSystemForMrUser(srcPath.toUri(), cfg, true);
 
             if (extract) {
                 File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
 
                 if (!archivesPath.exists() && !archivesPath.mkdir())
                     throw new IOException("Failed to create directory " +
-                         "[path=" + archivesPath + ", jobId=" + jobId + ']');
+                        "[path=" + archivesPath + ", jobId=" + jobId + ']');
 
                 File archiveFile = new File(archivesPath, locName);
 
@@ -287,7 +293,7 @@ public class HadoopV2JobResourceManager {
     public void cleanupStagingDirectory() {
         try {
             if (stagingDir != null)
-                stagingDir.getFileSystem(ctx.getJobConf()).delete(stagingDir, true);
+                HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), ctx.getJobConf(), true).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/1652fd18/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 e9c859bd..e89feba 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
@@ -28,17 +28,21 @@ import org.apache.hadoop.mapred.TaskAttemptID;
 import org.apache.hadoop.mapred.TaskID;
 import org.apache.hadoop.mapreduce.JobContext;
 import org.apache.hadoop.mapreduce.*;
+import org.apache.hadoop.security.*;
 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.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
+import java.security.*;
 import java.util.*;
+import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
@@ -239,9 +243,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
         Thread.currentThread().setContextClassLoader(jobConf().getClassLoader());
 
         try {
-            FileSystem fs = FileSystem.get(jobConf());
-
-            HadoopFileSystemsUtils.setUser(fs, jobConf().getUser());
+            FileSystem.get(jobConf());
 
             LocalFileSystem locFs = FileSystem.getLocal(jobConf());
 
@@ -421,7 +423,7 @@ 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 = FileSystem.get(jobDir.toUri(), jobConf());
+        try (FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf(), false);
             FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
 
             in.seek(split.offset());
@@ -450,4 +452,44 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
             throw new IgniteCheckedException(e);
         }
     }
+
+    /** {@inheritDoc} */
+    @Override public <T> T runAsJobOwner(final Callable<T> c) throws IgniteCheckedException {
+        String user = job.info().user();
+
+        user = IgfsUtils.fixUserName(user);
+
+        assert user != null;
+
+        String ugiUser;
+
+        try {
+            UserGroupInformation currUser = UserGroupInformation.getCurrentUser();
+
+            assert currUser != null;
+
+            ugiUser = currUser.getShortUserName();
+        }
+        catch (IOException ioe) {
+            throw new IgniteCheckedException(ioe);
+        }
+
+        try {
+            if (F.eq(user, ugiUser))
+                // if current UGI context user is the same, do direct call:
+                return c.call();
+            else {
+                UserGroupInformation ugi = UserGroupInformation.getBestUGI(null, user);
+
+                return ugi.doAs(new PrivilegedExceptionAction<T>() {
+                    @Override public T run() throws Exception {
+                        return c.call();
+                    }
+                });
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
index b94d9d1..b9f8179 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/client/hadoop/HadoopClientProtocolSelfTest.java
@@ -28,7 +28,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.hadoop.mapreduce.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.hadoop.proto.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -449,7 +448,7 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
      * @return Configuration.
      */
     private Configuration config(int port) {
-        Configuration conf = new Configuration();
+        Configuration conf = HadoopUtils.safeCreateConfiguration();
 
         setupFileSystems(conf);
 
@@ -521,9 +520,8 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
             ctx.getCounter(TestCounter.COUNTER2).increment(1);
 
             int sum = 0;
-            for (IntWritable value : values) {
+            for (IntWritable value : values)
                 sum += value.get();
-            }
 
             ctx.write(key, new IntWritable(sum));
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index d11cabb..9bcd5de 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.security.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
@@ -39,6 +40,7 @@ import org.jsr166.*;
 
 import java.io.*;
 import java.net.*;
+import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -58,6 +60,9 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
     /** Thread count for multithreaded tests. */
     private static final int THREAD_CNT = 8;
 
+    /** Secondary file system user. */
+    private static final String SECONDARY_FS_USER = "secondary-default";
+
     /** IP finder. */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
@@ -255,7 +260,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         if (mode != PRIMARY)
             cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(secondaryFileSystemUriPath(),
-                secondaryFileSystemConfigPath()));
+                secondaryFileSystemConfigPath(), SECONDARY_FS_USER));
 
         cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
         cfg.setManagementPort(-1);
@@ -278,11 +283,28 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         primaryFsCfg.addResource(U.resolveIgniteUrl(primaryFileSystemConfigPath()));
 
-        fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
+        UserGroupInformation ugi = UserGroupInformation.getBestUGI(null, getClientFsUser());
+
+        // Create Fs on behalf of the client user:
+        ugi.doAs(new PrivilegedExceptionAction<Object>() {
+            @Override public Object run() throws Exception {
+                fs = AbstractFileSystem.get(primaryFsUri, primaryFsCfg);
+
+                return null;
+            }
+        });
 
         barrier = new CyclicBarrier(THREAD_CNT);
     }
 
+    /**
+     * Gets the user the Fs client operates on bahalf of.
+     * @return The user the Fs client operates on bahalf of.
+     */
+    protected String getClientFsUser() {
+        return "foo";
+    }
+
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         try {
@@ -297,14 +319,17 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
     /** @throws Exception If failed. */
     public void testStatus() throws Exception {
+        Path file1 = new Path("/file1");
 
-        try (FSDataOutputStream file = fs.create(new Path("/file1"), EnumSet.noneOf(CreateFlag.class),
+        try (FSDataOutputStream file = fs.create(file1, EnumSet.noneOf(CreateFlag.class),
             Options.CreateOpts.perms(FsPermission.getDefault()))) {
             file.write(new byte[1024 * 1024]);
         }
 
         FsStatus status = fs.getFsStatus();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file1).getOwner());
+
         assertEquals(4, grid(0).cluster().nodes().size());
 
         long used = 0, max = 0;
@@ -707,6 +732,8 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
+
         fs.setOwner(file, "aUser", "aGroup");
 
         assertEquals("aUser", fs.getFileStatus(file).getOwner());
@@ -796,20 +823,20 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         int cnt = 2 * 1024;
 
-        FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
-            Options.CreateOpts.perms(FsPermission.getDefault()));
+        try (FSDataOutputStream out = fs.create(file, EnumSet.noneOf(CreateFlag.class),
+            Options.CreateOpts.perms(FsPermission.getDefault()))) {
 
-        for (long i = 0; i < cnt; i++)
-            out.writeLong(i);
+            for (long i = 0; i < cnt; i++)
+                out.writeLong(i);
+        }
 
-        out.close();
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
 
-        FSDataInputStream in = fs.open(file, 1024);
+        try (FSDataInputStream in = fs.open(file, 1024)) {
 
-        for (long i = 0; i < cnt; i++)
-            assertEquals(i, in.readLong());
-
-        in.close();
+            for (long i = 0; i < cnt; i++)
+                assertEquals(i, in.readLong());
+        }
     }
 
     /** @throws Exception If failed. */
@@ -1191,6 +1218,9 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
         assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
+
+        assertEquals(getClientFsUser(), fs.getFileStatus(dir).getOwner());
+        assertEquals(getClientFsUser(), fs.getFileStatus(nestedDir).getOwner());
     }
 
     /** @throws Exception If failed. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
index 9e84c51..b089995 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopSecondaryFileSystemConfigurationTest.java
@@ -162,9 +162,9 @@ public class HadoopSecondaryFileSystemConfigurationTest extends IgfsCommonAbstra
             primaryConfFullPath = null;
 
         SecondaryFileSystemProvider provider =
-            new SecondaryFileSystemProvider(primaryFsUriStr, primaryConfFullPath, null);
+            new SecondaryFileSystemProvider(primaryFsUriStr, primaryConfFullPath);
 
-        primaryFs = provider.createFileSystem();
+        primaryFs = provider.createFileSystem(null);
 
         primaryFsUri = provider.uri();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
index d3440fc..c0f73af 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
@@ -73,10 +73,7 @@ public class IgfsNearOnlyMultiNodeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index e8ef414..f215efb 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.permission.*;
+import org.apache.hadoop.security.*;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
@@ -43,6 +44,7 @@ import org.jsr166.*;
 import java.io.*;
 import java.lang.reflect.*;
 import java.net.*;
+import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -72,6 +74,9 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     /** Secondary file system configuration path. */
     private static final String SECONDARY_CFG_PATH = "/work/core-site-test.xml";
 
+    /** Secondary file system user. */
+    private static final String SECONDARY_FS_USER = "secondary-default";
+
     /** Secondary endpoint configuration. */
     protected static final IgfsIpcEndpointConfiguration SECONDARY_ENDPOINT_CFG;
 
@@ -145,6 +150,14 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
         endpoint = skipLocShmem ? "127.0.0.1:10500" : "shmem:10500";
     }
 
+    /**
+     * Gets the user the Fs client operates on bahalf of.
+     * @return The user the Fs client operates on bahalf of.
+     */
+    protected String getClientFsUser() {
+        return "foo";
+    }
+
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         Configuration secondaryConf = configuration(SECONDARY_AUTHORITY, true, true);
@@ -235,7 +248,17 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         primaryFsCfg = configuration(PRIMARY_AUTHORITY, skipEmbed, skipLocShmem);
 
-        fs = FileSystem.get(primaryFsUri, primaryFsCfg);
+        UserGroupInformation clientUgi = UserGroupInformation.getBestUGI(null, getClientFsUser());
+        assertNotNull(clientUgi);
+
+        // Create the Fs on behalf of the specific user:
+        clientUgi.doAs(new PrivilegedExceptionAction<Object>() {
+            @Override public Object run() throws Exception {
+                fs = FileSystem.get(primaryFsUri, primaryFsCfg);
+
+                return null;
+            }
+        });
 
         barrier = new CyclicBarrier(THREAD_CNT);
     }
@@ -324,7 +347,8 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
         cfg.setDefaultMode(mode);
 
         if (mode != PRIMARY)
-            cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG_PATH));
+            cfg.setSecondaryFileSystem(new IgniteHadoopIgfsSecondaryFileSystem(
+                SECONDARY_URI, SECONDARY_CFG_PATH, SECONDARY_FS_USER));
 
         cfg.setIpcEndpointConfiguration(primaryIpcEndpointConfiguration(gridName));
 
@@ -870,6 +894,8 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         os.close();
 
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
+
         fs.setOwner(file, "aUser", "aGroup");
 
         assertEquals("aUser", fs.getFileStatus(file).getOwner());
@@ -1001,19 +1027,19 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         int cnt = 2 * 1024;
 
-        FSDataOutputStream out = fs.create(file, true, 1024);
-
-        for (long i = 0; i < cnt; i++)
-            out.writeLong(i);
+        try (FSDataOutputStream out = fs.create(file, true, 1024)) {
 
-        out.close();
+            for (long i = 0; i < cnt; i++)
+                out.writeLong(i);
+        }
 
-        FSDataInputStream in = fs.open(file, 1024);
+        assertEquals(getClientFsUser(), fs.getFileStatus(file).getOwner());
 
-        for (long i = 0; i < cnt; i++)
-            assertEquals(i, in.readLong());
+        try (FSDataInputStream in = fs.open(file, 1024)) {
 
-        in.close();
+            for (long i = 0; i < cnt; i++)
+                assertEquals(i, in.readLong());
+        }
     }
 
     /** @throws Exception If failed. */
@@ -1344,7 +1370,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         String path = fs.getFileStatus(file).getPath().toString();
 
-        assertTrue(path.endsWith("/user/" + System.getProperty("user.name", "anonymous") + "/file"));
+        assertTrue(path.endsWith("/user/" + getClientFsUser() + "/file"));
     }
 
     /** @throws Exception If failed. */
@@ -1374,7 +1400,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     public void testGetWorkingDirectoryIfDefault() throws Exception {
         String path = fs.getWorkingDirectory().toString();
 
-        assertTrue(path.endsWith("/user/" + System.getProperty("user.name", "anonymous")));
+        assertTrue(path.endsWith("/user/" + getClientFsUser()));
     }
 
     /** @throws Exception If failed. */
@@ -1412,17 +1438,20 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     @SuppressWarnings("OctalInteger")
     public void testMkdirs() throws Exception {
         Path fsHome = new Path(PRIMARY_URI);
-        Path dir = new Path(fsHome, "/tmp/staging");
-        Path nestedDir = new Path(dir, "nested");
+        final Path dir = new Path(fsHome, "/tmp/staging");
+        final Path nestedDir = new Path(dir, "nested");
 
-        FsPermission dirPerm = FsPermission.createImmutable((short)0700);
-        FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
+        final FsPermission dirPerm = FsPermission.createImmutable((short)0700);
+        final FsPermission nestedDirPerm = FsPermission.createImmutable((short)111);
 
         assertTrue(fs.mkdirs(dir, dirPerm));
         assertTrue(fs.mkdirs(nestedDir, nestedDirPerm));
 
         assertEquals(dirPerm, fs.getFileStatus(dir).getPermission());
         assertEquals(nestedDirPerm, fs.getFileStatus(nestedDir).getPermission());
+
+        assertEquals(getClientFsUser(), fs.getFileStatus(dir).getOwner());
+        assertEquals(getClientFsUser(), fs.getFileStatus(nestedDir).getOwner());
     }
 
     /** @throws Exception If failed. */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
index b92b213..fcfd587 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemClientSelfTest.java
@@ -125,7 +125,7 @@ public class IgniteHadoopFileSystemClientSelfTest extends IgfsCommonAbstractTest
         try {
             switchHandlerErrorFlag(true);
 
-            HadoopIgfs client = new HadoopIgfsOutProc("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG);
+            HadoopIgfs client = new HadoopIgfsOutProc("127.0.0.1", 10500, getTestGridName(0), "igfs", LOG, null);
 
             client.handshake(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
index e103c5f..2c17ba9 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemIpcCacheSelfTest.java
@@ -144,6 +144,8 @@ public class IgniteHadoopFileSystemIpcCacheSelfTest extends IgfsCommonAbstractTe
 
         Map<String, HadoopIgfsIpcIo> cache = (Map<String, HadoopIgfsIpcIo>)cacheField.get(null);
 
+        cache.clear(); // avoid influence of previous tests in the same process.
+
         String name = "igfs:" + getTestGridName(0) + "@";
 
         Configuration cfg = new Configuration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
index af1a1e1..e8a0a6f 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopAbstractSelfTest.java
@@ -22,7 +22,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem;
 import org.apache.ignite.internal.processors.hadoop.fs.*;
-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.*;
@@ -62,6 +61,17 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
     /** Initial REST port. */
     private int restPort = REST_PORT;
 
+    /** Secondary file system REST endpoint configuration. */
+    protected static final IgfsIpcEndpointConfiguration SECONDARY_REST_CFG;
+
+    static {
+        SECONDARY_REST_CFG = new IgfsIpcEndpointConfiguration();
+
+        SECONDARY_REST_CFG.setType(IgfsIpcEndpointType.TCP);
+        SECONDARY_REST_CFG.setPort(11500);
+    }
+
+
     /** Initial classpath. */
     private static String initCp;
 
@@ -133,7 +143,7 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
     /**
      * @return IGFS configuration.
      */
-    public FileSystemConfiguration igfsConfiguration() {
+    public FileSystemConfiguration igfsConfiguration() throws Exception {
         FileSystemConfiguration cfg = new FileSystemConfiguration();
 
         cfg.setName(igfsName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
index d10ee5c..c66cdf3 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopCommandLineTest.java
@@ -19,12 +19,16 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import com.google.common.base.*;
 import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
+import org.apache.ignite.internal.processors.resource.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jsr166.*;
 
@@ -205,7 +209,15 @@ public class HadoopCommandLineTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        igfs = (IgfsEx) Ignition.start("config/hadoop/default-config.xml").fileSystem(igfsName);
+        String cfgPath = "config/hadoop/default-config.xml";
+
+        IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> tup = IgnitionEx.loadConfiguration(cfgPath);
+
+        IgniteConfiguration cfg = tup.get1();
+
+        cfg.setLocalHost("127.0.0.1"); // Avoid connecting to other nodes.
+
+        igfs = (IgfsEx) Ignition.start(cfg).fileSystem(igfsName);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
index 8cf31a2..5f90bd4 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopFileSystemsTest.java
@@ -33,6 +33,7 @@ import java.util.concurrent.atomic.*;
  * Test file systems for the working directory multi-threading support.
  */
 public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
+    /** the number of threads */
     private static final int THREAD_COUNT = 3;
 
     /** {@inheritDoc} */
@@ -87,10 +88,6 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
                 try {
                     int curThreadNum = threadNum.getAndIncrement();
 
-                    FileSystem fs = FileSystem.get(uri, cfg);
-
-                    HadoopFileSystemsUtils.setUser(fs, "user" + curThreadNum);
-
                     if ("file".equals(uri.getScheme()))
                         FileSystem.get(uri, cfg).setWorkingDirectory(new Path("file:///user/user" + curThreadNum));
 
@@ -149,24 +146,6 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
     }
 
     /**
-     * Test IGFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testIgfs() throws Exception {
-        testFileSystem(URI.create(igfsScheme()));
-    }
-
-    /**
-     * Test HDFS multi-thread working directory.
-     *
-     * @throws Exception If fails.
-     */
-    public void testHdfs() throws Exception {
-        testFileSystem(URI.create("hdfs://localhost/"));
-    }
-
-    /**
      * Test LocalFS multi-thread working directory.
      *
      * @throws Exception If fails.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 8a3a0ac..66c14b5 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
@@ -24,31 +24,104 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
 import org.apache.ignite.igfs.*;
+import org.apache.ignite.igfs.secondary.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.*;
+import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.util.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.igfs.IgfsMode.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
 /**
  * Test of whole cycle of map-reduce processing via Job tracker.
  */
 public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
+    /** IGFS block size. */
+    protected static final int IGFS_BLOCK_SIZE = 512 * 1024;
+
+    /** Amount of blocks to prefetch. */
+    protected static final int PREFETCH_BLOCKS = 1;
+
+    /** Amount of sequential block reads before prefetch is triggered. */
+    protected static final int SEQ_READS_BEFORE_PREFETCH = 2;
+
+    /** Secondary file system URI. */
+    protected static final String SECONDARY_URI = "igfs://igfs-secondary:grid-secondary@127.0.0.1:11500/";
+
+    /** Secondary file system configuration path. */
+    protected static final String SECONDARY_CFG = "modules/core/src/test/config/hadoop/core-site-loopback-secondary.xml";
+
+    /** The user to run Hadoop job on behalf of. */
+    protected static final String USER = "vasya";
+
+    /** Secondary IGFS name. */
+    protected static final String SECONDARY_IGFS_NAME = "igfs-secondary";
+
+    /** The secondary Ignite node. */
+    protected Ignite igniteSecondary;
+
+    /** The secondary Fs. */
+    protected IgfsSecondaryFileSystem secondaryFs;
+
     /** {@inheritDoc} */
     @Override protected int gridCount() {
         return 3;
     }
 
     /**
+     * Gets owner of a IgfsEx path.
+     * @param p The path.
+     * @return The owner.
+     */
+    private static String getOwner(IgfsEx i, IgfsPath p) {
+        return i.info(p).property(IgfsEx.PROP_USER_NAME);
+    }
+
+    /**
+     * Gets owner of a secondary Fs path.
+     * @param secFs The sec Fs.
+     * @param p The path.
+     * @return The owner.
+     */
+    private static String getOwnerSecondary(final IgfsSecondaryFileSystem secFs, final IgfsPath p) {
+        return IgfsUserContext.doAs(USER, new IgniteOutClosure<String>() {
+            @Override public String apply() {
+                return secFs.info(p).property(IgfsEx.PROP_USER_NAME);
+            }
+        });
+    }
+
+    /**
+     * Checks owner of the path.
+     * @param p The path.
+     */
+    private void checkOwner(IgfsPath p) {
+        String ownerPrim = getOwner(igfs, p);
+        assertEquals(USER, ownerPrim);
+
+        String ownerSec = getOwnerSecondary(secondaryFs, p);
+        assertEquals(USER, ownerSec);
+    }
+
+    /**
      * Tests whole job execution with all phases in all combination of new and old versions of API.
      * @throws Exception If fails.
      */
@@ -59,9 +132,14 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
 
         IgfsPath inFile = new IgfsPath(inDir, HadoopWordCount2.class.getSimpleName() + "-input");
 
-        generateTestFile(inFile.toString(), "red", 100000, "blue", 200000, "green", 150000, "yellow", 70000 );
+        final int red = 10_000;
+        final int blue = 20_000;
+        final int green = 15_000;
+        final int yellow = 7_000;
 
-        for (int i = 0; i < 8; i++) {
+        generateTestFile(inFile.toString(), "red", red, "blue", blue, "green", green, "yellow", yellow );
+
+        for (int i = 0; i < 3; i++) {
             igfs.delete(new IgfsPath(PATH_OUTPUT), true);
 
             boolean useNewMapper = (i & 1) == 0;
@@ -71,7 +149,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             JobConf jobConf = new JobConf();
 
             jobConf.set(JOB_COUNTER_WRITER_PROPERTY, IgniteHadoopFileSystemCounterWriter.class.getName());
-            jobConf.setUser("yyy");
+            jobConf.setUser(USER);
             jobConf.set(IgniteHadoopFileSystemCounterWriter.COUNTER_WRITER_DIR_PROPERTY, "/xxx/${USER}/zzz");
 
             //To split into about 40 items for v2
@@ -105,13 +183,19 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
 
             checkJobStatistics(jobId);
 
+            final String outFile = PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000";
+
+            checkOwner(new IgfsPath(PATH_OUTPUT + "/" + "_SUCCESS"));
+
+            checkOwner(new IgfsPath(outFile));
+
             assertEquals("Use new mapper: " + useNewMapper + ", new combiner: " + useNewCombiner + ", new reducer: " +
                 useNewReducer,
-                "blue\t200000\n" +
-                "green\t150000\n" +
-                "red\t100000\n" +
-                "yellow\t70000\n",
-                readAndSortFile(PATH_OUTPUT + "/" + (useNewReducer ? "part-r-" : "part-") + "00000")
+                "blue\t" + blue + "\n" +
+                "green\t" + green + "\n" +
+                "red\t" + red + "\n" +
+                "yellow\t" + yellow + "\n",
+                readAndSortFile(outFile)
             );
         }
     }
@@ -182,7 +266,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             }
         }
 
-        final IgfsPath statPath = new IgfsPath("/xxx/yyy/zzz/" + jobId + "/performance");
+        final IgfsPath statPath = new IgfsPath("/xxx/" + USER + "/zzz/" + jobId + "/performance");
 
         assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
@@ -212,4 +296,85 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
                 ", actual=" + HadoopTestUtils.simpleCheckJobStatFile(reader) + ']';
         }
     }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        igniteSecondary = startGridWithIgfs("grid-secondary", SECONDARY_IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+
+        super.beforeTest();
+    }
+
+    /**
+     * Start grid with IGFS.
+     *
+     * @param gridName Grid name.
+     * @param igfsName IGFS name
+     * @param mode IGFS mode.
+     * @param secondaryFs Secondary file system (optional).
+     * @param restCfg Rest configuration string (optional).
+     * @return Started grid instance.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
+
+        igfsCfg.setDataCacheName("dataCache");
+        igfsCfg.setMetaCacheName("metaCache");
+        igfsCfg.setName(igfsName);
+        igfsCfg.setBlockSize(IGFS_BLOCK_SIZE);
+        igfsCfg.setDefaultMode(mode);
+        igfsCfg.setIpcEndpointConfiguration(restCfg);
+        igfsCfg.setSecondaryFileSystem(secondaryFs);
+        igfsCfg.setPrefetchBlocks(PREFETCH_BLOCKS);
+        igfsCfg.setSequentialReadsBeforePrefetch(SEQ_READS_BEFORE_PREFETCH);
+
+        CacheConfiguration dataCacheCfg = defaultCacheConfiguration();
+
+        dataCacheCfg.setName("dataCache");
+        dataCacheCfg.setCacheMode(PARTITIONED);
+        dataCacheCfg.setNearConfiguration(null);
+        dataCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        dataCacheCfg.setAffinityMapper(new IgfsGroupDataBlocksKeyMapper(2));
+        dataCacheCfg.setBackups(0);
+        dataCacheCfg.setAtomicityMode(TRANSACTIONAL);
+        dataCacheCfg.setOffHeapMaxMemory(0);
+
+        CacheConfiguration metaCacheCfg = defaultCacheConfiguration();
+
+        metaCacheCfg.setName("metaCache");
+        metaCacheCfg.setCacheMode(REPLICATED);
+        metaCacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        metaCacheCfg.setAtomicityMode(TRANSACTIONAL);
+
+        IgniteConfiguration cfg = new IgniteConfiguration();
+
+        cfg.setGridName(gridName);
+
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+
+        cfg.setDiscoverySpi(discoSpi);
+        cfg.setCacheConfiguration(dataCacheCfg, metaCacheCfg);
+        cfg.setFileSystemConfiguration(igfsCfg);
+
+        cfg.setLocalHost("127.0.0.1");
+        cfg.setConnectorConfiguration(null);
+
+        return G.start(cfg);
+    }
+
+    /**
+     * @return IGFS configuration.
+     */
+    @Override public FileSystemConfiguration igfsConfiguration() throws Exception {
+        FileSystemConfiguration fsCfg = super.igfsConfiguration();
+
+        secondaryFs = new IgniteHadoopIgfsSecondaryFileSystem(SECONDARY_URI, SECONDARY_CFG);
+
+        fsCfg.setSecondaryFileSystem(secondaryFs);
+
+        return fsCfg;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
index 8dc9830..eee5c8b 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskExecutionSelfTest.java
@@ -72,7 +72,7 @@ public class HadoopTaskExecutionSelfTest extends HadoopAbstractSelfTest {
 
 
     /** {@inheritDoc} */
-    @Override public FileSystemConfiguration igfsConfiguration() {
+    @Override public FileSystemConfiguration igfsConfiguration() throws Exception {
         FileSystemConfiguration cfg = super.igfsConfiguration();
 
         cfg.setFragmentizerEnabled(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
index aaf0f92..6930020 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksAllVersionsTest.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.io.*;
 import org.apache.ignite.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.io.*;
 import java.net.*;
@@ -43,7 +42,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    public abstract HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception;
+    public abstract HadoopJob getHadoopJob(String inFile, String outFile) throws Exception;
 
     /**
      * @return prefix of reducer output file name. It's "part-" for v1 and "part-r-" for v2 API
@@ -79,7 +78,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, fileBlock1.length(),
                 igfs.info(inFile).length() - fileBlock1.length());
 
-        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(igfsScheme() + inFile.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock1);
 
@@ -110,7 +109,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runTaskWithInput(HadoopV2Job gridJob, HadoopTaskType taskType,
+    private HadoopTestTaskContext runTaskWithInput(HadoopJob gridJob, HadoopTaskType taskType,
         int taskNum, String... words) throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(taskType, gridJob.id(), taskNum, 0, null);
 
@@ -136,7 +135,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testReduceTask() throws Exception {
-        HadoopV2Job gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(igfsScheme() + PATH_INPUT, igfsScheme() + PATH_OUTPUT);
 
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 0, "word1", "5", "word2", "10");
         runTaskWithInput(gridJob, HadoopTaskType.REDUCE, 1, "word3", "7", "word4", "15");
@@ -162,7 +161,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @throws Exception If fails.
      */
     public void testCombinerTask() throws Exception {
-        HadoopV2Job gridJob = getHadoopJob("/", "/");
+        HadoopJob gridJob = getHadoopJob("/", "/");
 
         HadoopTestTaskContext ctx =
             runTaskWithInput(gridJob, HadoopTaskType.COMBINE, 0, "word1", "5", "word2", "10");
@@ -182,7 +181,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
      * @return Context of combine task with mock output.
      * @throws IgniteCheckedException If fails.
      */
-    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopV2Job gridJob)
+    private HadoopTestTaskContext runMapCombineTask(HadoopFileBlock fileBlock, HadoopJob gridJob)
         throws IgniteCheckedException {
         HadoopTaskInfo taskInfo = new HadoopTaskInfo(HadoopTaskType.MAP, gridJob.id(), 0, 0, fileBlock);
 
@@ -228,7 +227,7 @@ abstract class HadoopTasksAllVersionsTest extends HadoopAbstractWordCountTest {
         HadoopFileBlock fileBlock1 = new HadoopFileBlock(HOSTS, inFileUri, 0, l);
         HadoopFileBlock fileBlock2 = new HadoopFileBlock(HOSTS, inFileUri, l, fileLen - l);
 
-        HadoopV2Job gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
+        HadoopJob gridJob = getHadoopJob(inFileUri.toString(), igfsScheme() + PATH_OUTPUT);
 
         HadoopTestTaskContext combine1Ctx = runMapCombineTask(fileBlock1, gridJob);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 b41a260..48e83cc 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,7 +19,6 @@ 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.*;
@@ -38,7 +37,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws IOException If fails.
      */
-    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
         JobConf jobConf = HadoopWordCount1.getJob(inFile, outFile);
 
         setupFileSystems(jobConf);
@@ -47,7 +46,7 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
 
-        return new HadoopV2Job(jobId, jobInfo, log);
+        return jobInfo.createJob(jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 b677c63..e73fae3 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,7 +24,6 @@ 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.*;
 
@@ -42,7 +41,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
      * @return Hadoop job.
      * @throws Exception if fails.
      */
-    @Override public HadoopV2Job getHadoopJob(String inFile, String outFile) throws Exception {
+    @Override public HadoopJob getHadoopJob(String inFile, String outFile) throws Exception {
         Job job = Job.getInstance();
 
         job.setOutputKeyClass(Text.class);
@@ -65,7 +64,7 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
 
         HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
 
-        return new HadoopV2Job(jobId, jobInfo, log);
+        return jobInfo.createJob(jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/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 ebc89f4..f3b9307 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
@@ -66,7 +66,11 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
         cfg.setMapOutputValueClass(Text.class);
         cfg.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
 
-        HadoopJob job = new HadoopV2Job(new HadoopJobId(UUID.randomUUID(), 1), createJobInfo(cfg), log);
+        HadoopDefaultJobInfo info = createJobInfo(cfg);
+
+        HadoopJobId id = new HadoopJobId(UUID.randomUUID(), 1);
+
+        HadoopJob job = info.createJob(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/1652fd18/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 b4ed5e1..9395c5e 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
@@ -28,6 +28,7 @@ import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 /**
  * Abstract class for maps test.
@@ -95,9 +96,20 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
             assert false;
         }
 
+        /** {@inheritDoc} */
         @Override public void cleanupTaskEnvironment() throws IgniteCheckedException {
             assert false;
         }
+
+        /** {@inheritDoc} */
+        @Override public <T> T runAsJobOwner(Callable<T> c) throws IgniteCheckedException {
+            try {
+                return c.call();
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException(e);
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
index 8a046e0..89bf830 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipListSelfTest.java
@@ -61,10 +61,10 @@ public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
 
             int sigma = max((int)ceil(precission * exp), 5);
 
-            X.println("Level: " + level + " exp: " + exp + " act: " + levelsCnts[level] + " precission: " + precission +
+            X.println("Level: " + level + " exp: " + exp + " act: " + levelsCnts[level] + " precision: " + precission +
                 " sigma: " + sigma);
 
-            assertTrue(abs(exp - levelsCnts[level]) <= sigma);
+            assertTrue(abs(exp - levelsCnts[level]) <= sigma); // Sometimes fails.
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
new file mode 100644
index 0000000..cfad322
--- /dev/null
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/store/hibernate/CacheHibernateStoreSessionListener.java
@@ -0,0 +1,216 @@
+/*
+ * 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.cache.store.hibernate;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.resources.*;
+import org.hibernate.*;
+import org.hibernate.cfg.*;
+
+import javax.cache.integration.*;
+import java.io.*;
+import java.net.*;
+
+/**
+ * Hibernate-based cache store session listener.
+ * <p>
+ * This listener creates a new Hibernate session for each store
+ * session. If there is an ongoing cache transaction, a corresponding
+ * Hibernate transaction is created as well.
+ * <p>
+ * The Hibernate session is saved as a store session
+ * {@link CacheStoreSession#attachment() attachment}.
+ * The listener guarantees that the session will be
+ * available for any store operation. If there is an
+ * ongoing cache transaction, all operations within this
+ * transaction will share a DB transaction.
+ * <p>
+ * As an example, here is how the {@link CacheStore#write(javax.cache.Cache.Entry)}
+ * method can be implemented if {@link CacheHibernateStoreSessionListener}
+ * is configured:
+ * <pre name="code" class="java">
+ * private static class Store extends CacheStoreAdapter&lt;Integer, Integer&gt; {
+ *     &#64;CacheStoreSessionResource
+ *     private CacheStoreSession ses;
+ *
+ *     &#64;Override public void write(Cache.Entry&lt;? extends Integer, ? extends Integer&gt; entry) throws CacheWriterException {
+ *         // Get Hibernate session from the current store session.
+ *         Session hibSes = ses.attachment();
+ *
+ *         // Persist the value.
+ *         hibSes.persist(entry.getValue());
+ *     }
+ * }
+ * </pre>
+ * Hibernate session will be automatically created by the listener
+ * at the start of the session and closed when it ends.
+ * <p>
+ * {@link CacheHibernateStoreSessionListener} requires that either
+ * {@link #setSessionFactory(SessionFactory)} session factory}
+ * or {@link #setHibernateConfigurationPath(String) Hibernate configuration file}
+ * is provided. If non of them is set, exception is thrown. Is both are provided,
+ * session factory will be used.
+ */
+public class CacheHibernateStoreSessionListener implements CacheStoreSessionListener, LifecycleAware {
+    /** Hibernate session factory. */
+    private SessionFactory sesFactory;
+
+    /** Hibernate configuration file path. */
+    private String hibernateCfgPath;
+
+    /** Logger. */
+    @LoggerResource
+    private IgniteLogger log;
+
+    /** Whether to close session on stop. */
+    private boolean closeSesOnStop;
+
+    /**
+     * Sets Hibernate session factory.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param sesFactory Session factory.
+     */
+    public void setSessionFactory(SessionFactory sesFactory) {
+        this.sesFactory = sesFactory;
+    }
+
+    /**
+     * Gets Hibernate session factory.
+     *
+     * @return Session factory.
+     */
+    public SessionFactory getSessionFactory() {
+        return sesFactory;
+    }
+
+    /**
+     * Sets hibernate configuration path.
+     * <p>
+     * Either session factory or configuration file is required.
+     * If none is provided, exception will be thrown on startup.
+     *
+     * @param hibernateCfgPath Hibernate configuration path.
+     */
+    public void setHibernateConfigurationPath(String hibernateCfgPath) {
+        this.hibernateCfgPath = hibernateCfgPath;
+    }
+
+    /**
+     * Gets hibernate configuration path.
+     *
+     * @return Hibernate configuration path.
+     */
+    public String getHibernateConfigurationPath() {
+        return hibernateCfgPath;
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override public void start() throws IgniteException {
+        if (sesFactory == null && F.isEmpty(hibernateCfgPath))
+            throw new IgniteException("Either session factory or Hibernate configuration file is required by " +
+                getClass().getSimpleName() + '.');
+
+        if (!F.isEmpty(hibernateCfgPath)) {
+            if (sesFactory == null) {
+                try {
+                    URL url = new URL(hibernateCfgPath);
+
+                    sesFactory = new Configuration().configure(url).buildSessionFactory();
+                }
+                catch (MalformedURLException ignored) {
+                    // No-op.
+                }
+
+                if (sesFactory == null) {
+                    File cfgFile = new File(hibernateCfgPath);
+
+                    if (cfgFile.exists())
+                        sesFactory = new Configuration().configure(cfgFile).buildSessionFactory();
+                }
+
+                if (sesFactory == null)
+                    sesFactory = new Configuration().configure(hibernateCfgPath).buildSessionFactory();
+
+                if (sesFactory == null)
+                    throw new IgniteException("Failed to resolve Hibernate configuration file: " + hibernateCfgPath);
+
+                closeSesOnStop = true;
+            }
+            else
+                U.warn(log, "Hibernate configuration file configured in " + getClass().getSimpleName() +
+                    " will be ignored (session factory is already set).");
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop() throws IgniteException {
+        if (closeSesOnStop && sesFactory != null && !sesFactory.isClosed())
+            sesFactory.close();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionStart(CacheStoreSession ses) {
+        if (ses.attachment() == null) {
+            try {
+                Session hibSes = sesFactory.openSession();
+
+                ses.attach(hibSes);
+
+                if (ses.isWithinTransaction())
+                    hibSes.beginTransaction();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to start store session [tx=" + ses.transaction() + ']', e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onSessionEnd(CacheStoreSession ses, boolean commit) {
+        Session hibSes = ses.attach(null);
+
+        if (hibSes != null) {
+            try {
+                Transaction tx = hibSes.getTransaction();
+
+                if (commit) {
+                    hibSes.flush();
+
+                    if (tx.isActive())
+                        tx.commit();
+                }
+                else if (tx.isActive())
+                    tx.rollback();
+            }
+            catch (HibernateException e) {
+                throw new CacheWriterException("Failed to end store session [tx=" + ses.transaction() + ']', e);
+            }
+            finally {
+                hibSes.close();
+            }
+        }
+    }
+}


[15/28] incubator-ignite git commit: ignite-545: merge from sprint-6

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
deleted file mode 100644
index 4196306..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ /dev/null
@@ -1,1264 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spi.discovery.tcp;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.events.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.*;
-import org.apache.ignite.spi.discovery.tcp.internal.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
-import org.apache.ignite.spi.discovery.tcp.messages.*;
-import org.jetbrains.annotations.*;
-import org.jsr166.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHeartbeatMessage.*;
-
-/**
- * Client discovery SPI implementation that uses TCP/IP for node discovery.
- * <p>
- * This discovery SPI requires at least on server node configured with
- * {@link TcpDiscoverySpi}. It will try to connect to random IP taken from
- * {@link TcpDiscoveryIpFinder} which should point to one of these server
- * nodes and will maintain connection only with this node (will not enter the ring).
- * If this connection is broken, it will try to reconnect using addresses from
- * the same IP finder.
- */
-@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-@IgniteSpiMultipleInstancesSupport(true)
-@DiscoverySpiOrderSupport(true)
-@DiscoverySpiHistorySupport(true)
-public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpClientDiscoverySpiMBean {
-    /** Default disconnect check interval. */
-    public static final long DFLT_DISCONNECT_CHECK_INT = 2000;
-
-    /** Remote nodes. */
-    private final ConcurrentMap<UUID, TcpDiscoveryNode> rmtNodes = new ConcurrentHashMap8<>();
-
-    /** Socket. */
-    private volatile Socket sock;
-
-    /** Socket reader. */
-    private volatile SocketReader sockRdr;
-
-    /** Heartbeat sender. */
-    private volatile HeartbeatSender hbSender;
-
-    /** Disconnect handler. */
-    private volatile DisconnectHandler disconnectHnd;
-
-    /** Last message ID. */
-    private volatile IgniteUuid lastMsgId;
-
-    /** Current topology version. */
-    private volatile long topVer;
-
-    /** Join error. */
-    private IgniteSpiException joinErr;
-
-    /** Whether reconnect failed. */
-    private boolean reconFailed;
-
-    /** Joined latch. */
-    private CountDownLatch joinLatch;
-
-    /** Left latch. */
-    private volatile CountDownLatch leaveLatch;
-
-    /** Disconnect check interval. */
-    private long disconnectCheckInt = DFLT_DISCONNECT_CHECK_INT;
-
-    /** {@inheritDoc} */
-    @Override public long getDisconnectCheckInterval() {
-        return disconnectCheckInt;
-    }
-
-    /**
-     * Sets disconnect check interval.
-     *
-     * @param disconnectCheckInt Disconnect check interval.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setDisconnectCheckInterval(long disconnectCheckInt) {
-        this.disconnectCheckInt = disconnectCheckInt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getSocketTimeout() {
-        return sockTimeout;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getAckTimeout() {
-        return ackTimeout;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getNetworkTimeout() {
-        return netTimeout;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getThreadPriority() {
-        return threadPri;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getHeartbeatFrequency() {
-        return hbFreq;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getIpFinderFormatted() {
-        return ipFinder.toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getMessageWorkerQueueSize() {
-        SocketReader sockRdr0 = sockRdr;
-
-        return sockRdr0 != null ? sockRdr0.msgWrk.queueSize() : 0;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getNodesJoined() {
-        return stats.joinedNodesCount();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getNodesLeft() {
-        return stats.leftNodesCount();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getNodesFailed() {
-        return stats.failedNodesCount();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getAvgMessageProcessingTime() {
-        return stats.avgMessageProcessingTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getMaxMessageProcessingTime() {
-        return stats.maxMessageProcessingTime();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalReceivedMessages() {
-        return stats.totalReceivedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Integer> getReceivedMessages() {
-        return stats.receivedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getTotalProcessedMessages() {
-        return stats.totalProcessedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Map<String, Integer> getProcessedMessages() {
-        return stats.processedMessages();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        assertParameter(ipFinder != null, "ipFinder != null");
-        assertParameter(netTimeout > 0, "networkTimeout > 0");
-        assertParameter(sockTimeout > 0, "sockTimeout > 0");
-        assertParameter(ackTimeout > 0, "ackTimeout > 0");
-        assertParameter(hbFreq > 0, "heartbeatFreq > 0");
-        assertParameter(threadPri > 0, "threadPri > 0");
-
-        try {
-            locHost = U.resolveLocalHost(locAddr);
-        }
-        catch (IOException e) {
-            throw new IgniteSpiException("Unknown local address: " + locAddr, e);
-        }
-
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("localHost", locHost.getHostAddress()));
-            log.debug(configInfo("threadPri", threadPri));
-            log.debug(configInfo("networkTimeout", netTimeout));
-            log.debug(configInfo("sockTimeout", sockTimeout));
-            log.debug(configInfo("ackTimeout", ackTimeout));
-            log.debug(configInfo("ipFinder", ipFinder));
-            log.debug(configInfo("heartbeatFreq", hbFreq));
-        }
-
-        // Warn on odd network timeout.
-        if (netTimeout < 3000)
-            U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
-
-        registerMBean(gridName, this, TcpClientDiscoverySpiMBean.class);
-
-        try {
-            locHost = U.resolveLocalHost(locAddr);
-        }
-        catch (IOException e) {
-            throw new IgniteSpiException("Unknown local address: " + locAddr, e);
-        }
-
-        if (ipFinder instanceof TcpDiscoveryMulticastIpFinder) {
-            TcpDiscoveryMulticastIpFinder mcastIpFinder = ((TcpDiscoveryMulticastIpFinder)ipFinder);
-
-            if (mcastIpFinder.getLocalAddress() == null)
-                mcastIpFinder.setLocalAddress(locAddr);
-        }
-
-        IgniteBiTuple<Collection<String>, Collection<String>> addrs;
-
-        try {
-            addrs = U.resolveLocalAddresses(locHost);
-        }
-        catch (IOException | IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to resolve local host to set of external addresses: " + locHost, e);
-        }
-
-        locNode = new TcpDiscoveryNode(
-            getLocalNodeId(),
-            addrs.get1(),
-            addrs.get2(),
-            0,
-            metricsProvider,
-            locNodeVer);
-
-        locNode.setAttributes(locNodeAttrs);
-        locNode.local(true);
-
-        sockTimeoutWorker = new SocketTimeoutWorker();
-        sockTimeoutWorker.start();
-
-        joinTopology(false);
-
-        disconnectHnd = new DisconnectHandler();
-        disconnectHnd.start();
-
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        rmtNodes.clear();
-
-        U.interrupt(disconnectHnd);
-        U.join(disconnectHnd, log);
-
-        U.interrupt(hbSender);
-        U.join(hbSender, log);
-
-        Socket sock0 = sock;
-
-        sock = null;
-
-        if (sock0 != null) {
-            leaveLatch = new CountDownLatch(1);
-
-            try {
-                TcpDiscoveryNodeLeftMessage msg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId());
-
-                msg.client(true);
-
-                writeToSocket(sock0, msg);
-
-                if (!U.await(leaveLatch, netTimeout, MILLISECONDS)) {
-                    if (log.isDebugEnabled())
-                        log.debug("Did not receive node left message for local node (will stop anyway) [sock=" +
-                            sock0 + ']');
-                }
-            }
-            catch (IOException | IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    U.error(log, "Failed to send node left message (will stop anyway) [sock=" + sock0 + ']', e);
-            }
-            finally {
-                U.closeQuiet(sock0);
-            }
-        }
-
-        U.interrupt(sockRdr);
-        U.join(sockRdr, log);
-
-        U.interrupt(sockTimeoutWorker);
-        U.join(sockTimeoutWorker, log);
-
-        unregisterMBean();
-
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<Object> injectables() {
-        return Arrays.<Object>asList(ipFinder);
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> getRemoteNodes() {
-        return F.view(U.<TcpDiscoveryNode, ClusterNode>arrayList(rmtNodes.values(), new P1<TcpDiscoveryNode>() {
-            @Override public boolean apply(TcpDiscoveryNode node) {
-                return node.visible();
-            }
-        }));
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
-        if (getLocalNodeId().equals(nodeId))
-            return locNode;
-
-        TcpDiscoveryNode node = rmtNodes.get(nodeId);
-
-        return node != null && node.visible() ? node : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean pingNode(UUID nodeId) {
-        assert nodeId != null;
-
-        if (nodeId.equals(getLocalNodeId()))
-            return true;
-
-        TcpDiscoveryNode node = rmtNodes.get(nodeId);
-
-        return node != null && node.visible();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disconnect() throws IgniteSpiException {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendCustomEvent(Serializable evt) {
-        throw new UnsupportedOperationException();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        ClusterNode node = rmtNodes.get(nodeId);
-
-        if (node != null) {
-            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
-                node.id(), node.order());
-
-            sockRdr.addMessage(msg);
-        }
-    }
-
-    /**
-     * @param recon Reconnect flag.
-     * @return Whether joined successfully.
-     * @throws IgniteSpiException In case of error.
-     */
-    private boolean joinTopology(boolean recon) throws IgniteSpiException {
-        if (!recon)
-            stats.onJoinStarted();
-
-        Collection<InetSocketAddress> addrs = null;
-
-        while (!Thread.currentThread().isInterrupted()) {
-            try {
-                while (addrs == null || addrs.isEmpty()) {
-                    addrs = resolvedAddresses();
-
-                    if (!F.isEmpty(addrs)) {
-                        if (log.isDebugEnabled())
-                            log.debug("Resolved addresses from IP finder: " + addrs);
-                    }
-                    else {
-                        U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + ipFinder);
-
-                        U.sleep(2000);
-                    }
-                }
-
-                Collection<InetSocketAddress> addrs0 = new ArrayList<>(addrs);
-
-                Iterator<InetSocketAddress> it = addrs.iterator();
-
-                while (it.hasNext() && !Thread.currentThread().isInterrupted()) {
-                    InetSocketAddress addr = it.next();
-
-                    Socket sock = null;
-
-                    try {
-                        long ts = U.currentTimeMillis();
-
-                        IgniteBiTuple<Socket, UUID> t = initConnection(addr);
-
-                        sock = t.get1();
-
-                        UUID rmtNodeId = t.get2();
-
-                        stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
-
-                        locNode.clientRouterNodeId(rmtNodeId);
-
-                        TcpDiscoveryAbstractMessage msg = recon ?
-                            new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId,
-                                lastMsgId) :
-                            new TcpDiscoveryJoinRequestMessage(locNode, null);
-
-                        msg.client(true);
-
-                        writeToSocket(sock, msg);
-
-                        int res = readReceipt(sock, ackTimeout);
-
-                        switch (res) {
-                            case RES_OK:
-                                this.sock = sock;
-
-                                sockRdr = new SocketReader(rmtNodeId, new MessageWorker(recon));
-                                sockRdr.start();
-
-                                if (U.await(joinLatch, netTimeout, MILLISECONDS)) {
-                                    IgniteSpiException joinErr0 = joinErr;
-
-                                    if (joinErr0 != null)
-                                        throw joinErr0;
-
-                                    if (reconFailed) {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Failed to reconnect, will try to rejoin [locNode=" +
-                                                locNode + ']');
-
-                                        U.closeQuiet(sock);
-
-                                        U.interrupt(sockRdr);
-                                        U.join(sockRdr, log);
-
-                                        this.sock = null;
-
-                                        return false;
-                                    }
-
-                                    if (log.isDebugEnabled())
-                                        log.debug("Successfully connected to topology [sock=" + sock + ']');
-
-                                    hbSender = new HeartbeatSender();
-                                    hbSender.start();
-
-                                    stats.onJoinFinished();
-
-                                    return true;
-                                }
-                                else {
-                                    U.warn(log, "Join process timed out (will try other address) [sock=" + sock +
-                                        ", timeout=" + netTimeout + ']');
-
-                                    U.closeQuiet(sock);
-
-                                    U.interrupt(sockRdr);
-                                    U.join(sockRdr, log);
-
-                                    it.remove();
-
-                                    break;
-                                }
-
-                            case RES_CONTINUE_JOIN:
-                            case RES_WAIT:
-                                U.closeQuiet(sock);
-
-                                break;
-
-                            default:
-                                if (log.isDebugEnabled())
-                                    log.debug("Received unexpected response to join request: " + res);
-
-                                U.closeQuiet(sock);
-                        }
-                    }
-                    catch (IgniteInterruptedCheckedException ignored) {
-                        if (log.isDebugEnabled())
-                            log.debug("Joining thread was interrupted.");
-
-                        return false;
-                    }
-                    catch (IOException | IgniteCheckedException e) {
-                        if (log.isDebugEnabled())
-                            U.error(log, "Failed to establish connection with address: " + addr, e);
-
-                        U.closeQuiet(sock);
-
-                        it.remove();
-                    }
-                }
-
-                if (addrs.isEmpty()) {
-                    U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
-                        "in 2000ms): " + addrs0);
-
-                    U.sleep(2000);
-                }
-            }
-            catch (IgniteInterruptedCheckedException ignored) {
-                if (log.isDebugEnabled())
-                    log.debug("Joining thread was interrupted.");
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * @param addr Address.
-     * @return Remote node ID.
-     * @throws IOException In case of I/O error.
-     * @throws IgniteCheckedException In case of other error.
-     */
-    private IgniteBiTuple<Socket, UUID> initConnection(InetSocketAddress addr) throws IOException, IgniteCheckedException {
-        assert addr != null;
-
-        joinLatch = new CountDownLatch(1);
-
-        Socket sock = openSocket(addr);
-
-        TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId());
-
-        req.client(true);
-
-        writeToSocket(sock, req);
-
-        TcpDiscoveryHandshakeResponse res = readMessage(sock, null, ackTimeout);
-
-        UUID nodeId = res.creatorNodeId();
-
-        assert nodeId != null;
-        assert !getLocalNodeId().equals(nodeId);
-
-        return F.t(sock, nodeId);
-    }
-
-    /**
-     * FOR TEST PURPOSE ONLY!
-     */
-    void simulateNodeFailure() {
-        U.warn(log, "Simulating client node failure: " + getLocalNodeId());
-
-        U.closeQuiet(sock);
-
-        U.interrupt(disconnectHnd);
-        U.join(disconnectHnd, log);
-
-        U.interrupt(hbSender);
-        U.join(hbSender, log);
-
-        U.interrupt(sockRdr);
-        U.join(sockRdr, log);
-
-        U.interrupt(sockTimeoutWorker);
-        U.join(sockTimeoutWorker, log);
-    }
-
-    /**
-     * Disconnect handler.
-     */
-    private class DisconnectHandler extends IgniteSpiThread {
-        /**
-         */
-        protected DisconnectHandler() {
-            super(gridName, "tcp-client-disco-disconnect-hnd", log);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            while (!isInterrupted()) {
-                try {
-                    U.sleep(disconnectCheckInt);
-
-                    if (sock == null) {
-                        if (log.isDebugEnabled())
-                            log.debug("Node is disconnected from topology, will try to reconnect.");
-
-                        U.interrupt(hbSender);
-                        U.join(hbSender, log);
-
-                        U.interrupt(sockRdr);
-                        U.join(sockRdr, log);
-
-                        // If reconnection fails, try to rejoin.
-                        if (!joinTopology(true)) {
-                            rmtNodes.clear();
-
-                            locNode.order(0);
-
-                            joinTopology(false);
-
-                            getSpiContext().recordEvent(new DiscoveryEvent(locNode,
-                                "Client node reconnected: " + locNode,
-                                EVT_CLIENT_NODE_RECONNECTED, locNode));
-                        }
-                    }
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    if (log.isDebugEnabled())
-                        log.debug("Disconnect handler was interrupted.");
-
-                    return;
-                }
-                catch (IgniteSpiException e) {
-                    U.error(log, "Failed to reconnect to topology after failure.", e);
-                }
-            }
-        }
-    }
-
-    /**
-     * Heartbeat sender.
-     */
-    private class HeartbeatSender extends IgniteSpiThread {
-        /**
-         */
-        protected HeartbeatSender() {
-            super(gridName, "tcp-client-disco-heartbeat-sender", log);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            Socket sock0 = sock;
-
-            if (sock0 == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to start heartbeat sender, node is already disconnected.");
-
-                return;
-            }
-
-            try {
-                while (!isInterrupted()) {
-                    U.sleep(hbFreq);
-
-                    TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getLocalNodeId());
-
-                    msg.client(true);
-
-                    sockRdr.addMessage(msg);
-                }
-            }
-            catch (IgniteInterruptedCheckedException ignored) {
-                if (log.isDebugEnabled())
-                    log.debug("Heartbeat sender was interrupted.");
-            }
-        }
-    }
-
-    /**
-     * Socket reader.
-     */
-    private class SocketReader extends IgniteSpiThread {
-        /** Remote node ID. */
-        private final UUID nodeId;
-
-        /** Message worker. */
-        private final MessageWorker msgWrk;
-
-        /**
-         * @param nodeId Node ID.
-         * @param msgWrk Message worker.
-         */
-        protected SocketReader(UUID nodeId, MessageWorker msgWrk) {
-            super(gridName, "tcp-client-disco-sock-reader", log);
-
-            assert nodeId != null;
-            assert msgWrk != null;
-
-            this.nodeId = nodeId;
-            this.msgWrk = msgWrk;
-        }
-
-        /** {@inheritDoc} */
-        @Override public synchronized void start() {
-            super.start();
-
-            msgWrk.start();
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            Socket sock0 = sock;
-
-            if (sock0 == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to start socket reader, node is already disconnected.");
-
-                return;
-            }
-
-            try {
-                InputStream in = new BufferedInputStream(sock0.getInputStream());
-
-                sock0.setKeepAlive(true);
-                sock0.setTcpNoDelay(true);
-
-                while (!isInterrupted()) {
-                    try {
-                        TcpDiscoveryAbstractMessage msg = marsh.unmarshal(in, U.gridClassLoader());
-
-                        msg.senderNodeId(nodeId);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Message has been received: " + msg);
-
-                        stats.onMessageReceived(msg);
-
-                        if (joinLatch.getCount() > 0) {
-                            IgniteSpiException err = null;
-
-                            if (msg instanceof TcpDiscoveryDuplicateIdMessage)
-                                err = duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
-                            else if (msg instanceof TcpDiscoveryAuthFailedMessage)
-                                err = authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
-                            else if (msg instanceof TcpDiscoveryCheckFailedMessage)
-                                err = checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
-
-                            if (err != null) {
-                                joinErr = err;
-
-                                joinLatch.countDown();
-
-                                return;
-                            }
-                        }
-
-                        msgWrk.addMessage(msg);
-                    }
-                    catch (IgniteCheckedException e) {
-                        if (log.isDebugEnabled())
-                            U.error(log, "Failed to read message [sock=" + sock0 + ", " +
-                                "locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + nodeId + ']', e);
-
-                        IOException ioEx = X.cause(e, IOException.class);
-
-                        if (ioEx != null)
-                            throw ioEx;
-
-                        ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
-
-                        if (clsNotFoundEx != null)
-                            LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
-                                "(make sure same versions of all classes are available on all nodes) " +
-                                "[rmtNodeId=" + nodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
-                        else
-                            LT.error(log, e, "Failed to read message [sock=" + sock0 + ", locNodeId=" +
-                                getLocalNodeId() + ", rmtNodeId=" + nodeId + ']');
-                    }
-                }
-            }
-            catch (IOException e) {
-                if (log.isDebugEnabled())
-                    U.error(log, "Connection failed [sock=" + sock0 + ", locNodeId=" +
-                        getLocalNodeId() + ", rmtNodeId=" + nodeId + ']', e);
-            }
-            finally {
-                U.closeQuiet(sock0);
-
-                U.interrupt(msgWrk);
-
-                try {
-                    U.join(msgWrk);
-                }
-                catch (IgniteInterruptedCheckedException ignored) {
-                    // No-op.
-                }
-
-                sock = null;
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        void addMessage(TcpDiscoveryAbstractMessage msg) {
-            assert msg != null;
-
-            msgWrk.addMessage(msg);
-        }
-    }
-
-    /**
-     * Message worker.
-     */
-    private class MessageWorker extends MessageWorkerAdapter {
-        /** Topology history. */
-        private final NavigableMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
-
-        /** Indicates that reconnection is in progress. */
-        private boolean recon;
-
-        /** Indicates that pending messages are currently processed. */
-        private boolean pending;
-
-        /**
-         * @param recon Whether reconnection is in progress.
-         */
-        protected MessageWorker(boolean recon) {
-            super("tcp-client-disco-msg-worker");
-
-            this.recon = recon;
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void processMessage(TcpDiscoveryAbstractMessage msg) {
-            assert msg != null;
-            assert msg.verified() || msg.senderNodeId() == null;
-
-            stats.onMessageProcessingStarted(msg);
-
-            if (msg instanceof TcpDiscoveryClientReconnectMessage)
-                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
-            else {
-                if (recon && !pending) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding message received during reconnection: " + msg);
-                }
-                else {
-                    if (msg instanceof TcpDiscoveryNodeAddedMessage)
-                        processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
-                    else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
-                        processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
-                    else if (msg instanceof TcpDiscoveryNodeLeftMessage)
-                        processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
-                    else if (msg instanceof TcpDiscoveryNodeFailedMessage)
-                        processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
-                    else if (msg instanceof TcpDiscoveryHeartbeatMessage)
-                        processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
-
-                    if (ensured(msg))
-                        lastMsgId = msg.id();
-                }
-            }
-
-            stats.onMessageProcessingFinished(msg);
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
-            if (leaveLatch != null)
-                return;
-
-            TcpDiscoveryNode node = msg.node();
-
-            UUID newNodeId = node.id();
-
-            if (getLocalNodeId().equals(newNodeId)) {
-                if (joinLatch.getCount() > 0) {
-                    Collection<TcpDiscoveryNode> top = msg.topology();
-
-                    if (top != null) {
-                        gridStartTime = msg.gridStartTime();
-
-                        for (TcpDiscoveryNode n : top) {
-                            if (n.order() > 0)
-                                n.visible(true);
-
-                            rmtNodes.put(n.id(), n);
-                        }
-
-                        topHist.clear();
-
-                        if (msg.topologyHistory() != null)
-                            topHist.putAll(msg.topologyHistory());
-
-                        Map<UUID, Map<Integer, byte[]>> dataMap = msg.oldNodesDiscoveryData();
-
-                        if (dataMap != null) {
-                            for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                                onExchange(newNodeId, entry.getKey(), entry.getValue(), null);
-                        }
-
-                        locNode.setAttributes(node.attributes());
-                        locNode.visible(true);
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Discarding node added message with empty topology: " + msg);
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("Discarding node added message (this message has already been processed) " +
-                        "[msg=" + msg + ", locNode=" + locNode + ']');
-            }
-            else {
-                boolean topChanged = rmtNodes.putIfAbsent(newNodeId, node) == null;
-
-                if (topChanged) {
-                    if (log.isDebugEnabled())
-                        log.debug("Added new node to topology: " + node);
-
-                    Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
-
-                    if (data != null)
-                        onExchange(newNodeId, newNodeId, data, null);
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
-            if (leaveLatch != null)
-                return;
-
-            if (getLocalNodeId().equals(msg.nodeId())) {
-                if (joinLatch.getCount() > 0) {
-                    long topVer = msg.topologyVersion();
-
-                    locNode.order(topVer);
-
-                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer));
-
-                    joinErr = null;
-
-                    joinLatch.countDown();
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("Discarding node add finished message (this message has already been processed) " +
-                        "[msg=" + msg + ", locNode=" + locNode + ']');
-            }
-            else {
-                TcpDiscoveryNode node = rmtNodes.get(msg.nodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node add finished message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                long topVer = msg.topologyVersion();
-
-                node.order(topVer);
-                node.visible(true);
-
-                if (locNodeVer.equals(node.version()))
-                    node.version(locNodeVer);
-
-                Collection<ClusterNode> top = updateTopologyHistory(topVer);
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node add finished message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_JOINED, topVer, node, top);
-
-                stats.onNodeJoined();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeLeftMessage(TcpDiscoveryNodeLeftMessage msg) {
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                if (log.isDebugEnabled())
-                    log.debug("Received node left message for local node: " + msg);
-
-                CountDownLatch leaveLatch0 = leaveLatch;
-
-                assert leaveLatch0 != null;
-
-                leaveLatch0.countDown();
-            }
-            else {
-                if (leaveLatch != null)
-                    return;
-
-                TcpDiscoveryNode node = rmtNodes.remove(msg.creatorNodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node left message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                Collection<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node left message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_LEFT, msg.topologyVersion(), node, top);
-
-                stats.onNodeLeft();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) {
-            if (leaveLatch != null)
-                return;
-
-            if (!getLocalNodeId().equals(msg.creatorNodeId())) {
-                TcpDiscoveryNode node = rmtNodes.remove(msg.failedNodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node failed message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                Collection<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node failed message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
-
-                stats.onNodeFailed();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) {
-            if (leaveLatch != null)
-                return;
-
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                if (msg.senderNodeId() == null) {
-                    Socket sock0 = sock;
-
-                    if (sock0 != null) {
-                        UUID nodeId = ignite.configuration().getNodeId();
-
-                        msg.setMetrics(nodeId, metricsProvider.metrics());
-
-                        msg.setCacheMetrics(nodeId, metricsProvider.cacheMetrics());
-
-                        try {
-                            writeToSocket(sock0, msg);
-
-                            if (log.isDebugEnabled())
-                                log.debug("Heartbeat message sent [sock=" + sock0 + ", msg=" + msg + ']');
-                        }
-                        catch (IOException | IgniteCheckedException e) {
-                            if (log.isDebugEnabled())
-                                U.error(log, "Failed to send heartbeat message [sock=" + sock0 +
-                                    ", msg=" + msg + ']', e);
-
-                            U.closeQuiet(sock0);
-
-                            sock = null;
-
-                            interrupt();
-                        }
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Failed to send heartbeat message (node is disconnected): " + msg);
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("Received heartbeat response: " + msg);
-            }
-            else {
-                long tstamp = U.currentTimeMillis();
-
-                if (msg.hasMetrics()) {
-                    for (Map.Entry<UUID, MetricsSet> e : msg.metrics().entrySet()) {
-                        UUID nodeId = e.getKey();
-
-                        MetricsSet metricsSet = e.getValue();
-
-                        Map<Integer, CacheMetrics> cacheMetrics = msg.hasCacheMetrics() ?
-                                msg.cacheMetrics().get(nodeId) : Collections.<Integer, CacheMetrics>emptyMap();
-
-                        updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp);
-
-                        for (T2<UUID, ClusterMetrics> t : metricsSet.clientMetrics())
-                            updateMetrics(t.get1(), t.get2(), cacheMetrics, tstamp);
-                    }
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) {
-            if (leaveLatch != null)
-                return;
-
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                if (msg.success()) {
-                    pending = true;
-
-                    try {
-                        for (TcpDiscoveryAbstractMessage pendingMsg : msg.pendingMessages())
-                            processMessage(pendingMsg);
-                    }
-                    finally {
-                        pending = false;
-                    }
-
-                    joinErr = null;
-                    reconFailed = false;
-
-                    joinLatch.countDown();
-                }
-                else {
-                    joinErr = null;
-                    reconFailed = true;
-
-                    getSpiContext().recordEvent(new DiscoveryEvent(locNode,
-                        "Client node disconnected: " + locNode,
-                        EVT_CLIENT_NODE_DISCONNECTED, locNode));
-
-                    joinLatch.countDown();
-                }
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Discarding reconnect message for another client: " + msg);
-        }
-
-        /**
-         * @param nodeId Node ID.
-         * @param metrics Metrics.
-         * @param cacheMetrics Cache metrics.
-         * @param tstamp Timestamp.
-         */
-        private void updateMetrics(UUID nodeId,
-            ClusterMetrics metrics,
-            Map<Integer, CacheMetrics> cacheMetrics,
-            long tstamp)
-        {
-            assert nodeId != null;
-            assert metrics != null;
-            assert cacheMetrics != null;
-
-            TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
-
-            if (node != null && node.visible()) {
-                node.setMetrics(metrics);
-                node.setCacheMetrics(cacheMetrics);
-
-                node.lastUpdateTime(tstamp);
-
-                notifyDiscovery(EVT_NODE_METRICS_UPDATED, topVer, node, allNodes());
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Received metrics from unknown node: " + nodeId);
-        }
-
-        /**
-         * @param topVer New topology version.
-         * @return Latest topology snapshot.
-         */
-        private Collection<ClusterNode> updateTopologyHistory(long topVer) {
-            TcpClientDiscoverySpi.this.topVer = topVer;
-
-            Collection<ClusterNode> allNodes = allNodes();
-
-            if (!topHist.containsKey(topVer)) {
-                assert topHist.isEmpty() || topHist.lastKey() == topVer - 1 :
-                    "lastVer=" + topHist.lastKey() + ", newVer=" + topVer;
-
-                topHist.put(topVer, allNodes);
-
-                if (topHist.size() > topHistSize)
-                    topHist.pollFirstEntry();
-
-                assert topHist.lastKey() == topVer;
-                assert topHist.size() <= topHistSize;
-            }
-
-            return allNodes;
-        }
-
-        /**
-         * @return All nodes.
-         */
-        private Collection<ClusterNode> allNodes() {
-            Collection<ClusterNode> allNodes = new TreeSet<>();
-
-            for (TcpDiscoveryNode node : rmtNodes.values()) {
-                if (node.visible())
-                    allNodes.add(node);
-            }
-
-            allNodes.add(locNode);
-
-            return allNodes;
-        }
-
-        /**
-         * @param type Event type.
-         * @param topVer Topology version.
-         * @param node Node.
-         * @param top Topology snapshot.
-         */
-        private void notifyDiscovery(int type, long topVer, ClusterNode node, Collection<ClusterNode> top) {
-            DiscoverySpiListener lsnr = TcpClientDiscoverySpi.this.lsnr;
-
-            if (lsnr != null) {
-                if (log.isDebugEnabled())
-                    log.debug("Discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
-                        ", topVer=" + topVer + ']');
-
-                lsnr.onDiscovery(type, topVer, node, top, new TreeMap<>(topHist), null);
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Skipped discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
-                    ", topVer=" + topVer + ']');
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
deleted file mode 100644
index 9fe4adc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.spi.discovery.tcp;
-
-import org.apache.ignite.mxbean.*;
-import org.apache.ignite.spi.*;
-
-import java.util.*;
-
-/**
- * Management bean for {@link TcpClientDiscoverySpi}.
- */
-public interface TcpClientDiscoverySpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets disconnect check interval.
-     *
-     * @return Disconnect check interval.
-     */
-    @MXBeanDescription("Disconnect check interval.")
-    public long getDisconnectCheckInterval();
-
-    /**
-     * Gets socket timeout.
-     *
-     * @return Socket timeout.
-     */
-    @MXBeanDescription("Socket timeout.")
-    public long getSocketTimeout();
-
-    /**
-     * Gets message acknowledgement timeout.
-     *
-     * @return Message acknowledgement timeout.
-     */
-    @MXBeanDescription("Message acknowledgement timeout.")
-    public long getAckTimeout();
-
-    /**
-     * Gets network timeout.
-     *
-     * @return Network timeout.
-     */
-    @MXBeanDescription("Network timeout.")
-    public long getNetworkTimeout();
-
-    /**
-     * Gets thread priority. All threads within SPI will be started with it.
-     *
-     * @return Thread priority.
-     */
-    @MXBeanDescription("Threads priority.")
-    public int getThreadPriority();
-
-    /**
-     * Gets delay between heartbeat messages sent by coordinator.
-     *
-     * @return Time period in milliseconds.
-     */
-    @MXBeanDescription("Heartbeat frequency.")
-    public long getHeartbeatFrequency();
-
-    /**
-     * Gets {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} (string representation).
-     *
-     * @return IPFinder (string representation).
-     */
-    @MXBeanDescription("IP Finder.")
-    public String getIpFinderFormatted();
-
-    /**
-     * Gets message worker queue current size.
-     *
-     * @return Message worker queue current size.
-     */
-    @MXBeanDescription("Message worker queue current size.")
-    public int getMessageWorkerQueueSize();
-
-    /**
-     * Gets joined nodes count.
-     *
-     * @return Nodes joined count.
-     */
-    @MXBeanDescription("Nodes joined count.")
-    public long getNodesJoined();
-
-    /**
-     * Gets left nodes count.
-     *
-     * @return Left nodes count.
-     */
-    @MXBeanDescription("Nodes left count.")
-    public long getNodesLeft();
-
-    /**
-     * Gets failed nodes count.
-     *
-     * @return Failed nodes count.
-     */
-    @MXBeanDescription("Nodes failed count.")
-    public long getNodesFailed();
-
-    /**
-     * Gets avg message processing time.
-     *
-     * @return Avg message processing time.
-     */
-    @MXBeanDescription("Avg message processing time.")
-    public long getAvgMessageProcessingTime();
-
-    /**
-     * Gets max message processing time.
-     *
-     * @return Max message processing time.
-     */
-    @MXBeanDescription("Max message processing time.")
-    public long getMaxMessageProcessingTime();
-
-    /**
-     * Gets total received messages count.
-     *
-     * @return Total received messages count.
-     */
-    @MXBeanDescription("Total received messages count.")
-    public int getTotalReceivedMessages();
-
-    /**
-     * Gets received messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    @MXBeanDescription("Received messages by type.")
-    public Map<String, Integer> getReceivedMessages();
-
-    /**
-     * Gets total processed messages count.
-     *
-     * @return Total processed messages count.
-     */
-    @MXBeanDescription("Total processed messages count.")
-    public int getTotalProcessedMessages();
-
-    /**
-     * Gets processed messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    @MXBeanDescription("Received messages by type.")
-    public Map<String, Integer> getProcessedMessages();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1652fd18/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
new file mode 100644
index 0000000..b7e9e53
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -0,0 +1,212 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+abstract class TcpDiscoveryImpl {
+    /** Response OK. */
+    protected static final int RES_OK = 1;
+
+    /** Response CONTINUE JOIN. */
+    protected static final int RES_CONTINUE_JOIN = 100;
+
+    /** Response WAIT. */
+    protected static final int RES_WAIT = 200;
+
+    /** */
+    protected final TcpDiscoverySpi spi;
+
+    /** */
+    protected final IgniteLogger log;
+
+    /** */
+    protected TcpDiscoveryNode locNode;
+
+    /**
+     * @param spi Adapter.
+     */
+    TcpDiscoveryImpl(TcpDiscoverySpi spi) {
+        this.spi = spi;
+
+        log = spi.log;
+    }
+
+    /**
+     * @return Local node ID.
+     */
+    public UUID getLocalNodeId() {
+        return spi.getLocalNodeId();
+    }
+
+    /**
+     * @param msg Error message.
+     * @param e Exception.
+     */
+    protected void onException(String msg, Exception e){
+        spi.getExceptionRegistry().onException(msg, e);
+    }
+
+    /**
+     * @param log Logger.
+     */
+    public abstract void dumpDebugInfo(IgniteLogger log);
+
+    /**
+     * @return SPI state string.
+     */
+    public abstract String getSpiState();
+
+    /**
+     * @return Message worker queue current size.
+     */
+    public abstract int getMessageWorkerQueueSize();
+
+    /**
+     * @return Coordinator ID.
+     */
+    public abstract UUID getCoordinator();
+
+    /**
+     * @return Collection of remote nodes.
+     */
+    public abstract Collection<ClusterNode> getRemoteNodes();
+
+    /**
+     * @param nodeId Node id.
+     * @return Node with given ID or {@code null} if node is not found.
+     */
+    @Nullable public abstract ClusterNode getNode(UUID nodeId);
+
+    /**
+     * @param nodeId Node id.
+     * @return {@code true} if node alive, {@code false} otherwise.
+     */
+    public abstract boolean pingNode(UUID nodeId);
+
+    /**
+     * Tells discovery SPI to disconnect from topology.
+     *
+     * @throws IgniteSpiException If failed.
+     */
+    public abstract void disconnect() throws IgniteSpiException;
+
+    /**
+     * @param msg Message.
+     * @throws IgniteException If failed.
+     */
+    public abstract void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;
+
+    /**
+     * @param nodeId Node id.
+     */
+    public abstract void failNode(UUID nodeId);
+
+    /**
+     * @param gridName Grid name.
+     * @throws IgniteSpiException If failed.
+     */
+    public abstract void spiStart(@Nullable String gridName) throws IgniteSpiException;
+
+    /**
+     * @throws IgniteSpiException If failed.
+     */
+    public abstract void spiStop() throws IgniteSpiException;
+
+    /**
+     * @param spiCtx Spi context.
+     * @throws IgniteSpiException If failed.
+     */
+    public abstract void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException;
+
+    /**
+     * @param t Thread.
+     * @return Status as string.
+     */
+    protected static String threadStatus(Thread t) {
+        if (t == null)
+            return "N/A";
+
+        return t.isAlive() ? "alive" : "dead";
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * Simulates this node failure by stopping service threads. So, node will become
+     * unresponsive.
+     * <p>
+     * This method is intended for test purposes only.
+     */
+    abstract void simulateNodeFailure();
+
+    /**
+     * FOR TEST PURPOSE ONLY!
+     */
+    public abstract void brakeConnection();
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     *
+     * @return Worker thread.
+     */
+    protected abstract IgniteSpiThread workerThread();
+
+    /**
+     * @throws IgniteSpiException If failed.
+     */
+    @SuppressWarnings("BusyWait")
+    protected final void registerLocalNodeAddress() throws IgniteSpiException {
+        // Make sure address registration succeeded.
+        while (true) {
+            try {
+                spi.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
+
+                // Success.
+                break;
+            }
+            catch (IllegalStateException e) {
+                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
+                    locNode.socketAddresses(), e);
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to register local node address in IP finder on start " +
+                    "(retrying every 2000 ms).");
+            }
+
+            try {
+                U.sleep(2000);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new IgniteSpiException("Thread has been interrupted.", e);
+            }
+        }
+    }
+}