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 2016/12/02 09:25:07 UTC

[01/19] ignite git commit: IGNITE-4249: ODBC: Fixed performance issue caused by ineddicient IO handling on CPP side. This closes #1254.

Repository: ignite
Updated Branches:
  refs/heads/master 88c06ec3c -> 31bc3bf6e


IGNITE-4249: ODBC: Fixed performance issue caused by ineddicient IO handling on CPP side. This closes #1254.


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

Branch: refs/heads/master
Commit: b038730ee56a662f73e02bbec83eb1712180fa82
Parents: 9d82f2c
Author: isapego <ig...@gmail.com>
Authored: Wed Nov 23 12:05:54 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Nov 23 12:05:54 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcRequestHandler.java     | 32 ++++++++++++++------
 .../src/impl/binary/binary_reader_impl.cpp      |  3 +-
 modules/platforms/cpp/odbc/src/connection.cpp   | 21 +++++++------
 3 files changed, 34 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b038730e/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index 4a31be3..eef9945 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
@@ -192,9 +192,7 @@ public class OdbcRequestHandler {
 
             QueryCursor qryCur = cache.query(qry);
 
-            Iterator iter = qryCur.iterator();
-
-            qryCursors.put(qryId, new IgniteBiTuple<>(qryCur, iter));
+            qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
 
             List<?> fieldsMeta = ((QueryCursorImpl) qryCur).fieldsMeta();
 
@@ -220,11 +218,15 @@ public class OdbcRequestHandler {
      */
     private OdbcResponse closeQuery(long reqId, OdbcQueryCloseRequest req) {
         try {
-            QueryCursor cur = qryCursors.get(req.queryId()).get1();
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
 
-            if (cur == null)
+            if (tuple == null)
                 return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Failed to find query with ID: " + req.queryId());
 
+            QueryCursor cur = tuple.get1();
+
+            assert(cur != null);
+
             cur.close();
 
             qryCursors.remove(req.queryId());
@@ -251,17 +253,27 @@ public class OdbcRequestHandler {
      */
     private OdbcResponse fetchQuery(long reqId, OdbcQueryFetchRequest req) {
         try {
-            Iterator cur = qryCursors.get(req.queryId()).get2();
+            IgniteBiTuple<QueryCursor, Iterator> tuple = qryCursors.get(req.queryId());
 
-            if (cur == null)
+            if (tuple == null)
                 return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Failed to find query with ID: " + req.queryId());
 
+            Iterator iter = tuple.get2();
+
+            if (iter == null) {
+                QueryCursor cur = tuple.get1();
+
+                iter = cur.iterator();
+
+                tuple.put(cur, iter);
+            }
+
             List<Object> items = new ArrayList<>();
 
-            for (int i = 0; i < req.pageSize() && cur.hasNext(); ++i)
-                items.add(cur.next());
+            for (int i = 0; i < req.pageSize() && iter.hasNext(); ++i)
+                items.add(iter.next());
 
-            OdbcQueryFetchResult res = new OdbcQueryFetchResult(req.queryId(), items, !cur.hasNext());
+            OdbcQueryFetchResult res = new OdbcQueryFetchResult(req.queryId(), items, !iter.hasNext());
 
             return new OdbcResponse(res);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b038730e/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
index c3f4fcc..fb75ba5 100644
--- a/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
+++ b/modules/platforms/cpp/binary/src/impl/binary/binary_reader_impl.cpp
@@ -459,8 +459,7 @@ namespace ignite
                     int32_t realLen = stream->ReadInt32();
 
                     if (res && len >= realLen) {
-                        for (int i = 0; i < realLen; i++)
-                            *(res + i) = static_cast<char>(stream->ReadInt8());
+                        stream->ReadInt8Array(reinterpret_cast<int8_t*>(res), realLen);
 
                         if (len > realLen)
                             *(res + realLen) = 0; // Set NULL terminator if possible.

http://git-wip-us.apache.org/repos/asf/ignite/blob/b038730e/modules/platforms/cpp/odbc/src/connection.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/connection.cpp b/modules/platforms/cpp/odbc/src/connection.cpp
index b8ed9fe..e8db376 100644
--- a/modules/platforms/cpp/odbc/src/connection.cpp
+++ b/modules/platforms/cpp/odbc/src/connection.cpp
@@ -19,6 +19,8 @@
 
 #include <sstream>
 
+#include <ignite/common/fixed_size_array.h>
+
 #include "ignite/odbc/utility.h"
 #include "ignite/odbc/statement.h"
 #include "ignite/odbc/connection.h"
@@ -178,26 +180,25 @@ namespace ignite
             if (!connected)
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_ILLEGAL_STATE, "Connection is not established");
 
-            OdbcProtocolHeader hdr;
+            common::FixedSizeArray<int8_t> msg(len + sizeof(OdbcProtocolHeader));
 
-            hdr.len = static_cast<int32_t>(len);
+            OdbcProtocolHeader *hdr = reinterpret_cast<OdbcProtocolHeader*>(msg.GetData());
 
-            size_t sent = SendAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
+            hdr->len = static_cast<int32_t>(len);
 
-            if (sent != sizeof(hdr))
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message header");
+            memcpy(msg.GetData() + sizeof(OdbcProtocolHeader), data, len);
 
-            sent = SendAll(data, len);
+            size_t sent = SendAll(msg.GetData(), msg.GetSize());
 
-            if (sent != len)
-                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message body");
+            if (sent != len + sizeof(OdbcProtocolHeader))
+                IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not send message");
         }
 
         size_t Connection::SendAll(const int8_t* data, size_t len)
         {
             int sent = 0;
 
-            while (sent != len)
+            while (sent != static_cast<int64_t>(len))
             {
                 int res = socket.Send(data + sent, len - sent);
 
@@ -221,7 +222,7 @@ namespace ignite
 
             OdbcProtocolHeader hdr;
 
-            size_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
+            int64_t received = ReceiveAll(reinterpret_cast<int8_t*>(&hdr), sizeof(hdr));
 
             if (received != sizeof(hdr))
                 IGNITE_ERROR_1(IgniteError::IGNITE_ERR_GENERIC, "Can not receive message header");


[02/19] ignite git commit: IGNITE-4270: Allow GridUnsafe.UNALIGNED flag override.

Posted by sb...@apache.org.
IGNITE-4270: Allow GridUnsafe.UNALIGNED flag override.


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

Branch: refs/heads/master
Commit: 7a47a0185d308cd3a58c7bfcb4d1cd548bff5b87
Parents: b038730
Author: devozerov <vo...@gridgain.com>
Authored: Thu Nov 24 11:14:08 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Nov 24 11:14:08 2016 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/IgniteSystemProperties.java | 7 +++++++
 .../java/org/apache/ignite/internal/util/GridUnsafe.java    | 9 ++++++++-
 2 files changed, 15 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a47a018/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 22cb9a6..043c95a 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -487,6 +487,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_BINARY_SORT_OBJECT_FIELDS = "IGNITE_BINARY_SORT_OBJECT_FIELDS";
 
     /**
+     * Whether Ignite can access unaligned memory addresses.
+     * <p>
+     * Defaults to {@code} false, meaning that unaligned access will be performed only on x86 architecture.
+     */
+    public static final String IGNITE_UNALIGNED_MEMORY_ACCESS = "IGNITE_UNALIGNED_MEMORY_ACCESS";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/7a47a018/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
index 1f7a53c..1c492ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUnsafe.java
@@ -22,6 +22,8 @@ import java.nio.ByteOrder;
 import java.security.AccessController;
 import java.security.PrivilegedActionException;
 import java.security.PrivilegedExceptionAction;
+
+import org.apache.ignite.IgniteSystemProperties;
 import sun.misc.Unsafe;
 
 /**
@@ -1204,7 +1206,12 @@ public abstract class GridUnsafe {
     private static boolean unaligned() {
         String arch = System.getProperty("os.arch");
 
-        return arch.equals("i386") || arch.equals("x86") || arch.equals("amd64") || arch.equals("x86_64");
+        boolean res = arch.equals("i386") || arch.equals("x86") || arch.equals("amd64") || arch.equals("x86_64");
+
+        if (!res)
+            res = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_UNALIGNED_MEMORY_ACCESS, false);
+
+        return res;
     }
 
     /**


[08/19] ignite git commit: ignite-4088 Added methods to create/destroy multiple caches. This closes #1174.

Posted by sb...@apache.org.
ignite-4088 Added methods to create/destroy multiple caches. This closes #1174.

(cherry picked from commit f445e7b)


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

Branch: refs/heads/master
Commit: 56998e704e9a67760c70481c10c56e72c0a866bb
Parents: 9c6824b
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Fri Oct 28 16:27:34 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Nov 24 18:00:03 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  77 ++++-
 .../apache/ignite/internal/IgniteKernal.java    |  81 ++++++
 .../processors/cache/GridCacheProcessor.java    | 291 ++++++++++++++-----
 .../cache/IgniteDynamicCacheStartSelfTest.java  | 217 ++++++++++++--
 .../processors/igfs/IgfsIgniteMock.java         |  19 ++
 .../ignite/testframework/junits/IgniteMock.java |  15 +
 .../junits/multijvm/IgniteProcessProxy.java     |  15 +
 .../org/apache/ignite/IgniteSpringBean.java     |  21 ++
 8 files changed, 616 insertions(+), 120 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index bd21468..0de08d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -20,6 +20,7 @@ package org.apache.ignite;
 import java.util.Collection;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
+import javax.cache.CacheException;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.affinity.Affinity;
 import org.apache.ignite.cluster.ClusterGroup;
@@ -220,8 +221,24 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheCfg Cache configuration to use.
      * @return Instance of started cache.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      */
-    public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg) throws CacheException;
+
+    /**
+     * Dynamically starts new caches with the given cache configurations.
+     * <p>
+     * If local node is an affinity node, this method will return the instance of started caches.
+     * Otherwise, it will create a client caches on local node.
+     * <p>
+     * If for one of configurations a cache with the same name already exists in the grid, an exception will be thrown regardless
+     * whether the given configuration matches the configuration of the existing cache or not.
+     *
+     * @param cacheCfgs Collection of cache configuration to use.
+     * @return Collection of instances of started caches.
+     * @throws CacheException If one of created caches exists or other error occurs.
+     */
+    public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) throws CacheException;
 
     /**
      * Dynamically starts new cache using template configuration.
@@ -233,8 +250,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheName Cache name.
      * @return Instance of started cache.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      */
-    public <K, V> IgniteCache<K, V> createCache(String cacheName);
+    public <K, V> IgniteCache<K, V> createCache(String cacheName) throws CacheException;
 
     /**
      * Gets existing cache with the given name or creates new one with the given configuration.
@@ -245,23 +263,39 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheCfg Cache configuration to use.
      * @return Existing or newly created cache.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) throws CacheException;
 
     /**
      * Gets existing cache with the given name or creates new one using template configuration.
      *
      * @param cacheName Cache name.
      * @return Existing or newly created cache.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName);
+    public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) throws CacheException;
+
+    /**
+     * Gets existing caches with the given name or created one with the given configuration.
+     * <p>
+     * If a cache with the same name already exist, this method will not check that the given
+     * configuration matches the configuration of existing cache and will return an instance
+     * of the existing cache.
+     *
+     * @param cacheCfgs Collection of cache configuration to use.
+     * @return Collection of existing or newly created caches.
+     * @throws CacheException If error occurs.
+     */
+    public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) throws CacheException;
 
     /**
      * Adds cache configuration template.
      *
      * @param cacheCfg Cache configuration template.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg);
+    public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) throws CacheException;
 
     /**
      * Dynamically starts new cache with the given cache configuration.
@@ -275,10 +309,11 @@ public interface Ignite extends AutoCloseable {
      * @param cacheCfg Cache configuration to use.
      * @param nearCfg Near cache configuration to use on local node in case it is not an
      *      affinity node.
+     * @throws CacheException If a cache with the same name already exists or other error occurs.
      * @return Instance of started cache.
      */
     public <K, V> IgniteCache<K, V> createCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg);
+        NearCacheConfiguration<K, V> nearCfg) throws CacheException;
 
     /**
      * Gets existing cache with the given cache configuration or creates one if it does not exist.
@@ -293,9 +328,10 @@ public interface Ignite extends AutoCloseable {
      * @param cacheCfg Cache configuration.
      * @param nearCfg Near cache configuration for client.
      * @return {@code IgniteCache} instance.
+     * @throws CacheException If error occurs.
      */
     public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg,
-        NearCacheConfiguration<K, V> nearCfg);
+        NearCacheConfiguration<K, V> nearCfg) throws CacheException;
 
     /**
      * Starts a near cache on local node if cache was previously started with one of the
@@ -305,8 +341,10 @@ public interface Ignite extends AutoCloseable {
      * @param cacheName Cache name.
      * @param nearCfg Near cache configuration.
      * @return Cache instance.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg);
+    public <K, V> IgniteCache<K, V> createNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg)
+        throws CacheException;
 
     /**
      * Gets existing near cache with the given name or creates a new one.
@@ -314,15 +352,26 @@ public interface Ignite extends AutoCloseable {
      * @param cacheName Cache name.
      * @param nearCfg Near configuration.
      * @return {@code IgniteCache} instance.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg);
+    public <K, V> IgniteCache<K, V> getOrCreateNearCache(@Nullable String cacheName, NearCacheConfiguration<K, V> nearCfg)
+        throws CacheException;
 
     /**
      * Stops dynamically started cache.
      *
      * @param cacheName Cache name to stop.
+     * @throws CacheException If error occurs.
+     */
+    public void destroyCache(String cacheName) throws CacheException;
+
+    /**
+     * Stops dynamically started caches.
+     *
+     * @param cacheNames Collection of cache names to stop.
+     * @throws CacheException If error occurs.
      */
-    public void destroyCache(String cacheName);
+    public void destroyCaches(Collection<String> cacheNames) throws CacheException;
 
     /**
      * Gets an instance of {@link IgniteCache} API. {@code IgniteCache} is a fully-compatible
@@ -330,8 +379,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param name Cache name.
      * @return Instance of the cache for the specified name.
+     * @throws CacheException If error occurs.
      */
-    public <K, V> IgniteCache<K, V> cache(@Nullable String name);
+    public <K, V> IgniteCache<K, V> cache(@Nullable String name) throws CacheException;
 
     /**
      * Gets the collection of names of currently available caches.
@@ -357,8 +407,9 @@ public interface Ignite extends AutoCloseable {
      *
      * @param cacheName Cache name ({@code null} for default cache).
      * @return Data streamer.
+     * @throws IllegalStateException If node is stopping.
      */
-    public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName);
+    public <K, V> IgniteDataStreamer<K, V> dataStreamer(@Nullable String cacheName) throws IllegalStateException;
 
     /**
      * Gets an instance of IGFS (Ignite In-Memory File System). If one is not
@@ -372,7 +423,7 @@ public interface Ignite extends AutoCloseable {
      * @return IGFS instance.
      * @throws IllegalArgumentException If IGFS with such name is not configured.
      */
-    public IgniteFileSystem fileSystem(String name);
+    public IgniteFileSystem fileSystem(String name) throws IllegalArgumentException;
 
     /**
      * Gets all instances of IGFS (Ignite In-Memory File System).

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/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 ef9c651..c536506 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
@@ -2541,6 +2541,33 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+
+    /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        A.notNull(cacheCfgs, "cacheCfgs");
+
+        guard();
+
+        try {
+            ctx.cache().dynamicStartCaches(cacheCfgs,
+                true,
+                true).get();
+
+            List<IgniteCache> createdCaches = new ArrayList<>(cacheCfgs.size());
+
+            for (CacheConfiguration cacheCfg : cacheCfgs)
+                createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName()));
+
+            return createdCaches;
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+        finally {
+            unguard();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         guard();
@@ -2585,6 +2612,32 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        A.notNull(cacheCfgs, "cacheCfgs");
+
+        guard();
+
+        try {
+            ctx.cache().dynamicStartCaches(cacheCfgs,
+                false,
+                true).get();
+
+            List<IgniteCache> createdCaches = new ArrayList<>(cacheCfgs.size());
+
+            for (CacheConfiguration cacheCfg : cacheCfgs)
+                createdCaches.add(ctx.cache().publicJCache(cacheCfg.getName()));
+
+            return createdCaches;
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+        finally {
+            unguard();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(
         CacheConfiguration<K, V> cacheCfg,
         NearCacheConfiguration<K, V> nearCfg
@@ -2745,6 +2798,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        IgniteInternalFuture stopFut = destroyCachesAsync(cacheNames, true);
+
+        try {
+            stopFut.get();
+        }
+        catch (IgniteCheckedException e) {
+            throw CU.convertToCacheException(e);
+        }
+    }
+
     /**
      * @param cacheName Cache name.
      * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
@@ -2761,6 +2826,22 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param cacheNames Collection of cache names.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Ignite future.
+     */
+    public IgniteInternalFuture<?> destroyCachesAsync(Collection<String> cacheNames, boolean checkThreadTx) {
+        guard();
+
+        try {
+            return ctx.cache().dynamicDestroyCaches(cacheNames, checkThreadTx);
+        }
+        finally {
+            unguard();
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(String cacheName) {
         guard();

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/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 5e777fd..0e0d769 100755
--- 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
@@ -2284,99 +2284,92 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (checkThreadTx)
             checkEmptyTransactions();
 
-        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
-
-        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
-
-        req.failIfExists(failIfExists);
-
-        if (ccfg != null) {
-            try {
-                cloneCheckSerializable(ccfg);
-            }
-            catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(e);
-            }
-
-            if (desc != null) {
-                if (failIfExists) {
-                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start cache " +
-                        "(a cache with the same name is already started): " + cacheName));
-                }
-                else {
-                    CacheConfiguration descCfg = desc.cacheConfiguration();
-
-                    // Check if we were asked to start a near cache.
-                    if (nearCfg != null) {
-                        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<>();
-                            else
-                                return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start near " +
-                                    "cache (local node is an affinity node for cache): " + cacheName));
-                        }
-                        else
-                            // If local node has near cache, return success.
-                            req.clientStartOnly(true);
-                    }
-                    else
-                        req.clientStartOnly(true);
+        try {
+            DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                ccfg,
+                cacheName,
+                nearCfg,
+                cacheType,
+                failIfExists,
+                failIfNotStarted);
 
-                    req.deploymentId(desc.deploymentId());
+            if (req != null)
+                return F.first(initiateCacheChanges(F.asList(req), failIfExists));
+            else
+                return new GridFinishedFuture<>();
+        }
+        catch (Exception e) {
+            return new GridFinishedFuture<>(e);
+        }
+    }
 
-                    req.startCacheConfiguration(descCfg);
-                }
-            }
-            else {
-                req.deploymentId(IgniteUuid.randomUuid());
+    /**
+     * Dynamically starts multiple caches.
+     *
+     * @param ccfgList Collection of cache configuration.
+     * @param failIfExists Fail if exists flag.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when all caches are deployed.
+     */
+    public IgniteInternalFuture<?> dynamicStartCaches(
+        Collection<CacheConfiguration> ccfgList,
+        boolean failIfExists,
+        boolean checkThreadTx
+    ) {
+        return dynamicStartCaches(ccfgList, CacheType.USER, failIfExists, checkThreadTx);
+    }
 
-                try {
-                    CacheConfiguration cfg = new CacheConfiguration(ccfg);
+    /**
+     * Dynamically starts multiple caches.
+     *
+     * @param ccfgList Collection of cache configuration.
+     * @param cacheType Cache type.
+     * @param failIfExists Fail if exists flag.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when all caches are deployed.
+     */
+    private IgniteInternalFuture<?> dynamicStartCaches(
+        Collection<CacheConfiguration> ccfgList,
+        CacheType cacheType,
+        boolean failIfExists,
+        boolean checkThreadTx
+    ) {
+        if (checkThreadTx)
+            checkEmptyTransactions();
 
-                    CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
+        List<DynamicCacheChangeRequest> reqList = new ArrayList<>(ccfgList.size());
 
-                    initialize(false, cfg, cacheObjCtx);
+        try {
+            for (CacheConfiguration ccfg : ccfgList) {
+                DynamicCacheChangeRequest req = prepareCacheChangeRequest(
+                    ccfg,
+                    ccfg.getName(),
+                    null,
+                    cacheType,
+                    failIfExists,
+                    true
+                );
 
-                    req.startCacheConfiguration(cfg);
-                }
-                catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture(e);
-                }
+                if (req != null)
+                    reqList.add(req);
             }
         }
-        else {
-            req.clientStartOnly(true);
-
-            if (desc != null)
-                ccfg = desc.cacheConfiguration();
-
-            if (ccfg == null) {
-                if (failIfNotStarted)
-                    return new GridFinishedFuture<>(new CacheExistsException("Failed to start client cache " +
-                        "(a cache with the given name is not started): " + cacheName));
-                else
-                    return new GridFinishedFuture<>();
-            }
-
-            req.deploymentId(desc.deploymentId());
-            req.startCacheConfiguration(ccfg);
+        catch (Exception e) {
+            return new GridFinishedFuture<>(e);
         }
 
-        // Fail cache with swap enabled creation on grid without swap space SPI.
-        if (ccfg.isSwapEnabled())
-            for (ClusterNode n : ctx.discovery().allNodes())
-                if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n))
-                    return new GridFinishedFuture<>(new IgniteCheckedException("Failed to start cache " +
-                        cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() +
-                        " has not swap SPI configured"));
+        if (!reqList.isEmpty()) {
+            GridCompoundFuture<?, ?> compoundFut = new GridCompoundFuture<>();
 
-        if (nearCfg != null)
-            req.nearCacheConfiguration(nearCfg);
+            for (DynamicCacheStartFuture fut : initiateCacheChanges(reqList, failIfExists))
+                compoundFut.add((IgniteInternalFuture)fut);
 
-        req.cacheType(cacheType);
+            compoundFut.markInitialized();
 
-        return F.first(initiateCacheChanges(F.asList(req), failIfExists));
+            return compoundFut;
+        }
+        else
+            return new GridFinishedFuture<>();
     }
 
     /**
@@ -2396,6 +2389,35 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param cacheNames Collection of cache names to destroy.
+     * @param checkThreadTx If {@code true} checks that current thread does not have active transactions.
+     * @return Future that will be completed when cache is destroyed.
+     */
+    public IgniteInternalFuture<?> dynamicDestroyCaches(Collection<String> cacheNames, boolean checkThreadTx) {
+        if (checkThreadTx)
+            checkEmptyTransactions();
+
+        List<DynamicCacheChangeRequest> reqs = new ArrayList<>(cacheNames.size());
+
+        for (String cacheName : cacheNames) {
+            DynamicCacheChangeRequest t = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
+
+            t.stop(true);
+
+            reqs.add(t);
+        }
+
+        GridCompoundFuture<?, ?> compoundFut = new GridCompoundFuture<>();
+
+        for (DynamicCacheStartFuture fut : initiateCacheChanges(reqs, false))
+            compoundFut.add((IgniteInternalFuture)fut);
+
+        compoundFut.markInitialized();
+
+        return compoundFut;
+    }
+
+    /**
      * @param cacheName Cache name to close.
      * @return Future that will be completed when cache is closed.
      */
@@ -2416,6 +2438,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
     /**
      * @param reqs Requests.
+     * @param failIfExists Fail if exists flag.
      * @return Collection of futures.
      */
     @SuppressWarnings("TypeMayBeWeakened")
@@ -3608,6 +3631,114 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Prepares DynamicCacheChangeRequest for cache creation.
+     *
+     * @param ccfg Cache configuration
+     * @param cacheName Cache name
+     * @param nearCfg Near cache configuration
+     * @param cacheType Cache type
+     * @param failIfExists Fail if exists flag.
+     * @param failIfNotStarted If {@code true} fails if cache is not started.
+     * @return Request or {@code null} if cache already exists.
+     * @throws IgniteCheckedException if some of pre-checks failed
+     * @throws CacheExistsException if cache exists and failIfExists flag is {@code true}
+     */
+    private DynamicCacheChangeRequest prepareCacheChangeRequest(
+        @Nullable CacheConfiguration ccfg,
+        String cacheName,
+        @Nullable NearCacheConfiguration nearCfg,
+        CacheType cacheType,
+        boolean failIfExists,
+        boolean failIfNotStarted
+    ) throws IgniteCheckedException {
+        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(cacheName));
+
+        DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(cacheName, ctx.localNodeId());
+
+        req.failIfExists(failIfExists);
+
+        if (ccfg != null) {
+            cloneCheckSerializable(ccfg);
+
+            if (desc != null) {
+                if (failIfExists) {
+                    throw new CacheExistsException("Failed to start cache " +
+                        "(a cache with the same name is already started): " + cacheName);
+                }
+                else {
+                    CacheConfiguration descCfg = desc.cacheConfiguration();
+
+                    // Check if we were asked to start a near cache.
+                    if (nearCfg != null) {
+                        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 null;
+                            else
+                                throw new IgniteCheckedException("Failed to start near " +
+                                    "cache (local node is an affinity node for cache): " + cacheName);
+                        }
+                        else
+                            // If local node has near cache, return success.
+                            req.clientStartOnly(true);
+                    }
+                    else
+                        req.clientStartOnly(true);
+
+                    req.deploymentId(desc.deploymentId());
+
+                    req.startCacheConfiguration(descCfg);
+                }
+            }
+            else {
+                req.deploymentId(IgniteUuid.randomUuid());
+
+                CacheConfiguration cfg = new CacheConfiguration(ccfg);
+
+                CacheObjectContext cacheObjCtx = ctx.cacheObjects().contextForCache(cfg);
+
+                initialize(false, cfg, cacheObjCtx);
+
+                req.startCacheConfiguration(cfg);
+            }
+        }
+        else {
+            req.clientStartOnly(true);
+
+            if (desc != null)
+                ccfg = desc.cacheConfiguration();
+
+            if (ccfg == null) {
+                if (failIfNotStarted) {
+                    throw new CacheExistsException("Failed to start client cache " +
+                        "(a cache with the given name is not started): " + cacheName);
+                }
+                else
+                    return null;
+            }
+
+            req.deploymentId(desc.deploymentId());
+            req.startCacheConfiguration(ccfg);
+        }
+
+        // Fail cache with swap enabled creation on grid without swap space SPI.
+        if (ccfg.isSwapEnabled())
+            for (ClusterNode n : ctx.discovery().allNodes())
+                if (!GridCacheUtils.clientNode(n) && !GridCacheUtils.isSwapEnabled(n)) {
+                    throw new IgniteCheckedException("Failed to start cache " +
+                        cacheName + " with swap enabled: Remote Node with ID " + n.id().toString().toUpperCase() +
+                        " has not swap SPI configured");
+                }
+
+        if (nearCfg != null)
+            req.nearCacheConfiguration(nearCfg);
+
+        req.cacheType(cacheType);
+
+        return req;
+    }
+
+    /**
      * @param obj Object to clone.
      * @return Object copy.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/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 c9cd750..48e06ee 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
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.CountDownLatch;
@@ -181,7 +183,8 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 info("Succeeded: " + System.identityHashCode(fut));
 
                 succeeded++;
-            } catch (IgniteCheckedException e) {
+            }
+            catch (IgniteCheckedException e) {
                 info(e.getMessage());
 
                 failed++;
@@ -246,7 +249,8 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 info("Succeeded: " + System.identityHashCode(fut));
 
                 succeeded++;
-            } catch (IgniteCheckedException e) {
+            }
+            catch (IgniteCheckedException e) {
                 info(e.getMessage());
 
                 failed++;
@@ -289,6 +293,20 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopCachesSimpleTransactional() throws Exception {
+        checkStartStopCachesSimple(CacheAtomicityMode.TRANSACTIONAL);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStopCachesSimpleAtomic() throws Exception {
+        checkStartStopCachesSimple(CacheAtomicityMode.ATOMIC);
+    }
+
+    /**
      * @param mode Cache atomicity mode.
      * @throws Exception If failed.
      */
@@ -325,10 +343,10 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         for (int g = 0; g < nodeCount(); g++)
             caches[g] = grid(g).cache(DYNAMIC_CACHE_NAME);
 
-        kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+        kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
         for (int g = 0; g < nodeCount(); g++) {
-            final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+            final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
             final int idx = g;
 
@@ -346,6 +364,87 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param mode Cache atomicity mode.
+     * @throws Exception If failed.
+     */
+    private void checkStartStopCachesSimple(CacheAtomicityMode mode) throws Exception {
+        final IgniteEx kernal = grid(0);
+        final int cacheCnt = 3;
+
+        List<CacheConfiguration> ccfgList = new ArrayList<>();
+
+        for (int i = 0; i < cacheCnt; i++) {
+            CacheConfiguration ccfg = new CacheConfiguration();
+            ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+            ccfg.setAtomicityMode(mode);
+            ccfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+            ccfgList.add(ccfg);
+        }
+
+        kernal.createCaches(ccfgList);
+
+        for (int g = 0; g < nodeCount(); g++) {
+            IgniteEx kernal0 = grid(g);
+
+            for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
+                f.get();
+
+            info("Getting cache for node: " + g);
+
+            for (int i = 0; i < cacheCnt; i++)
+                assertNotNull(grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)));
+        }
+
+        for (int i = 0; i < cacheCnt; i++)
+            grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).put(Integer.toString(i), Integer.toString(i));
+
+        for (int g = 0; g < nodeCount(); g++) {
+            for (int i = 0; i < cacheCnt; i++) {
+                assertEquals(
+                    Integer.toString(i),
+                    grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)).get(Integer.toString(i))
+                );
+            }
+        }
+
+        // Grab caches before stop.
+        final IgniteCache[] caches = new IgniteCache[nodeCount() * cacheCnt];
+
+        for (int g = 0; g < nodeCount(); g++) {
+            for (int i = 0; i < cacheCnt; i++)
+                caches[g * nodeCount() + i] = grid(g).cache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+        }
+
+        List<String> namesToDestroy = new ArrayList<>();
+
+        for (int i = 0; i < cacheCnt; i++)
+            namesToDestroy.add(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+        kernal.destroyCaches(namesToDestroy);
+
+        for (int g = 0; g < nodeCount(); g++) {
+            final IgniteKernal kernal0 = (IgniteKernal)grid(g);
+
+            for (int i = 0; i < cacheCnt; i++) {
+                final int idx = g * nodeCount() + i;
+                final int expVal = i;
+
+                for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
+                    f.get();
+
+                assertNull(kernal0.cache(DYNAMIC_CACHE_NAME));
+
+                GridTestUtils.assertThrows(log, new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        return caches[idx].get(Integer.toString(expVal));
+                    }
+                }, IllegalStateException.class, null);
+            }
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testStartStopCacheAddNode() throws Exception {
@@ -378,13 +477,13 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             }
 
             // Undeploy cache.
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
             startGrid(nodeCount() + 1);
 
             // Check that cache is not deployed on new node after undeploy.
             for (int g = 0; g < nodeCount() + 2; g++) {
-                final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+                final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
@@ -431,16 +530,16 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount(); g++) {
                 for (int i = 0; i < 100; i++) {
                     assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i)
-                            .contains(grid(nodeCount()).cluster().localNode()));
+                        .contains(grid(nodeCount()).cluster().localNode()));
 
                     assertFalse(grid(g).affinity(DYNAMIC_CACHE_NAME).mapKeyToPrimaryAndBackups(i)
-                            .contains(grid(nodeCount() + 1).cluster().localNode()));
+                        .contains(grid(nodeCount() + 1).cluster().localNode()));
                 }
             }
 
             // Check that cache is not deployed on new node after undeploy.
             for (int g = 0; g < nodeCount() + 2; g++) {
-                final IgniteKernal kernal0 = (IgniteKernal) grid(g);
+                final IgniteKernal kernal0 = (IgniteKernal)grid(g);
 
                 for (IgniteInternalFuture f : kernal0.context().cache().context().exchange().exchangeFutures())
                     f.get();
@@ -455,7 +554,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                     }, IllegalArgumentException.class, null);
             }
 
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
 
             stopGrid(nodeCount() + 1);
             stopGrid(nodeCount());
@@ -489,6 +588,36 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testFailWhenOneOfConfiguredCacheExists() throws Exception {
+        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                final Ignite kernal = grid(0);
+
+                CacheConfiguration ccfgDynamic = new CacheConfiguration();
+                ccfgDynamic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+                ccfgDynamic.setName(DYNAMIC_CACHE_NAME);
+
+                ccfgDynamic.setNodeFilter(NODE_FILTER);
+
+                CacheConfiguration ccfgStatic = new CacheConfiguration();
+                ccfgStatic.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+
+                // Cache is already configured, should fail.
+                ccfgStatic.setName(STATIC_CACHE_NAME);
+
+                ccfgStatic.setNodeFilter(NODE_FILTER);
+
+                return kernal.createCaches(F.asList(ccfgDynamic, ccfgStatic));
+            }
+        }, CacheExistsException.class, null);
+
+        assertNull(grid(0).cache(DYNAMIC_CACHE_NAME));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testClientCache() throws Exception {
         try {
             testAttribute = false;
@@ -522,7 +651,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
@@ -547,7 +676,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
 
             ccfg.setNodeFilter(NODE_FILTER);
 
-            final IgniteKernal started = (IgniteKernal) grid(nodeCount());
+            final IgniteKernal started = (IgniteKernal)grid(nodeCount());
 
             started.createCache(ccfg);
 
@@ -564,14 +693,13 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
         }
     }
 
-
     /**
      * @throws Exception If failed.
      */
@@ -610,7 +738,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             for (int g = 0; g < nodeCount() + 1; g++)
                 assertEquals("1", ignite(g).cache(DYNAMIC_CACHE_NAME).get("1"));
 
-            kernal.context().cache().dynamicDestroyCache(DYNAMIC_CACHE_NAME, true).get();
+            kernal.destroyCache(DYNAMIC_CACHE_NAME);
         }
         finally {
             stopGrid(nodeCount());
@@ -760,7 +888,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 nearGrid.getOrCreateNearCache(DYNAMIC_CACHE_NAME, new NearCacheConfiguration());
 
                 GridCacheContext<Object, Object> nCtx = ((IgniteKernal)nearGrid)
-                        .internalCache(DYNAMIC_CACHE_NAME).context();
+                    .internalCache(DYNAMIC_CACHE_NAME).context();
 
                 assertTrue(nCtx.isNear());
                 assertFalse(nCtx.affinityNode());
@@ -771,11 +899,12 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 clientGrid.getOrCreateCache(cfg);
 
                 GridCacheContext<Object, Object> cCtx = ((IgniteKernal)clientGrid)
-                        .internalCache(DYNAMIC_CACHE_NAME).context();
+                    .internalCache(DYNAMIC_CACHE_NAME).context();
 
                 assertFalse(cCtx.isNear());
                 assertFalse(cCtx.affinityNode());
-            } finally {
+            }
+            finally {
                 stopGrid(nodeCount() + 1);
                 stopGrid(nodeCount());
             }
@@ -785,6 +914,40 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         }
     }
 
+    /** {@inheritDoc} */
+    public void testGetOrCreateCollection() throws Exception {
+        final int cacheCnt = 3;
+
+        try {
+            final Collection<CacheConfiguration> ccfgs = new ArrayList<>();
+
+            for (int i = 0; i < cacheCnt; i++) {
+                final CacheConfiguration cfg = new CacheConfiguration();
+
+                cfg.setName(DYNAMIC_CACHE_NAME + Integer.toString(i));
+                cfg.setNodeFilter(NODE_FILTER);
+
+                ccfgs.add(cfg);
+
+                grid(0).getOrCreateCaches(ccfgs);
+            }
+
+            for (int i = 0; i < cacheCnt; i++) {
+                assertNotNull(grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i)));
+
+                IgniteCache<Object, Object> jcache = grid(0).cache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+
+                jcache.put(Integer.toString(i), Integer.toString(i));
+
+                assertEquals(jcache.get(Integer.toString(i)), Integer.toString(i));
+            }
+        }
+        finally {
+            for (int i = 0; i < cacheCnt; i++)
+                grid(0).destroyCache(DYNAMIC_CACHE_NAME + Integer.toString(i));
+        }
+    }
+
     /**
      * @throws Exception If failed.
      */
@@ -813,7 +976,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
             assertNull(err.get());
 
             for (int i = 0; i < nodeCount(); i++) {
-                GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
+                GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
                     .context();
 
                 assertTrue(ctx.affinityNode());
@@ -906,7 +1069,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 assertNull(err.get());
 
                 for (int i = 0; i < nodeCount(); i++) {
-                    GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
+                    GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(i)).internalCache(DYNAMIC_CACHE_NAME)
                         .context();
 
                     assertTrue(ctx.affinityNode());
@@ -914,7 +1077,7 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
                 }
 
                 for (int i = 0; i < clientCnt; i++) {
-                    GridCacheContext<Object, Object> ctx = ((IgniteKernal) ignite(nodeCount() + i))
+                    GridCacheContext<Object, Object> ctx = ((IgniteKernal)ignite(nodeCount() + i))
                         .internalCache(DYNAMIC_CACHE_NAME).context();
 
                     assertFalse(ctx.affinityNode());
@@ -995,12 +1158,12 @@ public class IgniteDynamicCacheStartSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < nodeCount(); i++) {
             final int idx = i;
 
-                latches[i] = new CountDownLatch(1);
-                lsnrs[i] = new IgnitePredicate<CacheEvent>() {
-                    @Override public boolean apply(CacheEvent e) {
-                        switch (e.type()) {
-                            case EventType.EVT_CACHE_NODES_LEFT:
-                                latches[idx].countDown();
+            latches[i] = new CountDownLatch(1);
+            lsnrs[i] = new IgnitePredicate<CacheEvent>() {
+                @Override public boolean apply(CacheEvent e) {
+                    switch (e.type()) {
+                        case EventType.EVT_CACHE_NODES_LEFT:
+                            latches[idx].countDown();
 
                             break;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
index c9f77cd..1b779c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsIgniteMock.java
@@ -291,6 +291,13 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         throwUnsupported();
 
@@ -312,6 +319,13 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throwUnsupported();
+
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throwUnsupported();
     }
@@ -354,6 +368,11 @@ public class IgfsIgniteMock implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        throwUnsupported();
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable String name) {
         throwUnsupported();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index b559897..5722fa3 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -230,6 +230,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg) {
         return null;
     }
@@ -261,6 +266,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         return null;
     }
@@ -276,6 +286,11 @@ public class IgniteMock implements Ignite {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
index 2598bc5..21fc28d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteProcessProxy.java
@@ -426,6 +426,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
@@ -441,6 +446,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> void addCacheConfiguration(CacheConfiguration<K, V> cacheCfg) {
         throw new UnsupportedOperationException("Operation isn't supported yet.");
     }
@@ -477,6 +487,11 @@ public class IgniteProcessProxy implements IgniteEx {
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        throw new UnsupportedOperationException("Operation isn't supported yet.");
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> cache(@Nullable final String name) {
         return new IgniteCacheProcessProxy<>(name, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/56998e70/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
index 7f79c0e..0a4bf20 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpringBean.java
@@ -302,6 +302,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> createCaches(Collection<CacheConfiguration> cacheCfgs) {
+        checkIgnite();
+
+        return g.createCaches(cacheCfgs);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> getOrCreateCache(CacheConfiguration<K, V> cacheCfg, NearCacheConfiguration<K, V> nearCfg) {
         checkIgnite();
 
@@ -330,6 +337,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public Collection<IgniteCache> getOrCreateCaches(Collection<CacheConfiguration> cacheCfgs) {
+        checkIgnite();
+
+        return g.getOrCreateCaches(cacheCfgs);
+    }
+
+    /** {@inheritDoc} */
     @Override public <K, V> IgniteCache<K, V> createCache(String cacheName) {
         checkIgnite();
 
@@ -351,6 +365,13 @@ public class IgniteSpringBean implements Ignite, DisposableBean, InitializingBea
     }
 
     /** {@inheritDoc} */
+    @Override public void destroyCaches(Collection<String> cacheNames) {
+        checkIgnite();
+
+        g.destroyCaches(cacheNames);
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteTransactions transactions() {
         checkIgnite();
 


[17/19] ignite git commit: ignite-4285 For serializable txs allow multiple threads to get read lock for the same key

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 b005b29..d77933e 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
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.NodeStoppingException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheEntryPredicate;
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -346,25 +347,6 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
     }
 
     /**
-     * @param cached Entry.
-     * @return {@code True} if locked.
-     * @throws GridCacheEntryRemovedException If removed.
-     */
-    private boolean locked(GridCacheEntryEx cached) throws GridCacheEntryRemovedException {
-        return (cached.lockedLocally(lockVer) && filter(cached)); // If filter failed, lock is failed.
-    }
-
-    /**
-     * @param cached Entry.
-     * @param owner Lock owner.
-     * @return {@code True} if locked.
-     */
-    private boolean locked(GridCacheEntryEx cached, GridCacheMvccCandidate owner) {
-        // Reentry-aware check (if filter failed, lock is failed).
-        return owner != null && owner.matches(lockVer, cctx.nodeId(), threadId) && filter(cached);
-    }
-
-    /**
      * Adds entry to future.
      *
      * @param entry Entry to add.
@@ -392,11 +374,11 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
             threadId,
             lockVer,
             null,
-            null,
             timeout,
             /*reenter*/false,
             inTx(),
-            implicitSingle()
+            implicitSingle(),
+            false
         );
 
         if (c == null && timeout < 0) {
@@ -575,10 +557,10 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                     break; // While.
 
                 try {
-                    GridCacheMvccCandidate owner = entry.readyLock(lockVer);
+                    CacheLockCandidates owners = entry.readyLock(lockVer);
 
                     if (timeout < 0) {
-                        if (owner == null || !owner.version().equals(lockVer)) {
+                        if (owners == null || !owners.hasCandidate(lockVer)) {
                             // We did not send any requests yet.
                             onFailed(false);
 
@@ -587,9 +569,9 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                     }
 
                     if (log.isDebugEnabled()) {
-                        if (!locked(entry, owner))
-                            log.debug("Entry is not locked (will keep waiting) [entry=" + entry +
-                                ", fut=" + this + ']');
+                        log.debug("Current lock owners [entry=" + entry +
+                            ", owners=" + owners +
+                            ", fut=" + this + ']');
                     }
 
                     break; // Inner while loop.

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 6b437b1..01bc4e0 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
@@ -273,7 +273,6 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                             nodeId,
                             req.threadId(),
                             req.version(),
-                            req.timeout(),
                             tx != null,
                             tx != null && tx.implicitSingle(),
                             null

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 1dbda69..a759194 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
@@ -40,6 +40,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.CacheInvokeEntry;
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -629,10 +630,10 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 try {
                     assert txEntry.explicitVersion() == null || entry.lockedBy(txEntry.explicitVersion());
 
-                    GridCacheMvccCandidate c = entry.readyLock(tx.xidVersion());
+                    CacheLockCandidates owners = entry.readyLock(tx.xidVersion());
 
                     if (log.isDebugEnabled())
-                        log.debug("Current lock owner for entry [owner=" + c + ", entry=" + entry + ']');
+                        log.debug("Current lock owners for entry [owner=" + owners + ", entry=" + entry + ']');
 
                     break; // While.
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 b0eea01..5557d34 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
@@ -312,14 +312,14 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     null,
                     threadId,
                     lockVer,
-                    timeout,
                     true,
                     tx.entry(txKey).locked(),
                     inTx(),
                     inTx() && tx.implicitSingle(),
                     false,
                     false,
-                    null);
+                    null,
+                    false);
 
                 cand.topologyVersion(topVer);
             }
@@ -332,14 +332,14 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
                     null,
                     threadId,
                     lockVer,
-                    timeout,
                     true,
                     false,
                     inTx(),
                     inTx() && tx.implicitSingle(),
                     false,
                     false,
-                    null);
+                    null,
+                    false);
 
                 cand.topologyVersion(topVer);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index d495f83..30fc213 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -21,6 +21,7 @@ import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryInfo;
@@ -444,7 +445,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
         long timeout,
         boolean reenter,
         boolean tx,
-        boolean implicitSingle) throws GridCacheEntryRemovedException {
+        boolean implicitSingle,
+        boolean read) throws GridCacheEntryRemovedException {
         return addNearLocal(
             null,
             threadId,
@@ -453,7 +455,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             timeout,
             reenter,
             tx,
-            implicitSingle
+            implicitSingle,
+            read
         );
     }
 
@@ -468,10 +471,11 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
+     * @param read Read lock flag.
      * @return New candidate.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
-    @Nullable public GridCacheMvccCandidate addNearLocal(
+    @Nullable GridCacheMvccCandidate addNearLocal(
         @Nullable UUID dhtNodeId,
         long threadId,
         GridCacheVersion ver,
@@ -479,10 +483,11 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
         long timeout,
         boolean reenter,
         boolean tx,
-        boolean implicitSingle)
+        boolean implicitSingle,
+        boolean read)
         throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate owner;
+        CacheLockCandidates prev;
+        CacheLockCandidates owner = null;
         GridCacheMvccCandidate cand;
 
         CacheObject val;
@@ -505,7 +510,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             if (c != null)
                 return reenter ? c.reenter() : null;
 
-            prev = mvcc.anyOwner();
+            prev = mvcc.allOwners();
 
             boolean emptyBefore = mvcc.isEmpty();
 
@@ -520,14 +525,12 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
                 dhtNodeId,
                 threadId,
                 ver,
-                timeout,
                 tx,
-                implicitSingle);
+                implicitSingle,
+                read);
 
             cand.topologyVersion(topVer);
 
-            owner = mvcc.anyOwner();
-
             boolean emptyAfter = mvcc.isEmpty();
 
             checkCallbacks(emptyBefore, emptyAfter);
@@ -536,6 +539,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
 
             if (emptyAfter)
                 mvccExtras(null);
+            else
+                owner = mvcc.allOwners();
         }
 
         // This call must be outside of synchronization.
@@ -572,8 +577,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @return Removed candidate, or <tt>null</tt> if thread still holds the lock.
      */
     @Nullable @Override public GridCacheMvccCandidate removeLock() {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -585,7 +590,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
@@ -605,7 +610,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
 
                     mvcc.remove(cand.version());
 
-                    owner = mvcc.anyOwner();
+                    owner = mvcc.allOwners();
                 }
                 else
                     return null;
@@ -630,13 +635,12 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
 
         cctx.mvcc().removeExplicitLock(cand);
 
-        if (prev != null && owner != prev)
-            checkThreadChain(prev);
+        checkThreadChain(cand);
 
         // This call must be outside of synchronization.
         checkOwnerChanged(prev, owner, val);
 
-        return owner != prev ? prev : null;
+        return cand;
     }
 
     /** {@inheritDoc} */
@@ -648,7 +652,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     /**
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
-    public synchronized void reserveEviction() throws GridCacheEntryRemovedException {
+    synchronized void reserveEviction() throws GridCacheEntryRemovedException {
         checkObsolete();
 
         evictReservations++;
@@ -657,7 +661,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     /**
      *
      */
-    public synchronized void releaseEviction() {
+    synchronized void releaseEviction() {
         assert evictReservations > 0 : this;
         assert !obsolete() : this;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 3d9b6ab..2431379 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
@@ -325,7 +325,8 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             timeout,
             !inTx(),
             inTx(),
-            implicitSingleTx()
+            implicitSingleTx(),
+            false
         );
 
         if (inTx()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 cf5d2e2..7ac3295 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
@@ -356,7 +356,6 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
                                 nodeId,
                                 req.threadId(),
                                 req.version(),
-                                req.timeout(),
                                 tx != null,
                                 tx != null && tx.implicitSingle(),
                                 req.owned(entry.key())
@@ -577,9 +576,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
                                     if (!primary.isLocal()) {
                                         assert req != null;
 
-                                        req.addKey(
-                                            entry.key(),
-                                            ctx);
+                                        req.addKey(entry.key(), ctx);
                                     }
                                     else
                                         locKeys.add(cacheKey);

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index 10fa116..bc61333 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.local;
 
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -29,9 +30,6 @@ import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;
 
-import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
-import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
-
 /**
  * Cache entry for local caches.
  */
@@ -46,7 +44,7 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
      * @param hash Key hash value.
      * @param val Entry value.
      */
-    public GridLocalCacheEntry(
+    GridLocalCacheEntry(
         GridCacheContext ctx,
         KeyCacheObject key,
         int hash,
@@ -71,10 +69,11 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit transaction flag.
+     * @param read Read lock flag.
      * @return New candidate.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
-    @Nullable public GridCacheMvccCandidate addLocal(
+    @Nullable GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
         @Nullable GridCacheVersion serOrder,
@@ -83,14 +82,14 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
         boolean reenter,
         boolean tx,
         boolean implicitSingle,
-        boolean keepBinary
+        boolean read
     ) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate cand;
-        GridCacheMvccCandidate owner;
+        assert serReadVer == null || serOrder != null;
 
         CacheObject val;
-        boolean hasVal;
+        GridCacheMvccCandidate cand;
+        CacheLockCandidates prev;
+        CacheLockCandidates owner = null;
 
         synchronized (this) {
             checkObsolete();
@@ -108,7 +107,7 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
                 mvccExtras(mvcc);
             }
 
-            prev = mvcc.localOwner();
+            prev = mvcc.localOwners();
 
             cand = mvcc.addLocal(
                 this,
@@ -121,59 +120,50 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
                 reenter,
                 tx,
                 implicitSingle,
-                /*dht-local*/false
+                /*dht-local*/false,
+                read
             );
 
-            owner = mvcc.localOwner();
-
-            val = this.val;
-
-            hasVal = hasValueUnlocked();
-
             if (mvcc.isEmpty())
                 mvccExtras(null);
-        }
-
-        if (cand != null) {
-            if (!cand.reentry())
-                cctx.mvcc().addNext(cctx, cand);
+            else
+                owner = mvcc.localOwners();
 
-            // Event notification.
-            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED))
-                cctx.events().addEvent(partition(), key, cand.nodeId(), cand, EVT_CACHE_OBJECT_LOCKED, val, hasVal,
-                    val, hasVal, null, null, null, keepBinary);
+            val = this.val;
         }
 
-        checkOwnerChanged(prev, owner);
+        if (cand != null && !cand.reentry())
+            cctx.mvcc().addNext(cctx, cand);
+
+        checkOwnerChanged(prev, owner, val);
 
         return cand;
     }
 
     /**
-     *
      * @param cand Candidate.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate readyLocal(GridCacheMvccCandidate cand) {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+    void readyLocal(GridCacheMvccCandidate cand) {
+        CacheObject val;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         synchronized (this) {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.localOwner();
+                prev = mvcc.localOwners();
 
                 owner = mvcc.readyLocal(cand);
 
                 if (mvcc.isEmpty())
                     mvccExtras(null);
             }
-        }
 
-        checkOwnerChanged(prev, owner);
+            val = this.val;
+        }
 
-        return owner;
+        checkOwnerChanged(prev, owner, val);
     }
 
     /** {@inheritDoc} */
@@ -181,7 +171,7 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
         long timeout,
         @Nullable GridCacheVersion serOrder,
         GridCacheVersion serReadVer,
-        boolean keepBinary)
+        boolean read)
         throws GridCacheEntryRemovedException {
         GridCacheMvccCandidate cand = addLocal(
             tx.threadId(),
@@ -192,7 +182,7 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
             /*reenter*/false,
             /*tx*/true,
             tx.implicitSingle(),
-            keepBinary
+            read
         );
 
         if (cand != null) {
@@ -206,50 +196,32 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
 
     /**
      * Rechecks if lock should be reassigned.
-     *
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate recheck() {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+    public void recheck() {
+        CacheObject val;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         synchronized (this) {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.localOwner();
+                prev = mvcc.allOwners();
 
                 owner = mvcc.recheck();
 
                 if (mvcc.isEmpty())
                     mvccExtras(null);
             }
-        }
-
-        checkOwnerChanged(prev, owner);
-
-        return owner;
-    }
 
-    /**
-     * @param prev Previous owner.
-     * @param owner Current owner.
-     */
-    private void checkOwnerChanged(GridCacheMvccCandidate prev, GridCacheMvccCandidate owner) {
-        assert !Thread.holdsLock(this);
-
-        if (owner != prev) {
-            cctx.mvcc().callback().onOwnerChanged(this, prev, owner);
-
-            if (owner != null)
-                checkThreadChain(owner);
+            val = this.val;
         }
+
+        checkOwnerChanged(prev, owner, val);
     }
 
-    /**
-     * @param owner Starting candidate in the chain.
-     */
-    private void checkThreadChain(GridCacheMvccCandidate owner) {
+    /** {@inheritDoc} */
+    @Override protected void checkThreadChain(GridCacheMvccCandidate owner) {
         assert !Thread.holdsLock(this);
 
         assert owner != null;
@@ -298,62 +270,50 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
      *
      * @param threadId Thread ID.
      */
-    void releaseLocal(long threadId) {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
-
+    private void releaseLocal(long threadId) {
         CacheObject val;
-        boolean hasVal;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         synchronized (this) {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.localOwner();
+                prev = mvcc.localOwners();
 
-                owner = mvcc.releaseLocal(threadId);
+                mvcc.releaseLocal(threadId);
 
                 if (mvcc.isEmpty())
                     mvccExtras(null);
+                else
+                    owner = mvcc.allOwners();
             }
 
             val = this.val;
-            hasVal = hasValueUnlocked();
         }
 
-        if (prev != null && owner != prev) {
-            checkThreadChain(prev);
+        if (prev != null) {
+            for (int i = 0; i < prev.size(); i++) {
+                GridCacheMvccCandidate cand = prev.candidate(i);
 
-            // Event notification.
-            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_UNLOCKED))
-                cctx.events().addEvent(partition(), key, prev.nodeId(), prev, EVT_CACHE_OBJECT_UNLOCKED, val, hasVal,
-                    val, hasVal, null, null, null, true);
-        }
+                boolean unlocked = owner == null || !owner.hasCandidate(cand.version());
 
-        checkOwnerChanged(prev, owner);
-    }
+                if (unlocked)
+                    checkThreadChain(cand);
+            }
+        }
 
-    /**
-     * Removes candidate regardless if it is owner or not.
-     *
-     * @param cand Candidate to remove.
-     * @throws GridCacheEntryRemovedException If the entry was removed by version other
-     *      than one passed in.
-     */
-    void removeLock(GridCacheMvccCandidate cand) throws GridCacheEntryRemovedException {
-        removeLock(cand.version());
+        checkOwnerChanged(prev, owner, val);
     }
 
     /** {@inheritDoc} */
     @Override public boolean removeLock(GridCacheVersion ver) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheObject val;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         GridCacheMvccCandidate doomed;
 
-        CacheObject val;
-        boolean hasVal;
-
         synchronized (this) {
             GridCacheVersion obsoleteVer = obsoleteVersionExtras();
 
@@ -365,28 +325,23 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
             doomed = mvcc == null ? null : mvcc.candidate(ver);
 
             if (doomed != null) {
-                prev = mvcc.localOwner();
+                prev = mvcc.allOwners();
 
-                owner = mvcc.remove(ver);
+                mvcc.remove(ver);
 
                 if (mvcc.isEmpty())
                     mvccExtras(null);
+                else
+                    owner = mvcc.allOwners();
             }
 
             val = this.val;
-            hasVal = hasValueUnlocked();
         }
 
-        if (doomed != null) {
+        if (doomed != null)
             checkThreadChain(doomed);
 
-            // Event notification.
-            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_UNLOCKED))
-                cctx.events().addEvent(partition(), key, doomed.nodeId(), doomed, EVT_CACHE_OBJECT_UNLOCKED,
-                    val, hasVal, val, hasVal, null, null, null, true);
-        }
-
-        checkOwnerChanged(prev, owner);
+        checkOwnerChanged(prev, owner, val);
 
         return doomed != null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
index c5bd71a..8e224c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
@@ -231,7 +231,7 @@ public final class GridLocalLockFuture<K, V> extends GridFutureAdapter<Boolean>
             !inTx(),
             inTx(),
             implicitSingle(),
-            true
+            false
         );
 
         entries.add(entry);

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 c72d7f7..2c02f96 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
@@ -53,7 +53,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheReturnCompletableWrapper;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxFinishSync;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryFuture;
@@ -105,6 +104,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_TX;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.SYSTEM_POOL;
+import static org.apache.ignite.internal.processors.cache.GridCacheOperation.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.RECOVERY_FINISH;
 import static org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx.FinalizationStatus.USER_FINISH;
@@ -1615,7 +1615,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
 
                     assert serReadVer == null || (tx.optimistic() && tx.serializable()) : txEntry1;
 
-                    if (!entry1.tmLock(tx, timeout, serOrder, serReadVer, txEntry1.keepBinary())) {
+                    boolean read = serOrder != null && txEntry1.op() == READ;
+
+                    if (!entry1.tmLock(tx, timeout, serOrder, serReadVer, read)) {
                         // Unlock locks locked so far.
                         for (IgniteTxEntry txEntry2 : entries) {
                             if (txEntry2 == txEntry1)

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
index 6a73f79..e9be108 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheSerializableTransactionsTest.java
@@ -590,7 +590,7 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testTxCommitReadOnlyGetAll(boolean needVer) throws Exception {
+    private void testTxCommitReadOnlyGetAll(boolean needVer) throws Exception {
         Ignite ignite0 = ignite(0);
 
         final IgniteTransactions txs = ignite0.transactions();
@@ -925,6 +925,71 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testTxConflictReadWrite3() throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        final IgniteTransactions txs = ignite0.transactions();
+
+        for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            IgniteCache<Integer, Integer> cache = ignite0.createCache(ccfg);
+
+            List<Integer> readKeys = new ArrayList<>();
+            List<Integer> writeKeys = new ArrayList<>();
+
+            readKeys.add(primaryKey(cache));
+            writeKeys.add(primaryKeys(cache, 1, 1000_0000).get(0));
+
+            if (ccfg.getBackups() > 0) {
+                readKeys.add(backupKey(cache));
+                writeKeys.add(backupKeys(cache, 1, 1000_0000).get(0));
+            }
+
+            if (ccfg.getCacheMode() == PARTITIONED) {
+                readKeys.add(nearKey(cache));
+                writeKeys.add(nearKeys(cache, 1, 1000_0000).get(0));
+            }
+
+            try {
+                for (Integer readKey : readKeys) {
+                    for (Integer writeKey : writeKeys) {
+                        try {
+                            try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                cache.get(readKey);
+
+                                cache.put(writeKey, writeKey);
+
+                                updateKey(cache, readKey, 0);
+
+                                tx.commit();
+                            }
+
+                            fail();
+                        }
+                        catch (TransactionOptimisticException e) {
+                            // Expected exception.
+                        }
+
+                        try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
+                            cache.get(readKey);
+
+                            cache.put(writeKey, writeKey);
+
+                            tx.commit();
+                        }
+                    }
+                }
+            }
+            finally {
+                destroyCache(ccfg.getName());
+            }
+        }
+    }
+
+    /**
      * @throws Exception If failed
      */
     public void testTxConflictGetAndPut1() throws Exception {
@@ -2446,6 +2511,529 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testNoReadLockConflict() throws Exception {
+        checkNoReadLockConflict(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoReadLockConflictGetEntry() throws Exception {
+        checkNoReadLockConflict(true);
+    }
+
+    /**
+     * @param entry If {@code true} then uses 'getEntry' to read value, otherwise uses 'get'.
+     * @throws Exception If failed.
+     */
+    private void checkNoReadLockConflict(final boolean entry) throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            final AtomicInteger putKey = new AtomicInteger(1_000_000);
+
+            ignite0.createCache(ccfg);
+
+            CacheConfiguration<Integer, Integer> readCacheCcfg = new CacheConfiguration<>(ccfg);
+
+            readCacheCcfg.setName(ccfg.getName() + "-read");
+
+            ignite0.createCache(readCacheCcfg);
+
+            try {
+                checkNoReadLockConflict(ignite(0), ccfg.getName(), ccfg.getName(), entry, putKey);
+
+                checkNoReadLockConflict(ignite(1), ccfg.getName(), ccfg.getName(), entry, putKey);
+
+                checkNoReadLockConflict(ignite(SRVS), ccfg.getName(), ccfg.getName(), entry, putKey);
+
+                checkNoReadLockConflict(ignite(0), readCacheCcfg.getName(), ccfg.getName(), entry, putKey);
+
+                checkNoReadLockConflict(ignite(1), readCacheCcfg.getName(), ccfg.getName(), entry, putKey);
+
+                checkNoReadLockConflict(ignite(SRVS), readCacheCcfg.getName(), ccfg.getName(), entry, putKey);
+            }
+            finally {
+                destroyCache(ccfg.getName());
+
+                destroyCache(readCacheCcfg.getName());
+            }
+        }
+    }
+
+    /**
+     * @param ignite Node.
+     * @param readCacheName Cache name for get.
+     * @param writeCacheName Cache name for put.
+     * @param entry If {@code true} then uses 'getEntry' to read value, otherwise uses 'get'.
+     * @param putKey Write key counter.
+     * @throws Exception If failed.
+     */
+    private void checkNoReadLockConflict(final Ignite ignite,
+        String readCacheName,
+        String writeCacheName,
+        final boolean entry,
+        final AtomicInteger putKey) throws Exception
+    {
+        final int THREADS = 64;
+
+        final IgniteCache<Integer, Integer> readCache = ignite.cache(readCacheName);
+        final IgniteCache<Integer, Integer> writeCache = ignite.cache(writeCacheName);
+
+        List<Integer> readKeys = testKeys(readCache);
+
+        for (final Integer readKey : readKeys) {
+            final CyclicBarrier barrier = new CyclicBarrier(THREADS);
+
+            readCache.put(readKey, Integer.MIN_VALUE);
+
+            GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                        if (entry)
+                            readCache.get(readKey);
+                        else
+                            readCache.getEntry(readKey);
+
+                        barrier.await();
+
+                        writeCache.put(putKey.incrementAndGet(), 0);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, THREADS, "test-thread");
+
+            assertEquals((Integer)Integer.MIN_VALUE, readCache.get(readKey));
+
+            readCache.put(readKey, readKey);
+
+            assertEquals(readKey, readCache.get(readKey));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNoReadLockConflictMultiNode() throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        for (final CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            final AtomicInteger putKey = new AtomicInteger(1_000_000);
+
+            ignite0.createCache(ccfg);
+
+            try {
+                final int THREADS = 64;
+
+                IgniteCache<Integer, Integer> cache0 = ignite0.cache(ccfg.getName());
+
+                List<Integer> readKeys = testKeys(cache0);
+
+                for (final Integer readKey : readKeys) {
+                    final CyclicBarrier barrier = new CyclicBarrier(THREADS);
+
+                    cache0.put(readKey, Integer.MIN_VALUE);
+
+                    final AtomicInteger idx = new AtomicInteger();
+
+                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            Ignite ignite = ignite(idx.incrementAndGet() % (CLIENTS + SRVS));
+
+                            IgniteCache<Integer, Integer> cache = ignite.cache(ccfg.getName());
+
+                            try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                cache.get(readKey);
+
+                                barrier.await();
+
+                                cache.put(putKey.incrementAndGet(), 0);
+
+                                tx.commit();
+                            }
+
+                            return null;
+                        }
+                    }, THREADS, "test-thread");
+
+                    assertEquals((Integer)Integer.MIN_VALUE, cache0.get(readKey));
+
+                    cache0.put(readKey, readKey);
+
+                    assertEquals(readKey, cache0.get(readKey));
+                }
+            }
+            finally {
+                destroyCache(ccfg.getName());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("UnnecessaryLocalVariable")
+    public void testReadLockPessimisticTxConflict() throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            ignite0.createCache(ccfg);
+
+            try {
+                Ignite ignite = ignite0;
+
+                IgniteCache<Integer, Integer> cache = ignite.cache(ccfg.getName());
+
+                Integer writeKey = Integer.MAX_VALUE;
+
+                List<Integer> readKeys = testKeys(cache);
+
+                for (Integer readKey : readKeys) {
+                    CountDownLatch latch = new CountDownLatch(1);
+
+                    IgniteInternalFuture<?> fut = lockKey(latch, cache, readKey);
+
+                    try {
+                        // No conflict for write, conflict with pessimistic tx for read.
+                        try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                            cache.put(writeKey, writeKey);
+
+                            cache.get(readKey);
+
+                            tx.commit();
+                        }
+
+                        fail();
+                    }
+                    catch (TransactionOptimisticException e) {
+                        log.info("Expected exception: " + e);
+                    }
+                    finally {
+                        latch.countDown();
+                    }
+
+                    fut.get();
+                }
+            }
+            finally {
+                destroyCache(ccfg.getName());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("UnnecessaryLocalVariable")
+    public void testReadWriteTxConflict() throws Exception {
+        Ignite ignite0 = ignite(0);
+
+        for (CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            ignite0.createCache(ccfg);
+
+            try {
+                Ignite ignite = ignite0;
+
+                IgniteCache<Integer, Integer> cache = ignite.cache(ccfg.getName());
+
+                Integer writeKey = Integer.MAX_VALUE;
+
+                List<Integer> readKeys = testKeys(cache);
+
+                for (Integer readKey : readKeys) {
+                    try {
+                        // No conflict for read, conflict for write.
+                        try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                            cache.getAndPut(writeKey, writeKey);
+
+                            cache.get(readKey);
+
+                            updateKey(cache, writeKey, writeKey + readKey);
+
+                            tx.commit();
+                        }
+
+                        fail();
+                    }
+                    catch (TransactionOptimisticException e) {
+                        log.info("Expected exception: " + e);
+                    }
+
+                    assertEquals((Integer)(writeKey + readKey), cache.get(writeKey));
+                    assertNull(cache.get(readKey));
+
+                    cache.put(readKey, readKey);
+
+                    assertEquals(readKey, cache.get(readKey));
+                }
+            }
+            finally {
+                destroyCache(ccfg.getName());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadWriteTransactionsNoDeadlock() throws Exception {
+        checkReadWriteTransactionsNoDeadlock(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadWriteTransactionsNoDeadlockMultinode() throws Exception {
+        checkReadWriteTransactionsNoDeadlock(true);
+    }
+
+    /**
+     * @param multiNode Multi-node test flag.
+     * @throws Exception If failed.
+     */
+    private void checkReadWriteTransactionsNoDeadlock(final boolean multiNode) throws Exception {
+        final Ignite ignite0 = ignite(0);
+
+        for (final CacheConfiguration<Integer, Integer> ccfg : cacheConfigurations()) {
+            logCacheInfo(ccfg);
+
+            ignite0.createCache(ccfg);
+
+            try {
+                final long stopTime = U.currentTimeMillis() + 10_000;
+
+                final AtomicInteger idx = new AtomicInteger();
+
+                GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        Ignite ignite = multiNode ? ignite(idx.incrementAndGet() % (SRVS + CLIENTS)) : ignite0;
+
+                        IgniteCache<Integer, Integer> cache = ignite.cache(ccfg.getName());
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        while (U.currentTimeMillis() < stopTime) {
+                            try {
+                                try (Transaction tx = ignite.transactions().txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                    for (int i = 0; i < 10; i++) {
+                                        Integer key = rnd.nextInt(30);
+
+                                        if (rnd.nextBoolean())
+                                            cache.get(key);
+                                        else
+                                            cache.put(key, key);
+                                    }
+
+                                    tx.commit();
+                                }
+                            }
+                            catch (TransactionOptimisticException ignore) {
+                                // No-op.
+                            }
+                        }
+
+                        return null;
+                    }
+                }, 32, "test-thread");
+            }
+            finally {
+                destroyCache(ccfg.getName());
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReadWriteAccountTx() throws Exception {
+        final CacheConfiguration<Integer, Integer> ccfg = cacheConfiguration(PARTITIONED,
+            FULL_SYNC,
+            1,
+            false,
+            false);
+
+        ignite(0).createCache(ccfg);
+
+        try {
+            final int ACCOUNTS = 50;
+            final int VAL_PER_ACCOUNT = 1000;
+
+            IgniteCache<Integer, Account> cache0 = ignite(0).cache(ccfg.getName());
+
+            final Set<Integer> keys = new HashSet<>();
+
+            for (int i = 0; i < ACCOUNTS; i++) {
+                cache0.put(i, new Account(VAL_PER_ACCOUNT));
+
+                keys.add(i);
+            }
+
+            final List<Ignite> clients = clients();
+
+            final AtomicBoolean stop = new AtomicBoolean();
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            IgniteInternalFuture<?> readFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    try {
+                        int threadIdx = idx.getAndIncrement();
+
+                        int nodeIdx = threadIdx % (SRVS + CLIENTS);
+
+                        Ignite node = ignite(nodeIdx);
+
+                        IgniteCache<Integer, Account> cache = node.cache(ccfg.getName());
+
+                        IgniteTransactions txs = node.transactions();
+
+                        Integer putKey = ACCOUNTS + threadIdx;
+
+                        while (!stop.get()) {
+                            int sum;
+
+                            while (true) {
+                                sum = 0;
+
+                                try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                    Map<Integer, Account> data = cache.getAll(keys);
+
+                                    for (int i = 0; i < ACCOUNTS; i++) {
+                                        Account account = data.get(i);
+
+                                        assertNotNull(account);
+
+                                        sum += account.value();
+                                    }
+
+                                    cache.put(putKey, new Account(sum));
+
+                                    tx.commit();
+                                }
+                                catch (TransactionOptimisticException e) {
+                                    continue;
+                                }
+
+                                break;
+                            }
+
+                            assertEquals(ACCOUNTS * VAL_PER_ACCOUNT, sum);
+                        }
+
+                        return null;
+                    }
+                    catch (Throwable e) {
+                        stop.set(true);
+
+                        log.error("Unexpected error: " + e);
+
+                        throw e;
+                    }
+                }
+            }, (SRVS + CLIENTS) * 2, "update-thread");
+
+            IgniteInternalFuture<?> updateFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    try {
+                        int nodeIdx = idx.getAndIncrement() % clients.size();
+
+                        Ignite node = clients.get(nodeIdx);
+
+                        IgniteCache<Integer, Account> cache = node.cache(ccfg.getName());
+
+                        IgniteTransactions txs = node.transactions();
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        while (!stop.get()) {
+                            int id1 = rnd.nextInt(ACCOUNTS);
+
+                            int id2 = rnd.nextInt(ACCOUNTS);
+
+                            while (id2 == id1)
+                                id2 = rnd.nextInt(ACCOUNTS);
+
+                            while (true) {
+                                try (Transaction tx = txs.txStart(OPTIMISTIC, SERIALIZABLE)) {
+                                    Account a1 = cache.get(id1);
+                                    Account a2 = cache.get(id2);
+
+                                    assertNotNull(a1);
+                                    assertNotNull(a2);
+
+                                    if (a1.value() > 0) {
+                                        a1 = new Account(a1.value() - 1);
+                                        a2 = new Account(a2.value() + 1);
+                                    }
+
+                                    cache.put(id1, a1);
+                                    cache.put(id2, a2);
+
+                                    tx.commit();
+                                }
+                                catch (TransactionOptimisticException e) {
+                                    continue;
+                                }
+
+                                break;
+                            }
+                        }
+
+                        return null;
+                    }
+                    catch (Throwable e) {
+                        stop.set(true);
+
+                        log.error("Unexpected error: " + e);
+
+                        throw e;
+                    }
+                }
+            }, 2, "update-thread");
+
+            try {
+                U.sleep(15_000);
+            }
+            finally {
+                stop.set(true);
+            }
+
+            readFut.get();
+            updateFut.get();
+            int sum = 0;
+
+            for (int i = 0; i < ACCOUNTS; i++) {
+                Account a = cache0.get(i);
+
+                assertNotNull(a);
+                assertTrue(a.value() >= 0);
+
+                log.info("Account: " + a.value());
+
+                sum += a.value();
+            }
+
+            assertEquals(ACCOUNTS * VAL_PER_ACCOUNT, sum);
+        }
+        finally {
+            ignite(0).destroyCache(ccfg.getName());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testNearCacheReaderUpdate() throws Exception {
         Ignite ignite0 = ignite(0);
 
@@ -4189,13 +4777,17 @@ public class CacheSerializableTransactionsTest extends GridCommonAbstractTest {
 
         List<Integer> keys = new ArrayList<>();
 
-        if (ccfg.getCacheMode() == PARTITIONED)
-            keys.add(nearKey(cache));
+        if (!cache.unwrap(Ignite.class).configuration().isClientMode()) {
+            if (ccfg.getCacheMode() == PARTITIONED)
+                keys.add(nearKey(cache));
 
-        keys.add(primaryKey(cache));
+            keys.add(primaryKey(cache));
 
-        if (ccfg.getBackups() != 0)
-            keys.add(backupKey(cache));
+            if (ccfg.getBackups() != 0)
+                keys.add(backupKey(cache));
+        }
+        else
+            keys.add(nearKey(cache));
 
         return keys;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
index 234f362..ff2d62d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
@@ -82,14 +82,14 @@ public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
             ver,
             1,
             ver,
-            0,
             true,
             true,
             true,
             true,
             true,
             true,
-            null
+            null,
+            false
         );
 
         c.setOwner();
@@ -123,14 +123,14 @@ public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
             ver,
             1,
             ver,
-            0,
             false,
             false,
             false,
             false,
             false,
             false,
-            null
+            null,
+            false
         );
 
         short flags = c.flags();

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
index 1b97663..11a91b5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
@@ -37,6 +38,9 @@ import static org.apache.ignite.cache.CacheMode.PARTITIONED;
  * Test cases for multi-threaded tests in partitioned cache.
  */
 public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final UUID nodeId = UUID.randomUUID();
+
     /** Grid. */
     private IgniteKernal grid;
 
@@ -94,8 +98,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver1 = version(1);
         GridCacheVersion ver2 = version(2);
 
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -128,8 +132,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver1 = version(1);
         GridCacheVersion ver2 = version(2);
 
-        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true);
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
+        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -161,8 +165,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver1 = version(1);
         GridCacheVersion ver2 = version(2);
 
-        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true);
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
+        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -194,8 +198,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver1 = version(1);
         GridCacheVersion ver2 = version(2);
 
-        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, 0, true);
-        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true);
+        GridCacheMvccCandidate c1 = entry.addNearLocal(node1, 1, ver1, true);
+        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true);
 
         entry.readyNearLocal(ver2, ver2,  empty(), empty(), empty());
 
@@ -224,8 +228,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver1 = version(1);
         GridCacheVersion ver2 = version(2);
 
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, 0, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c2 = entry.addNearLocal(node1, 1, ver2, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -263,11 +267,11 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver0 = version(0);
         GridCacheVersion ver1 = version(1);
 
-        entry.addNearLocal(node1, 1, ver1, 0, true);
+        entry.addNearLocal(node1, 1, ver1, true);
 
         entry.readyNearLocal(ver1, ver1, empty(), empty(), Collections.singletonList(ver0));
 
-        entry.addRemote(node1, 1, ver0, 0, false, true);
+        entry.addRemote(node1, 1, ver0, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -297,13 +301,13 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(2);
         GridCacheVersion ver3 = version(3);
 
-        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true);
+        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true);
 
         entry.readyNearLocal(ver3, ver3, empty(), empty(), Arrays.asList(ver0, ver1, ver2));
 
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
 
@@ -340,13 +344,13 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(2);
         GridCacheVersion ver3 = version(3);
 
-        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true);
-        entry.addNearLocal(node1, 1, ver2, 0, true);
+        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true);
+        entry.addNearLocal(node1, 1, ver2, true);
 
         entry.readyNearLocal(ver3, ver3, empty(), empty(), Arrays.asList(ver0, ver1, ver2));
 
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
 
@@ -386,12 +390,12 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver5 = version(5);
         GridCacheVersion ver6 = version(6);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node1, 1, ver2, 0, false, true);
-        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, 0, true);
-        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true);
-        entry.addRemote(node1, 1, ver5, 0, false, true);
-        entry.addRemote(node1, 1, ver6, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node1, 1, ver2, false, true);
+        GridCacheMvccCandidate c3 = entry.addNearLocal(node1, 1, ver3, true);
+        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true);
+        entry.addRemote(node1, 1, ver5, false, true);
+        entry.addRemote(node1, 1, ver6, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
 
@@ -442,9 +446,9 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(20);
         GridCacheVersion ver3 = version(30);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addNearLocal(node1, 1, nearVer2, 0, true);
-        entry.addRemote(node1, 1, ver3, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addNearLocal(node1, 1, nearVer2, true);
+        entry.addRemote(node1, 1, ver3, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -480,9 +484,9 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(20);
         GridCacheVersion ver3 = version(30);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addNearLocal(node1, 1, nearVer2, 0, true);
-        entry.addRemote(node1, 1, ver3, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addNearLocal(node1, 1, nearVer2, true);
+        entry.addRemote(node1, 1, ver3, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -525,9 +529,9 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(20);
         GridCacheVersion ver3 = version(30);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addNearLocal(node1, 1, nearVer2, 0, true);
-        entry.addRemote(node1, 1, ver3, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addNearLocal(node1, 1, nearVer2, true);
+        entry.addRemote(node1, 1, ver3, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -570,9 +574,9 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(20);
         GridCacheVersion ver3 = version(30);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addNearLocal(node1, 1, nearVer2, 0, true);
-        entry.addRemote(node1, 1, ver3, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addNearLocal(node1, 1, nearVer2, true);
+        entry.addRemote(node1, 1, ver3, false, true);
 
         Collection<GridCacheMvccCandidate> rmtCands = entry.remoteMvccSnapshot();
         Collection<GridCacheMvccCandidate> nearLocCands = entry.localCandidates();
@@ -597,6 +601,222 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testSerializableReadLocksAdd() throws Exception {
+        GridCacheAdapter<String, String> cache = grid.internalCache();
+
+        GridCacheVersion serOrder1 = new GridCacheVersion(0, 0, 10, 1);
+        GridCacheVersion serOrder2 = new GridCacheVersion(0, 0, 20, 1);
+        GridCacheVersion serOrder3 = new GridCacheVersion(0, 0, 15, 1);
+
+        {
+            GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+            GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+            GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true);
+
+            assertNotNull(cand1);
+
+            GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true);
+
+            assertNotNull(cand2);
+
+            GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false);
+
+            assertNull(cand3);
+
+            cand3 = addLocal(mvcc, e, version(3), serOrder3, true);
+
+            assertNotNull(cand3);
+        }
+
+        {
+            GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+            GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+            GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder2, true);
+
+            assertNotNull(cand1);
+
+            GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder1, true);
+
+            assertNotNull(cand2);
+
+            GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false);
+
+            assertNull(cand3);
+
+            cand3 = addLocal(mvcc, e, version(3), serOrder3, true);
+
+            assertNotNull(cand3);
+        }
+
+        {
+            GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+            GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+            GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder3, false);
+
+            assertNotNull(cand1);
+
+            GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true);
+
+            assertNotNull(cand2);
+
+            GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder1, true);
+
+            assertNull(cand3);
+
+            cand3 = addLocal(mvcc, e, version(3), serOrder1, false);
+
+            assertNull(cand3);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSerializableReadLocksAssign() throws Exception {
+        GridCacheAdapter<String, String> cache = grid.internalCache();
+
+        GridCacheVersion serOrder1 = new GridCacheVersion(0, 0, 10, 1);
+        GridCacheVersion serOrder2 = new GridCacheVersion(0, 0, 20, 1);
+        GridCacheVersion serOrder3 = new GridCacheVersion(0, 0, 15, 1);
+
+        {
+            GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+            GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+            GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true);
+
+            assertNotNull(cand1);
+
+            GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true);
+
+            assertNotNull(cand2);
+
+            GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false);
+
+            assertNull(cand3);
+
+            cand3 = addLocal(mvcc, e, version(3), serOrder3, true);
+
+            assertNotNull(cand3);
+
+            CacheLockCandidates owners = mvcc.recheck();
+
+            assertNull(owners);
+
+            cand1.setReady();
+
+            owners = mvcc.recheck();
+
+            assertSame(cand1, owners);
+            checkCandidates(owners, cand1.version());
+
+            cand2.setReady();
+
+            owners = mvcc.recheck();
+            checkCandidates(owners, cand1.version(), cand2.version());
+
+            mvcc.remove(cand1.version());
+
+            owners = mvcc.recheck();
+            assertSame(cand2, owners);
+            checkCandidates(owners, cand2.version());
+        }
+
+        {
+            GridCacheMvcc mvcc = new GridCacheMvcc(cache.context());
+
+            GridCacheTestEntryEx e = new GridCacheTestEntryEx(cache.context(), "1");
+
+            GridCacheMvccCandidate cand1 = addLocal(mvcc, e, version(1), serOrder1, true);
+
+            assertNotNull(cand1);
+
+            GridCacheMvccCandidate cand2 = addLocal(mvcc, e, version(2), serOrder2, true);
+
+            assertNotNull(cand2);
+
+            GridCacheMvccCandidate cand3 = addLocal(mvcc, e, version(3), serOrder3, false);
+
+            assertNull(cand3);
+
+            cand3 = addLocal(mvcc, e, version(3), serOrder3, true);
+
+            assertNotNull(cand3);
+
+            CacheLockCandidates owners = mvcc.recheck();
+
+            assertNull(owners);
+
+            cand2.setReady();
+
+            owners = mvcc.recheck();
+
+            assertSame(cand2, owners);
+            checkCandidates(owners, cand2.version());
+
+            cand1.setReady();
+
+            owners = mvcc.recheck();
+            checkCandidates(owners, cand1.version(), cand2.version());
+
+            mvcc.remove(cand2.version());
+
+            owners = mvcc.recheck();
+            assertSame(cand1, owners);
+            checkCandidates(owners, cand1.version());
+        }
+    }
+
+    /**
+     * @param all Candidates list.
+     * @param vers Expected candidates.
+     */
+    private void checkCandidates(CacheLockCandidates all, GridCacheVersion...vers) {
+        assertNotNull(all);
+        assertEquals(vers.length, all.size());
+
+        for (GridCacheVersion ver : vers)
+            assertTrue(all.hasCandidate(ver));
+    }
+
+    /**
+     * @param mvcc Mvcc.
+     * @param e Entry.
+     * @param ver Version.
+     * @param serOrder Serializable tx version.
+     * @param read Read lock flag.
+     * @return Candidate.
+     */
+    @Nullable private GridCacheMvccCandidate addLocal(GridCacheMvcc mvcc,
+        GridCacheEntryEx e,
+        GridCacheVersion ver,
+        GridCacheVersion serOrder,
+        boolean read) {
+        return mvcc.addLocal(e,
+            nodeId,
+            null,
+            1,
+            ver,
+            0,
+            serOrder,
+            false,
+            true,
+            false,
+            true,
+            read
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testSerializableLocks() throws Exception {
         checkSerializableAdd(false);
 
@@ -627,7 +847,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
         );
 
         assertNotNull(cand1);
@@ -642,7 +863,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
         );
 
         assertNull(cand2);
@@ -681,7 +903,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
             );
 
         assertNotNull(cand1);
@@ -696,7 +919,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
         );
 
         assertNotNull(cand2);
@@ -711,7 +935,8 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
         );
 
         assertNull(cand3);
@@ -726,36 +951,37 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
             false,
             true,
             false,
-            true
+            true,
+            false
         );
 
         assertNotNull(cand4);
 
-        GridCacheMvccCandidate owner = mvcc.recheck();
+        CacheLockCandidates owners = mvcc.recheck();
 
-        assertNull(owner);
+        assertNull(owners);
 
         cand2.setReady();
 
-        owner = mvcc.recheck();
+        owners = mvcc.recheck();
 
-        assertNull(owner);
+        assertNull(owners);
 
         cand1.setReady();
 
-        owner = mvcc.recheck();
+        owners = mvcc.recheck();
 
-        assertSame(cand1, owner);
+        assertSame(cand1, owners);
 
-        owner = mvcc.recheck();
+        owners = mvcc.recheck();
 
-        assertSame(cand1, owner);
+        assertSame(cand1, owners);
 
         mvcc.remove(cand1.version());
 
-        owner = mvcc.recheck();
+        owners = mvcc.recheck();
 
-        assertSame(cand2, owner);
+        assertSame(cand2, owners);
     }
 
     /**


[19/19] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-1.7.4'

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-1.7.4'


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

Branch: refs/heads/master
Commit: 31bc3bf6ed84ef6ea2253d3fafb9716c5d1e91e0
Parents: 88c06ec 33dda46
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 2 12:19:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 2 12:19:04 2016 +0300

----------------------------------------------------------------------
 .../rest/RestProcessorMultiStartSelfTest.java   |  48 +-
 .../apache/ignite/IgniteSystemProperties.java   |  13 +
 .../apache/ignite/internal/IgniteKernal.java    |  12 +-
 .../processors/cache/CacheLockCandidates.java   |  42 ++
 .../cache/CacheLockCandidatesList.java          |  71 +++
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../processors/cache/GridCacheMapEntry.java     | 117 +++-
 .../processors/cache/GridCacheMvcc.java         | 376 ++++++++----
 .../processors/cache/GridCacheMvccCallback.java |   4 +-
 .../cache/GridCacheMvccCandidate.java           |  80 ++-
 .../processors/cache/GridCacheMvccManager.java  |  19 +-
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../distributed/GridDistributedCacheEntry.java  | 303 +++-------
 .../distributed/dht/GridDhtCacheEntry.java      |  32 +-
 .../distributed/dht/GridDhtLockFuture.java      |  34 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   5 +-
 ...GridNearAtomicSingleUpdateInvokeRequest.java |   5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../distributed/near/GridNearCacheEntry.java    |  44 +-
 .../distributed/near/GridNearLockFuture.java    |   3 +-
 .../near/GridNearTransactionalCache.java        |   5 +-
 .../cache/local/GridLocalCacheEntry.java        | 173 ++----
 .../cache/local/GridLocalLockFuture.java        |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  18 +-
 .../cache/transactions/IgniteTxManager.java     |   6 +-
 .../processors/odbc/OdbcRequestHandler.java     |  32 +-
 .../processors/rest/GridRestProcessor.java      |  15 +
 .../apache/ignite/internal/util/GridUnsafe.java |   9 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |   4 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../resources/META-INF/classnames.properties    |  86 ++-
 .../CacheSerializableTransactionsTest.java      | 604 ++++++++++++++++++-
 .../cache/GridCacheMvccFlagsTest.java           |   8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java | 334 ++++++++--
 .../processors/cache/GridCacheMvccSelfTest.java | 212 +++----
 .../processors/cache/GridCacheTestEntryEx.java  |  77 +--
 .../cache/OffheapCacheOnClientsTest.java        | 143 +++++
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   7 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../resources/META-INF/classnames.properties    | 114 ++++
 .../src/impl/binary/binary_reader_impl.cpp      |   3 +-
 .../putget-example/src/putget_example.cpp       |   2 +-
 .../query-example/src/query_example.cpp         |   2 +-
 modules/platforms/cpp/odbc/src/connection.cpp   |  21 +-
 modules/web-console/backend/app/agent.js        |  32 +-
 modules/web-console/backend/app/browser.js      |  13 +
 modules/web-console/backend/app/mongo.js        |   6 +
 .../backend/config/settings.json.sample         |   7 -
 modules/web-console/backend/index.js            |   6 +-
 modules/web-console/backend/middlewares/host.js |   7 +-
 modules/web-console/backend/routes/agent.js     |   4 +-
 modules/web-console/backend/services/agents.js  |   2 +-
 modules/web-console/backend/services/caches.js  |   2 +
 .../docker/compose/backend/.dockerignore        |   2 +
 .../docker/compose/backend/Dockerfile           |   6 +-
 .../web-console/docker/compose/backend/build.sh |   4 +-
 .../docker/compose/docker-compose.yml           |   5 -
 .../docker/compose/frontend/DockerfileBuild     |   4 +-
 .../docker/compose/frontend/build.sh            |   6 +-
 .../compose/frontend/nginx/web-console.conf     |   9 +
 .../web-console/docker/standalone/.dockerignore |   5 +
 .../web-console/docker/standalone/Dockerfile    |  10 +-
 modules/web-console/docker/standalone/build.sh  |   4 +-
 .../docker/standalone/docker-compose.yml        |   7 +-
 .../docker/standalone/nginx/web-console.conf    |   9 +
 .../frontend/app/data/pom-dependencies.json     |   2 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |   2 +-
 .../frontend/app/filters/duration.filter.js     |   3 +
 .../helpers/jade/form/form-field-checkbox.jade  |   2 +-
 .../helpers/jade/form/form-field-datalist.jade  |   2 +-
 .../helpers/jade/form/form-field-dropdown.jade  |   2 +-
 .../helpers/jade/form/form-field-number.jade    |   2 +-
 .../helpers/jade/form/form-field-password.jade  |   2 +-
 .../app/helpers/jade/form/form-field-text.jade  |   2 +-
 .../frontend/app/helpers/jade/mixins.jade       |  24 +-
 .../modules/configuration/generator/Beans.js    |   6 +
 .../generator/ConfigurationGenerator.js         | 165 ++---
 .../generator/JavaTransformer.service.js        |  15 +-
 .../generator/Properties.service.js             |  19 +
 .../generator/SpringTransformer.service.js      |  22 +-
 .../generator/defaults/cache.provider.js        |   8 +
 .../generator/defaults/cluster.provider.js      |   2 +-
 .../app/modules/form/field/tooltip.directive.js |   2 +-
 .../app/modules/form/group/tooltip.directive.js |   2 +-
 .../app/modules/form/panel/field.directive.js   |   4 +-
 .../app/modules/sql/scan-filter-input.jade      |   2 +-
 .../frontend/app/modules/sql/sql.controller.js  |   6 +-
 .../configuration/caches/client-near-cache.jade |   2 +-
 .../configuration/caches/concurrency.jade       |   2 +-
 .../states/configuration/caches/general.jade    |   2 +-
 .../states/configuration/caches/memory.jade     |   2 +-
 .../configuration/caches/near-cache-client.jade |   2 +-
 .../configuration/caches/near-cache-server.jade |   2 +-
 .../configuration/caches/node-filter.jade       |   2 +-
 .../states/configuration/caches/query.jade      |   2 +-
 .../states/configuration/caches/rebalance.jade  |   2 +-
 .../states/configuration/caches/statistics.jade |   2 +-
 .../states/configuration/caches/store.jade      |  20 +-
 .../states/configuration/clusters/atomic.jade   |   2 +-
 .../configuration/clusters/attributes.jade      |   2 +-
 .../states/configuration/clusters/binary.jade   |   2 +-
 .../configuration/clusters/cache-key-cfg.jade   |   2 +-
 .../configuration/clusters/checkpoint.jade      |   8 +-
 .../configuration/clusters/checkpoint/fs.jade   |   2 +-
 .../configuration/clusters/checkpoint/jdbc.jade |  41 +-
 .../configuration/clusters/checkpoint/s3.jade   | 229 +++----
 .../configuration/clusters/collision.jade       |  12 +-
 .../clusters/collision/custom.jade              |   2 +-
 .../clusters/collision/fifo-queue.jade          |   2 +-
 .../clusters/collision/job-stealing.jade        |   2 +-
 .../clusters/collision/priority-queue.jade      |   2 +-
 .../configuration/clusters/communication.jade   |   2 +-
 .../configuration/clusters/connector.jade       |   2 +-
 .../configuration/clusters/deployment.jade      |   2 +-
 .../configuration/clusters/discovery.jade       |   2 +-
 .../states/configuration/clusters/events.jade   |   2 +-
 .../states/configuration/clusters/failover.jade |   2 +-
 .../states/configuration/clusters/general.jade  |   2 +-
 .../clusters/general/discovery/cloud.jade       |   2 +-
 .../clusters/general/discovery/google.jade      |   2 +-
 .../clusters/general/discovery/jdbc.jade        |   3 +-
 .../clusters/general/discovery/multicast.jade   |   2 +-
 .../clusters/general/discovery/s3.jade          |   2 +-
 .../clusters/general/discovery/shared.jade      |   2 +-
 .../clusters/general/discovery/vm.jade          |   2 +-
 .../clusters/general/discovery/zookeeper.jade   |   2 +-
 .../bounded-exponential-backoff.jade            |   2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |   2 +-
 .../retrypolicy/exponential-backoff.jade        |   2 +-
 .../zookeeper/retrypolicy/forever.jade          |   2 +-
 .../zookeeper/retrypolicy/n-times.jade          |   2 +-
 .../zookeeper/retrypolicy/one-time.jade         |   2 +-
 .../zookeeper/retrypolicy/until-elapsed.jade    |   2 +-
 .../states/configuration/clusters/igfs.jade     |   2 +-
 .../configuration/clusters/load-balancing.jade  |   2 +-
 .../states/configuration/clusters/logger.jade   |   2 +-
 .../configuration/clusters/logger/custom.jade   |   2 +-
 .../configuration/clusters/logger/log4j.jade    |   2 +-
 .../configuration/clusters/logger/log4j2.jade   |   2 +-
 .../configuration/clusters/marshaller.jade      |   2 +-
 .../states/configuration/clusters/metrics.jade  |   2 +-
 .../states/configuration/clusters/odbc.jade     |   2 +-
 .../states/configuration/clusters/ssl.jade      |   2 +-
 .../states/configuration/clusters/swap.jade     |   2 +-
 .../states/configuration/clusters/thread.jade   |   2 +-
 .../states/configuration/clusters/time.jade     |   2 +-
 .../configuration/clusters/transactions.jade    |   2 +-
 .../states/configuration/domains/general.jade   |   2 +-
 .../states/configuration/domains/query.jade     |  16 +-
 .../states/configuration/domains/store.jade     |  12 +-
 .../modules/states/configuration/igfs/dual.jade |   2 +-
 .../states/configuration/igfs/fragmentizer.jade |   2 +-
 .../states/configuration/igfs/general.jade      |   2 +-
 .../modules/states/configuration/igfs/ipc.jade  |   2 +-
 .../modules/states/configuration/igfs/misc.jade |   6 +-
 .../states/configuration/igfs/secondary.jade    |   2 +-
 .../configuration/summary/summary.controller.js |  12 +-
 .../frontend/controllers/clusters-controller.js |  10 +
 .../frontend/controllers/domains-controller.js  |   8 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   4 +-
 .../frontend/gulpfile.babel.js/tasks/test.js    |  92 ---
 .../gulpfile.babel.js/webpack/common.js         |   5 +-
 .../webpack/environments/development.js         |   5 +
 modules/web-console/frontend/package.json       |   2 +-
 .../stylesheets/_font-awesome-custom.scss       |  18 +
 .../frontend/public/stylesheets/style.scss      |  13 +-
 .../frontend/views/configuration/caches.jade    |  22 +-
 .../frontend/views/configuration/clusters.jade  |  50 +-
 .../views/configuration/domains-import.jade     |   4 +-
 .../frontend/views/configuration/domains.jade   |   8 +-
 .../frontend/views/configuration/igfs.jade      |  14 +-
 .../frontend/views/configuration/summary.jade   |   2 +-
 .../frontend/views/includes/header.jade         |  15 +-
 .../frontend/views/settings/profile.jade        |   2 +-
 modules/web-console/frontend/views/sql/sql.jade |   2 +-
 .../views/templates/agent-download.jade         |   2 +-
 .../frontend/views/templates/batch-confirm.jade |   2 +-
 .../frontend/views/templates/clone.jade         |   2 +-
 .../frontend/views/templates/confirm.jade       |   2 +-
 modules/web-console/web-agent/README.txt        |   4 +-
 .../web-agent/bin/ignite-web-agent.bat          |   7 +-
 .../web-agent/bin/ignite-web-agent.sh           |   6 +-
 .../console/agent/AgentConfiguration.java       |   5 +-
 .../ignite/console/agent/AgentLauncher.java     |   6 +-
 .../ignite/console/demo/AgentClusterDemo.java   |   1 +
 186 files changed, 2869 insertions(+), 1486 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/31bc3bf6/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------


[09/19] ignite git commit: IGNITE-4299: Fixes for examples.

Posted by sb...@apache.org.
IGNITE-4299: Fixes for examples.


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

Branch: refs/heads/master
Commit: 3e2ccfd30427ba0552eea8667c0129ae5ace9c0b
Parents: 56998e7
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Nov 25 14:26:54 2016 +0300
Committer: Igor Sapego <is...@gridgain.com>
Committed: Fri Nov 25 14:26:54 2016 +0300

----------------------------------------------------------------------
 .../platforms/cpp/examples/putget-example/src/putget_example.cpp   | 2 +-
 modules/platforms/cpp/examples/query-example/src/query_example.cpp | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3e2ccfd3/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
index 0b57886..8bf9c8c 100644
--- a/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
+++ b/modules/platforms/cpp/examples/putget-example/src/putget_example.cpp
@@ -117,7 +117,7 @@ int main()
     }
 
     std::cout << std::endl;
-    std::cout << ">>> Example finished, press any key to exit ..." << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
     std::cout << std::endl;
 
     std::cin.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/3e2ccfd3/modules/platforms/cpp/examples/query-example/src/query_example.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/examples/query-example/src/query_example.cpp b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
index 9bf3e52..8c2ca0c 100644
--- a/modules/platforms/cpp/examples/query-example/src/query_example.cpp
+++ b/modules/platforms/cpp/examples/query-example/src/query_example.cpp
@@ -450,7 +450,7 @@ int main()
     }
 
     std::cout << std::endl;
-    std::cout << ">>> Example finished, press any key to exit ..." << std::endl;
+    std::cout << ">>> Example finished, press 'Enter' to exit ..." << std::endl;
     std::cout << std::endl;
 
     std::cin.get();


[04/19] ignite git commit: Web console beta-6.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
index ae13166..5062ce1 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/query.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'query'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade
index 824442c..d8ef3ad 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/rebalance.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'rebalance'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade
index 027a2bd..9f7a346 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/statistics.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'statistics'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
index a7d8f14..1cf80b8 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/store.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'store'
 -var model = 'backupItem'
@@ -88,6 +88,24 @@ mixin hibernateField(name, model, items, valid, save, newItem)
                                         'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect',
                                         'Choose JDBC dialect')
                                 .details-row
+                                    +number('Batch size:', pojoStoreFactory + '.batchSize', '"pojoBatchSize"', true, '512', '1',
+                                        'Maximum batch size for writeAll and deleteAll operations')
+                                .details-row
+                                    +number('Thread count:', pojoStoreFactory + '.maximumPoolSize', '"pojoMaximumPoolSize"', true, 'availableProcessors', '1',
+                                        'Maximum workers thread count.<br/>\
+                                        These threads are responsible for load cache.')
+                                .details-row
+                                    +number('Maximum write attempts:', pojoStoreFactory + '.maximumWriteAttempts', '"pojoMaximumWriteAttempts"', true, '2', '0',
+                                        'Maximum write attempts in case of database error')
+                                .details-row
+                                    +number('Parallel load threshold:', pojoStoreFactory + '.parallelLoadCacheMinimumThreshold', '"pojoParallelLoadCacheMinimumThreshold"', true, '512', '0',
+                                        'Parallel load cache minimum threshold.<br/>\
+                                        If <b>0</b> then load sequentially.')
+                                .details-row
+                                    +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator')
+                                .details-row
+                                    +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer')
+                                .details-row
                                     +checkbox('Escape table and filed names', pojoStoreFactory + '.sqlEscapeAll', '"sqlEscapeAll"',
                                         'If enabled than all schema, table and field names will be escaped with double quotes (for example: "tableName"."fieldName").<br/>\
                                         This enforces case sensitivity for field names and also allows having special characters in table and field names.<br/>\

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade
index ef83356..412ca3a 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/atomic.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'atomics'
 -var model = 'backupItem.atomicConfiguration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade
index b41b97c..0366ed8 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/attributes.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'attributes'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade
index 9994087..f8ec8f9 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/binary.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'binary'
 -var model = 'backupItem.binaryConfiguration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade
index 45ccc13..41a72c1 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/cache-key-cfg.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'cacheKeyCfg'
 -var model = 'backupItem.cacheKeyConfiguration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
index d2552a8..5cc996d 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'checkpoint'
 -var model = 'backupItem.checkpointSpi'
@@ -26,7 +26,7 @@ include ../../../../../app/helpers/jade/mixins.jade
         label Checkpointing
         ignite-form-field-tooltip.tipLabel
             | Checkpointing provides an ability to save an intermediate job state#[br]
-            | #[a(href="http://apacheignite.gridgain.org/docs/checkpointing" target="_blank") More info]
+            | #[a(href="http://apacheignite.readme.io/docs/checkpointing" target="_blank") More info]
         ignite-form-revert
     .panel-collapse(role='tabpanel' bs-collapse-target id=form)
         .panel-body(ng-if='ui.isPanelLoaded("#{form}")')
@@ -61,10 +61,10 @@ include ../../../../../app/helpers/jade/mixins.jade
                                     </ul>')
                                     +table-remove-button(model, 'Remove Checkpoint SPI')
 
-                            div(ng-if='model.kind === "FS"')
+                            div(ng-show='model.kind === "FS"')
                                 include ./checkpoint/fs.jade
 
-                            div(ng-if='model.kind === "Cache"')
+                            div(ng-show='model.kind === "Cache"')
                                 .settings-row
                                     +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true',
                                         'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
index 01f14f4..efb6ad0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/fs.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'checkpointFsPaths'
 -var dirPaths = 'model.FS.directoryPaths'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
index a7b217f..874799c 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/jdbc.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .settings-row
     +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name',
@@ -23,23 +23,24 @@ include ../../../../../../app/helpers/jade/mixins.jade
     +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"',
     'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect')
 .settings-row
-    +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name')
-.settings-row
-    +text('Table name:', 'model.JDBC.checkpointTableName', '"checkpointJdbcCheckpointTableName" + $index', 'false', 'CHECKPOINTS', 'Checkpoint table name')
-.settings-row
-    +text('Key field name:', 'model.JDBC.keyFieldName', '"checkpointJdbcKeyFieldName" + $index', 'false', 'NAME', 'Checkpoint key field name')
-.settings-row
-    +dropdown('Key field type:', 'model.JDBC.keyFieldType', '"checkpointJdbcKeyFieldType" + $index', 'true', 'VARCHAR', 'supportedJdbcTypes', 'Checkpoint key field type')
-.settings-row
-    +text('Value field name:', 'model.JDBC.valueFieldName', '"checkpointJdbcValueFieldName" + $index', 'false', 'VALUE', 'Checkpoint value field name')
-.settings-row
-    +dropdown('Value field type:', 'model.JDBC.valueFieldType', '"checkpointJdbcValueFieldType" + $index', 'true', 'BLOB', 'supportedJdbcTypes', 'Checkpoint value field type')
-.settings-row
-    +text('Expire date field name:', 'model.JDBC.expireDateFieldName', '"checkpointJdbcExpireDateFieldName" + $index', 'false', 'EXPIRE_DATE', 'Checkpoint expire date field name')
-.settings-row
-    +dropdown('Expire date field type:', 'model.JDBC.expireDateFieldType', '"checkpointJdbcExpireDateFieldType"', 'true', 'DATETIME', 'supportedJdbcTypes', 'Checkpoint expire date field type')
-.settings-row
-    +number('Number of retries:', 'model.JDBC.numberOfRetries', '"checkpointJdbcNumberOfRetries"', 'true', '2', '0', 'Number of retries in case of DB failure')
-.settings-row
     +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false',
-    'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name')
++showHideLink('jdbcExpanded', 'settings')
+    .details-row
+        +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name')
+    .details-row
+        +text('Table name:', 'model.JDBC.checkpointTableName', '"checkpointJdbcCheckpointTableName" + $index', 'false', 'CHECKPOINTS', 'Checkpoint table name')
+    .details-row
+        +text('Key field name:', 'model.JDBC.keyFieldName', '"checkpointJdbcKeyFieldName" + $index', 'false', 'NAME', 'Checkpoint key field name')
+    .details-row
+        +dropdown('Key field type:', 'model.JDBC.keyFieldType', '"checkpointJdbcKeyFieldType" + $index', 'true', 'VARCHAR', 'supportedJdbcTypes', 'Checkpoint key field type')
+    .details-row
+        +text('Value field name:', 'model.JDBC.valueFieldName', '"checkpointJdbcValueFieldName" + $index', 'false', 'VALUE', 'Checkpoint value field name')
+    .details-row
+        +dropdown('Value field type:', 'model.JDBC.valueFieldType', '"checkpointJdbcValueFieldType" + $index', 'true', 'BLOB', 'supportedJdbcTypes', 'Checkpoint value field type')
+    .details-row
+        +text('Expire date field name:', 'model.JDBC.expireDateFieldName', '"checkpointJdbcExpireDateFieldName" + $index', 'false', 'EXPIRE_DATE', 'Checkpoint expire date field name')
+    .details-row
+        +dropdown('Expire date field type:', 'model.JDBC.expireDateFieldType', '"checkpointJdbcExpireDateFieldType"', 'true', 'DATETIME', 'supportedJdbcTypes', 'Checkpoint expire date field type')
+    .details-row
+        +number('Number of retries:', 'model.JDBC.numberOfRetries', '"checkpointJdbcNumberOfRetries"', 'true', '2', '0', 'Number of retries in case of DB failure')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
index 8373b03..da28da7 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/checkpoint/s3.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var credentialsModel = 'model.S3.awsCredentials'
 -var clientCfgModel = 'model.S3.clientConfiguration'
@@ -42,6 +42,8 @@ include ../../../../../../app/helpers/jade/mixins.jade
         <li>Database - Session credentials with keys and session token</li>\
         <li>Custom - Custom AWS credentials provider</li>\
     </ul>')
+.settings-row
+    label Note, AWS credentials will be generated as stub
 .panel-details(ng-show=checkpointS3Path)
     .details-row
         +text('Path:', credentialsModel + '.Properties.path', '"checkpointS3PropertiesPath"', checkpointS3Path, 'Input properties file path',
@@ -53,122 +55,123 @@ include ../../../../../../app/helpers/jade/mixins.jade
 .settings-row
     +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix')
 .settings-row
-    +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\
-                                            {value: "HTTP", label: "HTTP"},\
-                                            {value: "HTTPS", label: "HTTPS"}\
+    +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false',
+        'Checkpoint listener implementation class name')
++showHideLink('s3Expanded', 'client configuration')
+    .details-row
+        +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\
+                {value: "HTTP", label: "HTTP"},\
+                {value: "HTTPS", label: "HTTPS"}\
+            ]',
+        'Provides an ability to save an intermediate job state\
+        <ul>\
+            <li>HTTP - Using the HTTP protocol is less secure than HTTPS, but can slightly reduce\
+                the system resources used when communicating with AWS</li>\
+            <li>HTTPS - Using the HTTPS protocol is more secure than using the HTTP protocol, but\
+                may use slightly more system resources. AWS recommends using HTTPS for maximize security</li>\
+        </ul>')
+    .details-row
+        +number('Maximum connections:', clientCfgModel + '.maxConnections', '"checkpointS3MaxConnections"',
+        'true', '50', '1', 'Maximum number of allowed open HTTP connections')
+    .details-row
+        +text('User agent:', clientCfgModel + '.userAgent', '"checkpointS3UserAgent"', 'false', 'System specific header',
+        'HTTP user agent header to send with all requests')
+    .details-row
+        +text-ip-address('Local address:', clientCfgModel + '.localAddress', '"checkpointS3LocalAddress"', 'true', 'Not specified',
+        'Optionally specifies the local address to bind to')
+    .details-row
+        +text('Proxy host:', clientCfgModel + '.proxyHost', '"checkpointS3ProxyHost"', 'false', 'Not specified',
+        'Optional proxy host the client will connect through')
+    .details-row
+        +number('Proxy port:', clientCfgModel + '.proxyPort', '"checkpointS3ProxyPort"', 'true', 'Not specified', '0',
+        'Optional proxy port the client will connect through')
+    .details-row
+        +text('Proxy user:', clientCfgModel + '.proxyUsername', '"checkpointS3ProxyUsername"', 'false', 'Not specified',
+        'Optional proxy user name to use if connecting through a proxy')
+    .details-row
+        +text('Proxy domain:', clientCfgModel + '.proxyDomain', '"checkpointS3ProxyDomain"', 'false', 'Not specified',
+        'Optional Windows domain name for configuring an NTLM proxy')
+    .details-row
+        +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified',
+        'Optional Windows workstation name for configuring NTLM proxy support')
+    .details-row
+        +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\
+                                            {value: "Default", label: "Default SDK retry policy"},\
+                                            {value: "DefaultMaxRetries", label: "Default with the specified max retry count"},\
+                                            {value: "DynamoDB", label: "Default for DynamoDB client"},\
+                                            {value: "DynamoDBMaxRetries", label: "DynamoDB with the specified max retry count"},\
+                                            {value: "Custom", label: "Custom configured"}\
                                         ]',
-    'Provides an ability to save an intermediate job state\
-    <ul>\
-        <li>HTTP - Using the HTTP protocol is less secure than HTTPS, but can slightly reduce\
-            the system resources used when communicating with AWS</li>\
-        <li>HTTPS - Using the HTTPS protocol is more secure than using the HTTP protocol, but\
-            may use slightly more system resources. AWS recommends using HTTPS for maximize security</li>\
-    </ul>')
-.settings-row
-    +number('Maximum connections:', clientCfgModel + '.maxConnections', '"checkpointS3MaxConnections"',
-    'true', '50', '1', 'Maximum number of allowed open HTTP connections')
-.settings-row
-    +text('User agent:', clientCfgModel + '.userAgent', '"checkpointS3UserAgent"', 'false', 'System specific header',
-    'HTTP user agent header to send with all requests')
-.settings-row
-    +text-ip-address('Local address:', clientCfgModel + '.localAddress', '"checkpointS3LocalAddress"', 'true', 'Not specified',
-    'Optionally specifies the local address to bind to')
-.settings-row
-    +text('Proxy host:', clientCfgModel + '.proxyHost', '"checkpointS3ProxyHost"', 'false', 'Not specified',
-    'Optional proxy host the client will connect through')
-.settings-row
-    +number('Proxy port:', clientCfgModel + '.proxyPort', '"checkpointS3ProxyPort"', 'true', 'Not specified', '0',
-    'Optional proxy port the client will connect through')
-.settings-row
-    +text('Proxy user:', clientCfgModel + '.proxyUsername', '"checkpointS3ProxyUsername"', 'false', 'Not specified',
-    'Optional proxy user name to use if connecting through a proxy')
-.settings-row
-    +text('Proxy domain:', clientCfgModel + '.proxyDomain', '"checkpointS3ProxyDomain"', 'false', 'Not specified',
-    'Optional Windows domain name for configuring an NTLM proxy')
-.settings-row
-    +text('Proxy workstation:', clientCfgModel + '.proxyWorkstation', '"checkpointS3ProxyWorkstation"', 'false', 'Not specified',
-    'Optional Windows workstation name for configuring NTLM proxy support')
-.settings-row
-    +dropdown('Retry policy:', clientRetryModel + '.kind', '"checkpointS3RetryPolicy"', 'true', 'Default', '[\
-                                        {value: "Default", label: "Default SDK retry policy"},\
-                                        {value: "DefaultMaxRetries", label: "Default with the specified max retry count"},\
-                                        {value: "DynamoDB", label: "Default for DynamoDB client"},\
-                                        {value: "DynamoDBMaxRetries", label: "DynamoDB with the specified max retry count"},\
-                                        {value: "Custom", label: "Custom configured"}\
-                                    ]',
-    'Provides an ability to save an intermediate job state\
-    <ul>\
-        <li>SDK default retry policy - This policy will honor the maxErrorRetry set in ClientConfiguration</li>\
-        <li>Default with the specified max retry count - Default SDK retry policy with the specified max retry count</li>\
-        <li>Default for DynamoDB client - This policy will honor the maxErrorRetry set in ClientConfiguration</li>\
-        <li>DynamoDB with the specified max retry count - This policy will honor the maxErrorRetry set in ClientConfiguration with the specified max retry count</li>\
-        <li>Custom configured - Custom configured SDK retry policy</li>\
-    </ul>')
-.panel-details(ng-show=checkpointS3DefaultMaxRetry)
+        'Provides an ability to save an intermediate job state\
+        <ul>\
+            <li>SDK default retry policy - This policy will honor the maxErrorRetry set in ClientConfiguration</li>\
+            <li>Default with the specified max retry count - Default SDK retry policy with the specified max retry count</li>\
+            <li>Default for DynamoDB client - This policy will honor the maxErrorRetry set in ClientConfiguration</li>\
+            <li>DynamoDB with the specified max retry count - This policy will honor the maxErrorRetry set in ClientConfiguration with the specified max retry count</li>\
+            <li>Custom configured - Custom configured SDK retry policy</li>\
+        </ul>')
+    .panel-details(ng-show=checkpointS3DefaultMaxRetry)
+        .details-row
+            +number-required('Maximum retry attempts:', clientRetryModel + '.DefaultMaxRetries.maxErrorRetry', '"checkpointS3DefaultMaxErrorRetry"', 'true', checkpointS3DefaultMaxRetry, '-1', '1',
+            'Maximum number of retry attempts for failed requests')
+    .panel-details(ng-show=checkpointS3DynamoDbMaxRetry)
+        .details-row
+            +number-required('Maximum retry attempts:', clientRetryModel + '.DynamoDBMaxRetries.maxErrorRetry', '"checkpointS3DynamoDBMaxErrorRetry"', 'true', checkpointS3DynamoDbMaxRetry, '-1', '1',
+            'Maximum number of retry attempts for failed requests')
+    .panel-details(ng-show=checkpointS3CustomRetry)
+        .details-row
+            +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry,
+            'Retry condition on whether a specific request and exception should be retried')
+        .details-row
+            +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry,
+            'Back-off strategy for controlling how long the next retry should wait')
+        .details-row
+            +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1',
+            'Maximum number of retry attempts for failed requests')
+        .details-row
+            +checkbox('Honor the max error retry set', clientRetryModel + '.Custom.honorMaxErrorRetryInClientConfig', '"checkpointS3CustomHonorMaxErrorRetryInClientConfig"',
+            'Whether this retry policy should honor the max error retry set by ClientConfiguration#setMaxErrorRetry(int)')
     .details-row
-        +number-required('Maximum retry attempts:', clientRetryModel + '.DefaultMaxRetries.maxErrorRetry', '"checkpointS3DefaultMaxErrorRetry"', 'true', checkpointS3DefaultMaxRetry, '-1', '1',
-        'Maximum number of retry attempts for failed requests')
-.panel-details(ng-show=checkpointS3DynamoDbMaxRetry)
+        +number('Maximum retry attempts:', clientCfgModel + '.maxErrorRetry', '"checkpointS3MaxErrorRetry"', 'true', '-1', '0',
+        'Maximum number of retry attempts for failed retryable requests<br/>\
+        If -1 the configured RetryPolicy will be used to control the retry count')
     .details-row
-        +number-required('Maximum retry attempts:', clientRetryModel + '.DynamoDBMaxRetries.maxErrorRetry', '"checkpointS3DynamoDBMaxErrorRetry"', 'true', checkpointS3DynamoDbMaxRetry, '-1', '1',
-        'Maximum number of retry attempts for failed requests')
-.panel-details(ng-show=checkpointS3CustomRetry)
+        +number('Socket timeout:', clientCfgModel + '.socketTimeout', '"checkpointS3SocketTimeout"', 'true', '50000', '0',
+        'Amount of time in milliseconds to wait for data to be transfered over an established, open connection before the connection times out and is closed<br/>\
+        A value of <b>0</b> means infinity')
     .details-row
-        +java-class('Retry condition:', clientRetryModel + '.Custom.retryCondition', '"checkpointS3CustomRetryPolicy" + $index', 'true', checkpointS3CustomRetry,
-        'Retry condition on whether a specific request and exception should be retried')
+        +number('Connection timeout:', clientCfgModel + '.connectionTimeout', '"checkpointS3ConnectionTimeout"', 'true', '50000', '0',
+        'Amount of time in milliseconds to wait when initially establishing a connection before giving up and timing out<br/>\
+        A value of <b>0</b> means infinity')
     .details-row
-        +java-class('Backoff strategy:', clientRetryModel + '.Custom.backoffStrategy', '"checkpointS3CustomBackoffStrategy" + $index', 'true', checkpointS3CustomRetry,
-        'Back-off strategy for controlling how long the next retry should wait')
+        +number('Request timeout:', clientCfgModel + '.requestTimeout', '"checkpointS3RequestTimeout"', 'true', '0', '-1',
+        'Amount of time in milliseconds to wait for the request to complete before giving up and timing out<br/>\
+        A non - positive value means infinity')
     .details-row
-        +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1',
-        'Maximum number of retry attempts for failed requests')
+        +text('Signature algorithm:', clientCfgModel + '.signerOverride', '"checkpointS3SignerOverride"', 'false', 'Not specified',
+        'Name of the signature algorithm to use for signing requests made by this client')
     .details-row
-        +checkbox('Honor the max error retry set', clientRetryModel + '.Custom.honorMaxErrorRetryInClientConfig', '"checkpointS3CustomHonorMaxErrorRetryInClientConfig"',
-        'Whether this retry policy should honor the max error retry set by ClientConfiguration#setMaxErrorRetry(int)')
-.settings-row
-    +number('Maximum retry attempts:', clientCfgModel + '.maxErrorRetry', '"checkpointS3MaxErrorRetry"', 'true', '-1', '0',
-    'Maximum number of retry attempts for failed retryable requests<br/>\
-    If -1 the configured RetryPolicy will be used to control the retry count')
-.settings-row
-    +number('Socket timeout:', clientCfgModel + '.socketTimeout', '"checkpointS3SocketTimeout"', 'true', '50000', '0',
-    'Amount of time in milliseconds to wait for data to be transfered over an established, open connection before the connection times out and is closed<br/>\
-    A value of <b>0</b> means infinity')
-.settings-row
-    +number('Connection timeout:', clientCfgModel + '.connectionTimeout', '"checkpointS3ConnectionTimeout"', 'true', '50000', '0',
-    'Amount of time in milliseconds to wait when initially establishing a connection before giving up and timing out<br/>\
-    A value of <b>0</b> means infinity')
-.settings-row
-    +number('Request timeout:', clientCfgModel + '.requestTimeout', '"checkpointS3RequestTimeout"', 'true', '0', '-1',
-    'Amount of time in milliseconds to wait for the request to complete before giving up and timing out<br/>\
-    A non - positive value means infinity')
-.settings-row
-    +text('Signature algorithm:', clientCfgModel + '.signerOverride', '"checkpointS3SignerOverride"', 'false', 'Not specified',
-    'Name of the signature algorithm to use for signing requests made by this client')
-.settings-row
-    +number('Connection TTL:', clientCfgModel + '.connectionTTL', '"checkpointS3ConnectionTTL"', 'true', '-1', '-1',
-    'Expiration time in milliseconds for a connection in the connection pool<br/>\
-    By default, it is set to <b>-1</b>, i.e. connections do not expire')
-.settings-row
-    +number('Idle timeout:', clientCfgModel + '.connectionMaxIdleMillis', '"checkpointS3ConnectionMaxIdleMillis"', 'true', '60000', '0',
-    'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse')
-.settings-row
-    +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false',
-    'DNS Resolver that should be used to for resolving AWS IP addresses')
-.settings-row
-    +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0',
-    'Response metadata cache size')
-.settings-row
-    +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false',
-    'SecureRandom to be used by the SDK class name')
-.settings-row
-    +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started')
-.settings-row
-    +checkbox('Use GZIP', clientCfgModel + '.useGzip', '"checkpointS3UseGzip"', 'Checks if gzip compression is used')
-.settings-row
-    +checkbox('Preemptively basic authentication', clientCfgModel + '.preemptiveBasicProxyAuth', '"checkpointS3PreemptiveBasicProxyAuth"',
-    'Attempt to authenticate preemptively against proxy servers using basic authentication')
-.settings-row
-    +checkbox('TCP KeepAlive', clientCfgModel + '.useTcpKeepAlive', '"checkpointS3UseTcpKeepAlive"', 'TCP KeepAlive support is enabled')
-.settings-row
-    +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false',
-    'Checkpoint listener implementation class name')
+        +number('Connection TTL:', clientCfgModel + '.connectionTTL', '"checkpointS3ConnectionTTL"', 'true', '-1', '-1',
+        'Expiration time in milliseconds for a connection in the connection pool<br/>\
+        By default, it is set to <b>-1</b>, i.e. connections do not expire')
+    .details-row
+        +number('Idle timeout:', clientCfgModel + '.connectionMaxIdleMillis', '"checkpointS3ConnectionMaxIdleMillis"', 'true', '60000', '0',
+        'Maximum amount of time that an idle connection may sit in the connection pool and still be eligible for reuse')
+    .details-row
+        +java-class('DNS resolver:', clientCfgModel + '.dnsResolver', '"checkpointS3DnsResolver" + $index', 'true', 'false',
+        'DNS Resolver that should be used to for resolving AWS IP addresses')
+    .details-row
+        +number('Response metadata cache size:', clientCfgModel + '.responseMetadataCacheSize', '"checkpointS3ResponseMetadataCacheSize"', 'true', '50', '0',
+        'Response metadata cache size')
+    .details-row
+        +java-class('SecureRandom class name:', clientCfgModel + '.secureRandom', '"checkpointS3SecureRandom" + $index', 'true', 'false',
+        'SecureRandom to be used by the SDK class name')
+    .details-row
+        +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started')
+    .details-row
+        +checkbox('Use GZIP', clientCfgModel + '.useGzip', '"checkpointS3UseGzip"', 'Checks if gzip compression is used')
+    .details-row
+        +checkbox('Preemptively basic authentication', clientCfgModel + '.preemptiveBasicProxyAuth', '"checkpointS3PreemptiveBasicProxyAuth"',
+        'Attempt to authenticate preemptively against proxy servers using basic authentication')
+    .details-row
+        +checkbox('TCP KeepAlive', clientCfgModel + '.useTcpKeepAlive', '"checkpointS3UseTcpKeepAlive"', 'TCP KeepAlive support is enabled')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade
index 1a5d6d6..91676df 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'collision'
 -var model = 'backupItem.collision'
@@ -48,15 +48,15 @@ include ../../../../../app/helpers/jade/mixins.jade
                             <li>Custom - custom CollisionSpi implementation</li>\
                             <li>Default - jobs are activated immediately on arrival to mapped node</li>\
                         </ul>')
-                .settings-row(ng-if='#{modelCollisionKind} !== "Noop"')
+                .settings-row(ng-show='#{modelCollisionKind} !== "Noop"')
                     .panel-details
-                        div(ng-if='#{modelCollisionKind} === "JobStealing"')
+                        div(ng-show='#{modelCollisionKind} === "JobStealing"')
                             include ./collision/job-stealing.jade
-                        div(ng-if='#{modelCollisionKind} === "FifoQueue"')
+                        div(ng-show='#{modelCollisionKind} === "FifoQueue"')
                             include ./collision/fifo-queue.jade
-                        div(ng-if='#{modelCollisionKind} === "PriorityQueue"')
+                        div(ng-show='#{modelCollisionKind} === "PriorityQueue"')
                             include ./collision/priority-queue.jade
-                        div(ng-if='#{modelCollisionKind} === "Custom"')
+                        div(ng-show='#{modelCollisionKind} === "Custom"')
                             include ./collision/custom.jade
             .col-sm-6
                 -var model = 'backupItem.collision'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
index 9238917..31a6be7 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/custom.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.collision.Custom'
 -var required = 'backupItem.collision.kind === "Custom"'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade
index f16363d..cd8b6a3 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/fifo-queue.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.collision.FifoQueue'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
index 3659115..d4e537a 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/job-stealing.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.collision.JobStealing'
 -var stealingAttributes = model + '.stealingAttributes'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade
index bdd1dac..6f52ee0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/collision/priority-queue.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.collision.PriorityQueue'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade
index 55bf909..047c9a2 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/communication.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'communication'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade
index 6e75283..40d65af 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/connector.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'connector'
 -var model = 'backupItem.connector'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
index 67b6b4f..4cfd9f5 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/deployment.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'deployment'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade
index c9a2be3..3bf0e29 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/discovery.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'discovery'
 -var model = 'backupItem.discovery'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
index 42c319c..3f2d6cb 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/events.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'events'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
index b29701d..aaed8e9 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/failover.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem'
 -var form = 'failoverSpi'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade
index 8ec9b13..b2ce71f 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade
index 640c78c..52fb21b 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/cloud.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var discoveryKind = 'Cloud'
 -var required = 'backupItem.discovery.kind == "' + discoveryKind + '"'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade
index b1a5958..c12bd04 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/google.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 
 -var discoveryKind = 'GoogleStorage'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade
index 42cf697..4e4246d 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/jdbc.jade
@@ -14,8 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.Jdbc'
 -var required = 'backupItem.discovery.kind === "Jdbc"'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade
index 829fbaa..6a7e9fa 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/multicast.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var model = 'backupItem.discovery.Multicast'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade
index e255f20..e5eaff3 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/s3.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var discoveryKind = 'S3'
 -var required = 'backupItem.discovery.kind == "' + discoveryKind + '"'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade
index 2a949a5..ddd9bfa 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/shared.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.SharedFs'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade
index 6588e6a..46ebae0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/vm.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.Vm'
 -var addresses = model + '.addresses'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
index da36993..2e567ed 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var discoveryKind = 'ZooKeeper'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade
index 5e4bbda..d3c1f9f 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/bounded-exponential-backoff.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.BoundedExponentialBackoff'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
index 3dcf514..5a03de8 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/custom.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy'
 -var retry = model + '.Custom'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade
index 618683c..0a072f7 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/exponential-backoff.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.ExponentialBackoff'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade
index 10e4fb1..a7bfd20 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/forever.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.Forever'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade
index 329f693..1746232 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/n-times.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.NTimes'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade
index 938ecea..5a7e3fe 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/one-time.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.OneTime'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade
index 513da6a..548211c 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/general/discovery/zookeeper/retrypolicy/until-elapsed.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem.discovery.ZooKeeper.retryPolicy.UntilElapsed'
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade
index 7eb1505..25bd5b0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/igfs.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'igfs'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
index 916504b..7fd78bf 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/load-balancing.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var model = 'backupItem'
 -var form = 'loadBalancing'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade
index 31f87a9..983d818 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'logger'
 -var model = 'backupItem.logger'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
index df80af0..385d647 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/custom.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'logger'
 -var model = 'backupItem.logger.Custom'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade
index cf556ec..1eaa529 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'logger'
 -var model = 'backupItem.logger.Log4j'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade
index 8b9d3e1..17df481 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/logger/log4j2.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'logger'
 -var model = 'backupItem.logger.Log4j2'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade
index 3f3661c..9316e79 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/marshaller.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'marshaller'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade
index 2b0dfd6..d5212a9 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/metrics.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'metrics'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade
index dd46d2a..101a11c 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/odbc.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'odbcConfiguration'
 -var model = 'backupItem.odbc'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
index 0a51cf7..85ec073 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/ssl.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'sslConfiguration'
 -var cluster = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade
index c76391c..8071f3c 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/swap.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'swap'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade
index 9669a87..d2bea86 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/thread.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'pools'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade
index 813948d..bcb1cb7 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/time.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'time'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade
index 2946158..6a18bb0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/clusters/transactions.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'transactions'
 -var model = 'backupItem.transactionConfiguration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade
index bf21ee1..40759e3 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/domains/general.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade
index 45051b2..8ed7b68 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/domains/query.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'query'
 -var model = 'backupItem'
@@ -41,15 +41,15 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition)
         .col-xs-8.col-sm-8.col-md-8
             label.fieldSep /
             .input-tip
-                button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0')
+                button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ignite-on-enter-focus-move='{{::"#{direction}S" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset(false)' tabindex='0')
         .col-xs-4.col-sm-4.col-md-4
             +btn-save(btnVisible, btnSave)
             .input-tip
-                button.select-toggle.form-control(id='{{::"#{direction}" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0')
+                button.select-toggle.form-control(id='{{::"#{direction}" + #{idAddition}}}' ng-model=directionModel bs-select bs-options='item.value as item.label for item in {{sortDirections}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)' tabindex='0')
     .col-xs-12(ng-if='!(#{sortAvailable})')
         +btn-save(btnVisible, btnSave)
         .input-tip
-            button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose index field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset()' tabindex='0')
+            button.select-toggle.form-control(id='{{::"#{fieldName}" + #{idAddition}}}' ng-model=fieldNameModel placeholder='{{fields("#{prefix}", #{fieldNameModel}).length > 0 ? "Choose index field" : "No fields configured"}}' bs-select bs-options='item.value as item.label for item in fields("#{prefix}", #{fieldNameModel})' ng-disabled='fields("#{prefix}", #{fieldNameModel}).length === 0' ignite-on-escape='tableReset(false)' tabindex='0')
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle)
@@ -137,11 +137,11 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition)
                                                     .col-sm-7
                                                         label.fieldSep /
                                                         .input-tip
-                                                            input.form-control(id='curIndexName{{$index}}' type='text' ignite-on-enter-focus-move='curIndexType{{$index}}' ng-model='indexesTbl.curIndexName' placeholder='Index name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+                                                            input.form-control(id='curIndexName{{$index}}' type='text' ignite-on-enter-focus-move='curIndexType{{$index}}' ng-model='indexesTbl.curIndexName' placeholder='Index name' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
                                                     .col-sm-5
                                                         +btn-save('tableIndexSaveVisible(indexesTbl, $index)', 'tableIndexSave(indexesTbl, $index)')
                                                         .input-tip
-                                                            button.select-toggle.form-control(id='curIndexType{{$index}}' bs-select ng-model='indexesTbl.curIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+                                                            button.select-toggle.form-control(id='curIndexType{{$index}}' bs-select ng-model='indexesTbl.curIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
                                                 .margin-left-dflt
                                                     table.links-edit-sub(st-table='item.fields' ng-init='itemIndex = $index')
                                                         tbody
@@ -163,10 +163,10 @@ mixin table-index-item-edit(prefix, index, sortAvailable, idAddition)
                                                 .col-sm-7
                                                     .fieldSep /
                                                     .input-tip
-                                                        input#newIndexName.form-control(type='text' ignite-on-enter-focus-move='newIndexType' ng-model='indexesTbl.newIndexName' placeholder='Index name' ignite-on-enter='tableIndexSaveVisible(indexesTbl, -1) && tableIndexSave(indexesTbl, -1)' ignite-on-escape='tableReset()')
+                                                        input#newIndexName.form-control(type='text' ignite-on-enter-focus-move='newIndexType' ng-model='indexesTbl.newIndexName' placeholder='Index name' ignite-on-enter='tableIndexSaveVisible(indexesTbl, -1) && tableIndexSave(indexesTbl, -1)' ignite-on-escape='tableReset(false)')
                                                 .col-sm-5
                                                     +btn-save('tableIndexSaveVisible(indexesTbl, -1)', 'tableIndexSave(indexesTbl, -1)')
                                                     .input-tip
-                                                        button#newIndexType.select-toggle.form-control(bs-select ng-model='indexesTbl.newIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+                                                        button#newIndexType.select-toggle.form-control(bs-select ng-model='indexesTbl.newIndexType' data-placeholder='Select index type' bs-options='item.value as item.label for item in indexType' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
             .col-sm-6
                 +preview-xml-java(model, 'domainModelQuery')


[12/19] ignite git commit: ignite-4332 Usage of cache.getEntry inside GridCacheQueryManager.runQuery causes to remote operations

Posted by sb...@apache.org.
ignite-4332 Usage of cache.getEntry inside GridCacheQueryManager.runQuery causes to remote operations


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

Branch: refs/heads/master
Commit: c06e4017771603df7118974758d3d6b9cadc41b5
Parents: 1a2de51
Author: Eduard Shangareev <es...@gridgain.com>
Authored: Wed Nov 30 14:34:47 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Nov 30 14:34:47 2016 +0300

----------------------------------------------------------------------
 .../cache/query/GridCacheQueryManager.java        | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c06e4017/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 ab8bd63..d4decb4 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
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheEntryImpl;
 import org.apache.ignite.internal.processors.cache.CacheMetricsImpl;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
@@ -1560,9 +1561,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         metrics.addGetTimeNanos(System.nanoTime() - start);
                     }
 
+                    K key0 = null;
+                    V val0 = null;
+
                     if (readEvt) {
-                        K key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary());
-                        V val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary());
+                        key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary());
+                        val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary());
 
                         switch (type) {
                             case SQL:
@@ -1631,12 +1635,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     }
 
                     if (rdc != null || trans != null) {
-                        Cache.Entry<K, V> entry;
+                        if (key0 == null)
+                            key0 = (K)cctx.unwrapBinaryIfNeeded(key, qry.keepBinary());
+                        if (val0 == null)
+                            val0 = (V)cctx.unwrapBinaryIfNeeded(val, qry.keepBinary());
 
-                        if (qry.keepBinary())
-                            entry = cache.<K, V>keepBinary().getEntry(key);
-                        else
-                            entry = cache.<K, V>getEntry(key);
+                        Cache.Entry<K, V> entry = new CacheEntryImpl(key0, val0);
 
                         // Reduce.
                         if (rdc != null) {


[15/19] ignite git commit: Updated classnames.properties for running nodes in IDE.

Posted by sb...@apache.org.
Updated classnames.properties for running nodes in IDE.


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

Branch: refs/heads/master
Commit: 12bdd6a031a33eda004a66e73cee9628f073ed68
Parents: e9ace77
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 2 16:09:29 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 2 16:09:29 2016 +0700

----------------------------------------------------------------------
 .../resources/META-INF/classnames.properties    |  86 ++++++++++----
 .../resources/META-INF/classnames.properties    | 114 +++++++++++++++++++
 2 files changed, 176 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/12bdd6a0/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index 71d236f..4c9596c 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -75,11 +75,13 @@ org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy$HolderComparator
 org.apache.ignite.cache.query.CacheQueryEntryEvent
 org.apache.ignite.cache.query.ContinuousQuery
 org.apache.ignite.cache.query.Query
+org.apache.ignite.cache.query.QueryCancelledException
 org.apache.ignite.cache.query.ScanQuery
 org.apache.ignite.cache.query.SpiQuery
 org.apache.ignite.cache.query.SqlFieldsQuery
 org.apache.ignite.cache.query.SqlQuery
 org.apache.ignite.cache.query.TextQuery
+org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$2
 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$1
 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$EntryMapping$2
 org.apache.ignite.cache.store.jdbc.CacheAbstractJdbcStore$TypeKind
@@ -366,6 +368,7 @@ org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakQu
 org.apache.ignite.internal.processors.cache.CacheWeakQueryIteratorsHolder$WeakReferenceCloseableIterator
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeBatch
 org.apache.ignite.internal.processors.cache.DynamicCacheChangeRequest
+org.apache.ignite.internal.processors.cache.EntryProcessorResourceInjectorProxy
 org.apache.ignite.internal.processors.cache.GridCacheAdapter
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$10
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$11
@@ -520,10 +523,8 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$15
 org.apache.ignite.internal.processors.cache.GridCacheUtils$16
 org.apache.ignite.internal.processors.cache.GridCacheUtils$17
 org.apache.ignite.internal.processors.cache.GridCacheUtils$18
-org.apache.ignite.internal.processors.cache.GridCacheUtils$19
 org.apache.ignite.internal.processors.cache.GridCacheUtils$2
 org.apache.ignite.internal.processors.cache.GridCacheUtils$20
-org.apache.ignite.internal.processors.cache.GridCacheUtils$22
 org.apache.ignite.internal.processors.cache.GridCacheUtils$3
 org.apache.ignite.internal.processors.cache.GridCacheUtils$4
 org.apache.ignite.internal.processors.cache.GridCacheUtils$5
@@ -533,6 +534,7 @@ org.apache.ignite.internal.processors.cache.GridCacheUtils$8
 org.apache.ignite.internal.processors.cache.GridCacheUtils$9
 org.apache.ignite.internal.processors.cache.GridCacheValueCollection
 org.apache.ignite.internal.processors.cache.GridCacheValueCollection$1
+org.apache.ignite.internal.processors.cache.GridDeferredAckMessageSender$DeferredAckMessageBuffer
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$1
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$10
@@ -545,12 +547,13 @@ org.apache.ignite.internal.processors.cache.IgniteCacheProxy$8
 org.apache.ignite.internal.processors.cache.IgniteCacheProxy$9
 org.apache.ignite.internal.processors.cache.KeyCacheObject
 org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl
+org.apache.ignite.internal.processors.cache.QueryCursorImpl$State
 org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityProxy
 org.apache.ignite.internal.processors.cache.binary.BinaryMetadataKey
 org.apache.ignite.internal.processors.cache.binary.CacheDefaultBinaryAffinityKeyMapper
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$1
-org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$4
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$5
+org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$6
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataEntryFilter
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetaDataPredicate
 org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl$MetadataProcessor
@@ -630,6 +633,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocal$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxLocalAdapter$2
+org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxOnePhaseCommitAckRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareFuture$3
@@ -648,6 +652,7 @@ org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedGetFu
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.GridPartitionedSingleGetFuture$3
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$10
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$11
@@ -670,26 +675,30 @@ org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomic
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$27
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$28
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$29
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$3
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$30
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$31
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$4
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$6
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$7
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$8
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$9
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$DeferredResponseBuffer
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicDeferredUpdateResponse
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicSingleUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicUpdateResponse
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractSingleUpdateRequest
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture$1
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateRequest
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFilterRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$2
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture$3
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateInvokeRequest
+org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$1
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$2
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$3
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture$4
-org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicFullUpdateRequest
 org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache
 org.apache.ignite.internal.processors.cache.distributed.dht.colocated.GridDhtColocatedCache$2
@@ -760,6 +769,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$3
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$4
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$LockTimeoutObject$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockRequest
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearLockResponse
@@ -772,6 +782,7 @@ org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticS
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$2
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$3
+org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$4
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFuture$MiniFuture$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearOptimisticTxPrepareFutureAdapter$1
 org.apache.ignite.internal.processors.cache.distributed.near.GridNearPessimisticTxPrepareFuture$1
@@ -821,20 +832,21 @@ org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManag
 org.apache.ignite.internal.processors.cache.query.GridCacheDistributedQueryManager$7
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryAdapter$ScanQueryFallbackClosableIterator
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryDetailMetricsAdapter
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$1
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryFutureAdapter$2
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$1$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$11
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$12
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$13
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$14
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15$1
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$2
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$3
-org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$15
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$16
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$17
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$18$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$1
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$4$2
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$5
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$6
+org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$7
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$8
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$9
 org.apache.ignite.internal.processors.cache.query.GridCacheQueryManager$CacheSqlIndexMetadata
@@ -896,6 +908,8 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$12
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$13
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$14
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$15
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$16
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$17
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxHandler$4
@@ -929,9 +943,9 @@ org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$Po
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure1$4
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostLockClosure2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter$PostMissClosure
-org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$2
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$3
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$4
+org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$5
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommitListener
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$CommittedVersion
 org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager$NodeFailureTimeoutObject$1
@@ -998,6 +1012,7 @@ org.apache.ignite.internal.processors.continuous.StopRoutineAckDiscoveryMessage
 org.apache.ignite.internal.processors.continuous.StopRoutineDiscoveryMessage
 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$3
 org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$4
+org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor$5
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Batched
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$BatchedSorted
 org.apache.ignite.internal.processors.datastreamer.DataStreamerCacheUpdaters$Individual
@@ -1005,9 +1020,9 @@ org.apache.ignite.internal.processors.datastreamer.DataStreamerEntry
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$1
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$4
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$5
-org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$6
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$1
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$2
+org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$Buffer$3
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DataStreamerPda
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$DefaultIoPolicyResolver
 org.apache.ignite.internal.processors.datastreamer.DataStreamerImpl$IsolatedUpdater
@@ -1084,6 +1099,7 @@ org.apache.ignite.internal.processors.datastructures.GridCacheSetProxy
 org.apache.ignite.internal.processors.datastructures.GridSetQueryPredicate
 org.apache.ignite.internal.processors.dr.GridDrType
 org.apache.ignite.internal.processors.dr.IgniteDrDataStreamerCacheUpdater
+org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo
 org.apache.ignite.internal.processors.hadoop.HadoopFileBlock
 org.apache.ignite.internal.processors.hadoop.HadoopInputSplit
 org.apache.ignite.internal.processors.hadoop.HadoopJobId
@@ -1119,7 +1135,6 @@ org.apache.ignite.internal.processors.igfs.IgfsFragmentizerManager$IdentityHashS
 org.apache.ignite.internal.processors.igfs.IgfsFragmentizerRequest
 org.apache.ignite.internal.processors.igfs.IgfsFragmentizerResponse
 org.apache.ignite.internal.processors.igfs.IgfsHandshakeResponse
-org.apache.ignite.internal.processors.igfs.IgfsImpl$12$1
 org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask
 org.apache.ignite.internal.processors.igfs.IgfsImpl$IgfsGlobalSpaceTask$1
 org.apache.ignite.internal.processors.igfs.IgfsInputStreamDescriptor
@@ -1155,6 +1170,7 @@ org.apache.ignite.internal.processors.igfs.client.IgfsClientSummaryCallable
 org.apache.ignite.internal.processors.igfs.client.IgfsClientUpdateCallable
 org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaIdsForPathCallable
 org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaInfoForPathCallable
+org.apache.ignite.internal.processors.igfs.client.meta.IgfsClientMetaUnlockCallable
 org.apache.ignite.internal.processors.igfs.data.IgfsDataPutProcessor
 org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryCreateProcessor
 org.apache.ignite.internal.processors.igfs.meta.IgfsMetaDirectoryListingAddProcessor
@@ -1172,6 +1188,8 @@ org.apache.ignite.internal.processors.igfs.meta.IgfsMetaUpdateTimesProcessor
 org.apache.ignite.internal.processors.job.GridJobProcessor$5
 org.apache.ignite.internal.processors.job.GridJobWorker$3
 org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor$SnapshotReducer
+org.apache.ignite.internal.processors.odbc.OdbcProtocolVersion
+org.apache.ignite.internal.processors.odbc.escape.OdbcEscapeType
 org.apache.ignite.internal.processors.platform.PlatformAbstractConfigurationClosure
 org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate
 org.apache.ignite.internal.processors.platform.PlatformEventFilterListener
@@ -1180,7 +1198,8 @@ org.apache.ignite.internal.processors.platform.PlatformExtendedException
 org.apache.ignite.internal.processors.platform.PlatformJavaObjectFactoryProxy
 org.apache.ignite.internal.processors.platform.PlatformNativeException
 org.apache.ignite.internal.processors.platform.PlatformNoCallbackException
-org.apache.ignite.internal.processors.platform.cache.PlatformCache$1
+org.apache.ignite.internal.processors.platform.PlatformProcessorImpl$1
+org.apache.ignite.internal.processors.platform.cache.PlatformCache$5
 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilter
 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryFilterImpl
 org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor
@@ -1211,6 +1230,8 @@ org.apache.ignite.internal.processors.platform.cpp.PlatformCppConfigurationClosu
 org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiver
 org.apache.ignite.internal.processors.platform.datastreamer.PlatformStreamReceiverImpl
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$1
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$10
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$11
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$2
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$3
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$4
@@ -1218,6 +1239,7 @@ org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$5
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$6
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$7
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$8
+org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetCacheStore$9
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetConfigurationClosure
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService
 org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl
@@ -1232,19 +1254,24 @@ org.apache.ignite.internal.processors.platform.transactions.PlatformTransactions
 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$1
 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$FutureListenable$1
 org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils$InternalFutureListenable$1
+org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionLockProcessor
+org.apache.ignite.internal.processors.platform.websession.PlatformDotNetSessionSetAndUnlockProcessor
 org.apache.ignite.internal.processors.query.GridQueryFieldMetadata
 org.apache.ignite.internal.processors.query.GridQueryIndexType
-org.apache.ignite.internal.processors.query.GridQueryProcessor$2
 org.apache.ignite.internal.processors.query.GridQueryProcessor$3
 org.apache.ignite.internal.processors.query.GridQueryProcessor$4
 org.apache.ignite.internal.processors.query.GridQueryProcessor$5
 org.apache.ignite.internal.processors.query.GridQueryProcessor$6
+org.apache.ignite.internal.processors.query.GridQueryProcessor$7
+org.apache.ignite.internal.processors.query.GridQueryProcessor$8
 org.apache.ignite.internal.processors.query.GridQueryProcessor$IndexType
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryCancelRequest
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryFailResponse
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageRequest
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryNextPageResponse
 org.apache.ignite.internal.processors.query.h2.twostep.messages.GridQueryRequest
+org.apache.ignite.internal.processors.resource.GridResourceIoc$AnnotationSet
+org.apache.ignite.internal.processors.resource.GridResourceIoc$ResourceAnnotation
 org.apache.ignite.internal.processors.rest.GridRestCommand
 org.apache.ignite.internal.processors.rest.GridRestProcessor$2$1
 org.apache.ignite.internal.processors.rest.GridRestProcessor$3
@@ -1332,7 +1359,7 @@ org.apache.ignite.internal.processors.service.ServiceDescriptorImpl
 org.apache.ignite.internal.processors.task.GridTaskProcessor$1
 org.apache.ignite.internal.processors.task.GridTaskThreadContextKey
 org.apache.ignite.internal.processors.task.GridTaskWorker$3
-org.apache.ignite.internal.processors.task.GridTaskWorker$4
+org.apache.ignite.internal.processors.task.GridTaskWorker$5
 org.apache.ignite.internal.processors.task.GridTaskWorker$State
 org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException
 org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException
@@ -1356,6 +1383,7 @@ org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap
 org.apache.ignite.internal.util.GridBoundedConcurrentOrderedSet
 org.apache.ignite.internal.util.GridBoundedLinkedHashMap
 org.apache.ignite.internal.util.GridBoundedLinkedHashSet
+org.apache.ignite.internal.util.GridBoundedPriorityQueue
 org.apache.ignite.internal.util.GridByteArrayList
 org.apache.ignite.internal.util.GridCloseableIteratorAdapter
 org.apache.ignite.internal.util.GridCloseableIteratorAdapterEx
@@ -1407,14 +1435,14 @@ org.apache.ignite.internal.util.IgniteUtils$11
 org.apache.ignite.internal.util.IgniteUtils$12
 org.apache.ignite.internal.util.IgniteUtils$13
 org.apache.ignite.internal.util.IgniteUtils$14
-org.apache.ignite.internal.util.IgniteUtils$16
-org.apache.ignite.internal.util.IgniteUtils$2
-org.apache.ignite.internal.util.IgniteUtils$22
+org.apache.ignite.internal.util.IgniteUtils$15
+org.apache.ignite.internal.util.IgniteUtils$17
 org.apache.ignite.internal.util.IgniteUtils$23
 org.apache.ignite.internal.util.IgniteUtils$24
 org.apache.ignite.internal.util.IgniteUtils$25
 org.apache.ignite.internal.util.IgniteUtils$26
 org.apache.ignite.internal.util.IgniteUtils$27
+org.apache.ignite.internal.util.IgniteUtils$28
 org.apache.ignite.internal.util.IgniteUtils$3
 org.apache.ignite.internal.util.IgniteUtils$4
 org.apache.ignite.internal.util.IgniteUtils$5
@@ -1647,12 +1675,19 @@ org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask
 org.apache.ignite.internal.visor.cache.VisorCachePartitionsTask$VisorCachePartitionsJob
 org.apache.ignite.internal.visor.cache.VisorCacheQueryConfiguration
 org.apache.ignite.internal.visor.cache.VisorCacheQueryConfigurationV2
+org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetrics
+org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask
+org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask$VisorCacheQueryDetailMetricsCollectorJob
 org.apache.ignite.internal.visor.cache.VisorCacheQueryMetrics
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceConfiguration
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask
 org.apache.ignite.internal.visor.cache.VisorCacheRebalanceTask$VisorCachesRebalanceJob
 org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask
 org.apache.ignite.internal.visor.cache.VisorCacheResetMetricsTask$VisorCacheResetMetricsJob
+org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask
+org.apache.ignite.internal.visor.cache.VisorCacheResetQueryDetailMetricsTask$VisorCacheResetQueryDetailMetricsJob
+org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask
+org.apache.ignite.internal.visor.cache.VisorCacheResetQueryMetricsTask$VisorCacheResetQueryMetricsJob
 org.apache.ignite.internal.visor.cache.VisorCacheStartTask
 org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartArg
 org.apache.ignite.internal.visor.cache.VisorCacheStartTask$VisorCacheStartJob
@@ -1682,6 +1717,7 @@ org.apache.ignite.internal.visor.debug.VisorThreadLockInfo
 org.apache.ignite.internal.visor.debug.VisorThreadMonitorInfo
 org.apache.ignite.internal.visor.event.VisorGridDeploymentEvent
 org.apache.ignite.internal.visor.event.VisorGridDiscoveryEvent
+org.apache.ignite.internal.visor.event.VisorGridDiscoveryEventV2
 org.apache.ignite.internal.visor.event.VisorGridEvent
 org.apache.ignite.internal.visor.event.VisorGridEventsLost
 org.apache.ignite.internal.visor.event.VisorGridJobEvent
@@ -1769,6 +1805,7 @@ org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException
 org.apache.ignite.internal.visor.util.VisorEventMapper
 org.apache.ignite.internal.visor.util.VisorExceptionWrapper
 org.apache.ignite.internal.visor.util.VisorTaskUtils$4
+org.apache.ignite.internal.visor.util.VisorTaskUtils$5
 org.apache.ignite.internal.websession.WebSessionAttributeProcessor
 org.apache.ignite.internal.websession.WebSessionEntity
 org.apache.ignite.lang.IgniteBiClosure
@@ -1798,6 +1835,7 @@ org.apache.ignite.plugin.PluginNotFoundException
 org.apache.ignite.plugin.PluginValidationException
 org.apache.ignite.plugin.extensions.communication.Message
 org.apache.ignite.plugin.extensions.communication.MessageCollectionItemType
+org.apache.ignite.plugin.security.SecurityBasicPermissionSet
 org.apache.ignite.plugin.security.SecurityCredentials
 org.apache.ignite.plugin.security.SecurityException
 org.apache.ignite.plugin.security.SecurityPermission

http://git-wip-us.apache.org/repos/asf/ignite/blob/12bdd6a0/modules/hadoop/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/resources/META-INF/classnames.properties b/modules/hadoop/src/main/resources/META-INF/classnames.properties
new file mode 100644
index 0000000..0ac17cf
--- /dev/null
+++ b/modules/hadoop/src/main/resources/META-INF/classnames.properties
@@ -0,0 +1,114 @@
+#
+# 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.
+#
+
+org.apache.ignite.hadoop.fs.BasicHadoopFileSystemFactory
+org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory
+org.apache.ignite.hadoop.fs.HadoopFileSystemFactory
+org.apache.ignite.hadoop.fs.KerberosHadoopFileSystemFactory
+org.apache.ignite.hadoop.mapreduce.IgniteHadoopWeightedMapReducePlanner$MapperPriority
+org.apache.ignite.hadoop.util.BasicUserNameMapper
+org.apache.ignite.hadoop.util.ChainedUserNameMapper
+org.apache.ignite.hadoop.util.KerberosUserNameMapper
+org.apache.ignite.hadoop.util.KerberosUserNameMapper$State
+org.apache.ignite.hadoop.util.UserNameMapper
+org.apache.ignite.internal.processors.hadoop.HadoopAttributes
+org.apache.ignite.internal.processors.hadoop.HadoopDefaultJobInfo
+org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit
+org.apache.ignite.internal.processors.hadoop.HadoopFileBlock
+org.apache.ignite.internal.processors.hadoop.HadoopInputSplit
+org.apache.ignite.internal.processors.hadoop.HadoopJobId
+org.apache.ignite.internal.processors.hadoop.HadoopJobInfo
+org.apache.ignite.internal.processors.hadoop.HadoopJobPhase
+org.apache.ignite.internal.processors.hadoop.HadoopJobProperty
+org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
+org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan
+org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper
+org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException
+org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo
+org.apache.ignite.internal.processors.hadoop.HadoopTaskType
+org.apache.ignite.internal.processors.hadoop.counter.HadoopCounterAdapter
+org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl
+org.apache.ignite.internal.processors.hadoop.counter.HadoopCountersImpl$CounterKey
+org.apache.ignite.internal.processors.hadoop.counter.HadoopLongCounter
+org.apache.ignite.internal.processors.hadoop.counter.HadoopPerformanceCounter
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsCommunicationException
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$1
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$10
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$11
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$12
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$13
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$14
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$15
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$16
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$2
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$3
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$4
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$5
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$6
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$8
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsInProc$9
+org.apache.ignite.internal.processors.hadoop.impl.igfs.HadoopIgfsOutProc$1
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobMetadata
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$1
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$CancelJobProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$IncrementCountersProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$InitializeReducersProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$JobLocalState$1
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$JobLocalState$2
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$RemoveMappersProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$RemoveReducerProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$StackedProcessor
+org.apache.ignite.internal.processors.hadoop.jobtracker.HadoopJobTracker$UpdatePhaseProcessor
+org.apache.ignite.internal.processors.hadoop.message.HadoopMessage
+org.apache.ignite.internal.processors.hadoop.planner.HadoopDefaultMapReducePlan
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobCountersTask
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobStatusTask
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolJobStatusTask$1
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolKillJobTask
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolNextTaskIdTask
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolSubmitJobTask
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskAdapter
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskAdapter$Job
+org.apache.ignite.internal.processors.hadoop.proto.HadoopProtocolTaskArguments
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffle$1
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffle$2
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleJob$4
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage
+org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap$State
+org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskState
+org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopTaskStatus
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$2
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$4
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$HadoopProcess
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopExternalTaskExecutor$HadoopProcess$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopJobInfoUpdateRequest
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopPrepareForJobRequest
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessDescriptor
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopProcessStartedAck
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopTaskExecutionRequest
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.HadoopTaskFinishedMessage
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$2$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$3
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopChildProcessRunner$MessageListener$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child.HadoopExternalProcessStarter$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$HandshakeAndBackpressureFilter$1
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopExternalCommunication$ProcessHandshakeMessage
+org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.HadoopHandshakeTimeoutException


[16/19] ignite git commit: ignite-4285 For serializable txs allow multiple threads to get read lock for the same key

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
index 59f9a9d..f46b290 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
@@ -95,14 +95,14 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
             version(1),
             123,
             version(2),
-            123,
             /*local*/false,
             /*reentry*/false,
             true,
             false,
             false,
             false,
-            null
+            null,
+            false
         );
 
         Marshaller marshaller = getTestResources().getMarshaller();
@@ -129,14 +129,14 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver4 = version(4);
         GridCacheVersion ver5 = version(5);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
 
         Collection<GridCacheMvccCandidate> cands = entry.remoteMvccSnapshot();
 
         assert cands.size() == 1;
         assert cands.iterator().next().version().equals(ver1);
 
-        entry.addRemote(node2, 5, ver5, 0, false, true);
+        entry.addRemote(node2, 5, ver5, false, true);
 
         cands = entry.remoteMvccSnapshot();
 
@@ -147,7 +147,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         // Check order.
         checkOrder(cands, ver1, ver5);
 
-        entry.addRemote(node1, 3, ver3, 0, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
 
         cands = entry.remoteMvccSnapshot();
 
@@ -162,7 +162,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         checkDone(entry.candidate(ver3));
 
-        entry.addRemote(node1, 2, ver2, 0, false, true);
+        entry.addRemote(node1, 2, ver2, false, true);
 
         cands = entry.remoteMvccSnapshot();
 
@@ -197,7 +197,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         checkDone(entry.candidate(ver5));
 
-        entry.addRemote(node1, 4, ver4, 0, false, true);
+        entry.addRemote(node1, 4, ver4, false, true);
 
         cands = entry.remoteMvccSnapshot();
 
@@ -278,10 +278,10 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver3 = version(3);
         GridCacheVersion ver4 = version(4);
 
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
-        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true);
-        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
+        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true);
+        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true);
 
         GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c1, c2, c3, c4};
 
@@ -322,12 +322,12 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver5 = version(5);
         GridCacheVersion ver6 = version(6);
 
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
-        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true);
-        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true);
-        GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, 0, false, true);
-        GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, 0, false, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
+        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true);
+        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true);
+        GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, false, true);
+        GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, false, true);
 
         GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c1, c2, c3, c4, c5, c6};
 
@@ -369,13 +369,13 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver5 = version(5);
         GridCacheVersion ver6 = version(6);
 
-        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, 0, false, true);
-        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, 0, false, true);
-        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, 0, false, true);
-        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, 0, false, true);
-        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, 0, false, true);
-        GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, 0, false, true);
-        GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, 0, false, true);
+        GridCacheMvccCandidate c0 = entry.addRemote(node1, 1, ver0, false, true);
+        GridCacheMvccCandidate c1 = entry.addRemote(node1, 1, ver1, false, true);
+        GridCacheMvccCandidate c2 = entry.addRemote(node1, 1, ver2, false, true);
+        GridCacheMvccCandidate c3 = entry.addRemote(node1, 1, ver3, false, true);
+        GridCacheMvccCandidate c4 = entry.addRemote(node1, 1, ver4, false, true);
+        GridCacheMvccCandidate c5 = entry.addRemote(node1, 1, ver5, false, true);
+        GridCacheMvccCandidate c6 = entry.addRemote(node1, 1, ver6, false, true);
 
         GridCacheMvccCandidate[] candArr = new GridCacheMvccCandidate[] {c0, c1, c2, c3, c4, c5, c6};
 
@@ -486,7 +486,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver2 = version(2);
         GridCacheVersion ver3 = version(3);
 
-        entry.addRemote(nodeId, 1, ver2, 0, false, true);
+        entry.addRemote(nodeId, 1, ver2, false, true);
 
         entry.addLocal(3, ver3, 0, false, true);
 
@@ -529,15 +529,15 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
         GridCacheVersion ver8 = version(8);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node2, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node2, 7, ver7, 0, false, true);
-        entry.addRemote(node2, 8, ver8, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node2, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node2, 7, ver7, false, true);
+        entry.addRemote(node2, 8, ver8, false, true);
 
-        GridCacheMvccCandidate doomed = entry.addRemote(node2, 6, ver6, 0, false, true);
+        GridCacheMvccCandidate doomed = entry.addRemote(node2, 6, ver6, false, true);
 
         // No reordering happens.
         checkOrder(entry.remoteMvccSnapshot(), ver1, ver2, ver3, ver4, ver5, ver7, ver8, ver6);
@@ -581,13 +581,13 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node2, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node2, 6, ver6, 0, false, true);
-        entry.addRemote(node2, 7, ver7, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node2, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node2, 6, ver6, false, true);
+        entry.addRemote(node2, 7, ver7, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver4, ver6, ver2);
 
@@ -623,13 +623,13 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node2, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node2, 6, ver6, 0, false, true);
-        entry.addRemote(node2, 7, ver7, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node2, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node2, 6, ver6, false, true);
+        entry.addRemote(node2, 7, ver7, false, true);
 
         List<GridCacheVersion> completed = Arrays.asList(ver4, ver6);
 
@@ -669,11 +669,11 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, false);
-        entry.addRemote(node2, 2, ver2, 0, false, false);
-        entry.addRemote(node1, 3, ver3, 0, false, false);
-        entry.addRemote(node2, 4, ver4, 0, false, false);
-        entry.addRemote(node1, 5, ver5, 0, false, false);
+        entry.addRemote(node1, 1, ver1, false, false);
+        entry.addRemote(node2, 2, ver2, false, false);
+        entry.addRemote(node1, 3, ver3, false, false);
+        entry.addRemote(node2, 4, ver4, false, false);
+        entry.addRemote(node1, 5, ver5, false, false);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver7);
 
@@ -708,13 +708,13 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, false);
-        entry.addRemote(node2, 2, ver2, 0, false, false);
-        entry.addRemote(node1, 3, ver3, 0, false, false);
-        entry.addRemote(node2, 4, ver4, 0, false, false);
-        entry.addRemote(node1, 5, ver5, 0, false, false);
-        entry.addRemote(node2, 6, ver6, 0, false, false);
-        entry.addRemote(node2, 7, ver7, 0, false, false);
+        entry.addRemote(node1, 1, ver1, false, false);
+        entry.addRemote(node2, 2, ver2, false, false);
+        entry.addRemote(node1, 3, ver3, false, false);
+        entry.addRemote(node2, 4, ver4, false, false);
+        entry.addRemote(node1, 5, ver5, false, false);
+        entry.addRemote(node2, 6, ver6, false, false);
+        entry.addRemote(node2, 7, ver7, false, false);
 
         List<GridCacheVersion> committed = Arrays.asList(ver4, ver6, ver3);
 
@@ -750,11 +750,11 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, false);
-        entry.addRemote(node2, 2, ver2, 0, false, false);
-        entry.addRemote(node1, 3, ver3, 0, false, false);
-        entry.addRemote(node2, 4, ver4, 0, false, false);
-        entry.addRemote(node1, 5, ver5, 0, false, false);
+        entry.addRemote(node1, 1, ver1, false, false);
+        entry.addRemote(node2, 2, ver2, false, false);
+        entry.addRemote(node1, 3, ver3, false, false);
+        entry.addRemote(node2, 4, ver4, false, false);
+        entry.addRemote(node1, 5, ver5, false, false);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver7);
 
@@ -789,11 +789,11 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver6 = version(6);
         GridCacheVersion ver7 = version(7);
 
-        entry.addRemote(node1, 1, ver1, 0, false, false);
-        entry.addRemote(node2, 2, ver2, 0, false, false);
-        entry.addRemote(node1, 3, ver3, 0, false, false);
-        entry.addRemote(node2, 4, ver4, 0, false, false);
-        entry.addRemote(node1, 5, ver5, 0, false, false);
+        entry.addRemote(node1, 1, ver1, false, false);
+        entry.addRemote(node2, 2, ver2, false, false);
+        entry.addRemote(node1, 3, ver3, false, false);
+        entry.addRemote(node2, 4, ver4, false, false);
+        entry.addRemote(node1, 5, ver5, false, false);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver7);
 
@@ -829,12 +829,12 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
 
         // Don't add version 2.
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node2, 6, ver6, 0, false, true);
-        entry.addRemote(node2, 7, ver7, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node2, 6, ver6, false, true);
+        entry.addRemote(node2, 7, ver7, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver4);
 
@@ -870,9 +870,9 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
 
         // Don't add versions 2, 5, 6, 7.
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver5, ver7);
 
@@ -905,12 +905,12 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
 
         // Don't add version 1.
-        entry.addRemote(node1, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node2, 6, ver6, 0, false, true);
-        entry.addRemote(node2, 7, ver7, 0, false, true);
+        entry.addRemote(node1, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node2, 6, ver6, false, true);
+        entry.addRemote(node2, 7, ver7, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver4, ver6, ver3);
 
@@ -946,12 +946,12 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
 
         // Don't add version 6, 7
-        entry.addRemote(node1, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
-        entry.addRemote(node1, 5, ver5, 0, false, true);
-        entry.addRemote(node1, 6, ver6, 0, false, true);
-        entry.addRemote(node1, 7, ver7, 0, false, true);
+        entry.addRemote(node1, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
+        entry.addRemote(node1, 5, ver5, false, true);
+        entry.addRemote(node1, 6, ver6, false, true);
+        entry.addRemote(node1, 7, ver7, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver2, ver3);
 
@@ -987,10 +987,10 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver7 = version(7);
 
         // Don't add version 5, 6, 7
-        entry.addRemote(node1, 1, ver1, 0, false, true);
-        entry.addRemote(node1, 2, ver2, 0, false, true);
-        entry.addRemote(node1, 3, ver3, 0, false, true);
-        entry.addRemote(node2, 4, ver4, 0, false, true);
+        entry.addRemote(node1, 1, ver1, false, true);
+        entry.addRemote(node1, 2, ver2, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
+        entry.addRemote(node2, 4, ver4, false, true);
 
         List<GridCacheVersion> committed = Arrays.asList(ver6, ver7);
 
@@ -1021,7 +1021,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         GridCacheVersion ver4 = version(4);
         GridCacheVersion ver5 = version(5);
 
-        entry.addRemote(node1, 1, ver1, 0, false, false);
+        entry.addRemote(node1, 1, ver1, false, false);
         entry.addLocal(2, ver2, 0, true, true);
 
         Collection<GridCacheMvccCandidate> cands = entry.remoteMvccSnapshot();
@@ -1029,7 +1029,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         assert cands.size() == 1;
         assert cands.iterator().next().version().equals(ver1);
 
-        entry.addRemote(node2, 5, ver5, 0, false, false);
+        entry.addRemote(node2, 5, ver5, false, false);
 
         cands = entry.remoteMvccSnapshot();
 
@@ -1040,7 +1040,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         checkOrder(cands, ver1, ver5);
         checkOrder(entry.localCandidates(true), ver2);
 
-        entry.addRemote(node1, 3, ver3, 0, false, true);
+        entry.addRemote(node1, 3, ver3, false, true);
         entry.addLocal(4, ver4, 0, /*reenter*/true, false);
 
         cands = entry.remoteMvccSnapshot();
@@ -1174,11 +1174,11 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         linkCandidates(ctx, c13, c33);
 
-        entry2.addRemote(nodeId, 3, ver2, 0, false, true);
+        entry2.addRemote(nodeId, 3, ver2, false, true);
 
         checkLocal(entry2.candidate(ver1), ver1, true, false, false);
 
-        entry3.addRemote(nodeId, 3, ver2, 0, false, false);
+        entry3.addRemote(nodeId, 3, ver2, false, false);
 
         entry3.readyLocal(ver3);
 
@@ -1245,11 +1245,11 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         linkCandidates(ctx, c13, c33);
 
-        entry2.addRemote(UUID.randomUUID(), 3, ver1, 0, false, true);
+        entry2.addRemote(UUID.randomUUID(), 3, ver1, false, true);
 
         checkLocal(entry2.candidate(ver2), ver2, true, false, false);
 
-        entry3.addRemote(UUID.randomUUID(), 3, ver1, 0, false, true);
+        entry3.addRemote(UUID.randomUUID(), 3, ver1, false, true);
 
         entry3.readyLocal(ver3);
 
@@ -1421,7 +1421,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         checkLocal(c1k1, ver2, true, true, false);
 
-        GridCacheMvccCandidate c2k1 = entry1.addRemote(id, 2, ver1, 0, false, true);
+        GridCacheMvccCandidate c2k1 = entry1.addRemote(id, 2, ver1, false, true);
 
         // Force recheck of assignments.
         entry1.recheckLock();
@@ -1437,7 +1437,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
         assert c1k2.previous() == c1k1;
 
-        GridCacheMvccCandidate c2k2 = entry2.addRemote(id, 3, ver1, 0, false, true);
+        GridCacheMvccCandidate c2k2 = entry2.addRemote(id, 3, ver1, false, true);
 
         entry2.readyLocal(c1k2);
 
@@ -1479,8 +1479,8 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         checkLocal(v3k2, ver3, false, false, false);
 
         // Remote locks.
-        GridCacheMvccCandidate v2k1 = entry1.addRemote(id, 3, ver2, 0, false, false);
-        GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 3, ver2, 0, false, false);
+        GridCacheMvccCandidate v2k1 = entry1.addRemote(id, 3, ver2, false, false);
+        GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 3, ver2, false, false);
 
         checkRemote(v2k1, ver2, false, false);
         checkRemote(v2k2, ver2, false, false);
@@ -1602,7 +1602,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
         checkLocal(v1k1, ver1, true, false, false);
         checkLocal(v1k2, ver1, true, false, false);
 
-        GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 5, ver2, 0, false, false);
+        GridCacheMvccCandidate v2k2 = entry2.addRemote(id, 5, ver2, false, false);
 
         checkRemote(v2k2, ver2, false, false);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 396629a..48621af 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
@@ -64,24 +64,12 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param ctx Context.
      * @param key Key.
      */
-    public GridCacheTestEntryEx(GridCacheContext ctx, Object key) {
+    GridCacheTestEntryEx(GridCacheContext ctx, Object key) {
         mvcc = new GridCacheMvcc(ctx);
 
         this.key = ctx.toCacheKeyObject(key);
     }
 
-    /**
-     * @param ctx Context.
-     * @param key Key.
-     * @param val Value.
-     */
-    public GridCacheTestEntryEx(GridCacheContext ctx, Object key, Object val) {
-        mvcc = new GridCacheMvcc(ctx);
-
-        this.key = ctx.toCacheKeyObject(key);
-        this.val = ctx.toCacheObject(val);
-    }
-
     /** {@inheritDoc} */
     @Override public int memorySize() throws IgniteCheckedException {
         return 1024;
@@ -146,7 +134,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @return New lock candidate if lock was added, or current owner if lock was reentered,
      *      or <tt>null</tt> if lock was owned by another thread and timeout is negative.
      */
-    @Nullable public GridCacheMvccCandidate addLocal(
+    @Nullable GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
         long timeout,
@@ -159,6 +147,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
             timeout,
             reenter,
             tx,
+            false,
             false
         );
     }
@@ -169,14 +158,13 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param nodeId Node ID.
      * @param threadId Thread ID.
      * @param ver Lock version.
-     * @param timeout Lock acquire timeout.
      * @param ec Not used.
      * @param tx Transaction flag.
      * @return Remote candidate.
      */
-    public GridCacheMvccCandidate addRemote(UUID nodeId, long threadId, GridCacheVersion ver, long timeout,
+    GridCacheMvccCandidate addRemote(UUID nodeId, long threadId, GridCacheVersion ver,
         boolean ec, boolean tx) {
-        return mvcc.addRemote(this, nodeId, null, threadId, ver, timeout, tx, true, false);
+        return mvcc.addRemote(this, nodeId, null, threadId, ver, tx, true, false);
     }
 
     /**
@@ -185,20 +173,19 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param nodeId Node ID.
      * @param threadId Thread ID.
      * @param ver Lock version.
-     * @param timeout Lock acquire timeout.
      * @param tx Transaction flag.
      * @return Remote candidate.
      */
-    public GridCacheMvccCandidate addNearLocal(UUID nodeId, long threadId, GridCacheVersion ver, long timeout,
+    GridCacheMvccCandidate addNearLocal(UUID nodeId, long threadId, GridCacheVersion ver,
         boolean tx) {
-        return mvcc.addNearLocal(this, nodeId, null, threadId, ver, timeout, tx, true);
+        return mvcc.addNearLocal(this, nodeId, null, threadId, ver, tx, true, false);
     }
 
     /**
      *
      * @param baseVer Base version.
      */
-    public void salvageRemote(GridCacheVersion baseVer) {
+    void salvageRemote(GridCacheVersion baseVer) {
         mvcc.salvageRemote(baseVer);
     }
 
@@ -210,17 +197,16 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param baseVer Base version.
      * @param committedVers Committed versions relative to base.
      * @param rolledbackVers Rolled back versions relative to base.
-     * @return Lock owner.
      */
-    @Nullable public GridCacheMvccCandidate orderCompleted(GridCacheVersion baseVer,
+    void orderCompleted(GridCacheVersion baseVer,
         Collection<GridCacheVersion> committedVers, Collection<GridCacheVersion> rolledbackVers) {
-        return mvcc.orderCompleted(baseVer, committedVers, rolledbackVers);
+        mvcc.orderCompleted(baseVer, committedVers, rolledbackVers);
     }
 
     /**
      * @param ver Version.
      */
-    public void doneRemote(GridCacheVersion ver) {
+    void doneRemote(GridCacheVersion ver) {
         mvcc.doneRemote(ver, Collections.<GridCacheVersion>emptyList(),
             Collections.<GridCacheVersion>emptyList(), Collections.<GridCacheVersion>emptyList());
     }
@@ -229,16 +215,15 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param baseVer Base version.
      * @param owned Owned.
      */
-    public void orderOwned(GridCacheVersion baseVer, GridCacheVersion owned) {
+    void orderOwned(GridCacheVersion baseVer, GridCacheVersion owned) {
         mvcc.markOwned(baseVer, owned);
     }
 
     /**
      * @param ver Lock version to acquire or set to ready.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate readyLocal(GridCacheVersion ver) {
-        return mvcc.readyLocal(ver);
+    void readyLocal(GridCacheVersion ver) {
+        mvcc.readyLocal(ver);
     }
 
     /**
@@ -247,44 +232,33 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
      * @param committedVers Committed versions.
      * @param rolledbackVers Rolled back versions.
      * @param pending Pending versions.
-     * @return Lock owner.
      */
-    @Nullable public GridCacheMvccCandidate readyNearLocal(GridCacheVersion ver, GridCacheVersion mapped,
+    void readyNearLocal(GridCacheVersion ver, GridCacheVersion mapped,
         Collection<GridCacheVersion> committedVers, Collection<GridCacheVersion> rolledbackVers,
         Collection<GridCacheVersion> pending) {
-        return mvcc.readyNearLocal(ver, mapped, committedVers, rolledbackVers, pending);
+        mvcc.readyNearLocal(ver, mapped, committedVers, rolledbackVers, pending);
     }
 
     /**
      * @param cand Candidate to set to ready.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate readyLocal(GridCacheMvccCandidate cand) {
-        return mvcc.readyLocal(cand);
-    }
-
-    /**
-     * Local local release.
-     * @return Removed lock candidate or <tt>null</tt> if candidate was not removed.
-     */
-    @Nullable public GridCacheMvccCandidate releaseLocal() {
-        return releaseLocal(Thread.currentThread().getId());
+    void readyLocal(GridCacheMvccCandidate cand) {
+        mvcc.readyLocal(cand);
     }
 
     /**
      * Local release.
      *
      * @param threadId ID of the thread.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate releaseLocal(long threadId) {
-        return mvcc.releaseLocal(threadId);
+    void releaseLocal(long threadId) {
+        mvcc.releaseLocal(threadId);
     }
 
     /**
      *
      */
-    public void recheckLock() {
+    void recheckLock() {
         mvcc.recheck();
     }
 
@@ -594,8 +568,9 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
         long timeout,
         @Nullable GridCacheVersion serOrder,
         GridCacheVersion serReadVer,
-        boolean keepBinary) {
-        assert false; return false;
+        boolean read) {
+        assert false;
+        return false;
     }
 
     /** @inheritDoc */
@@ -765,7 +740,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     }
 
     /** @inheritDoc */
-    public Collection<GridCacheMvccCandidate> localCandidates(boolean reentries, GridCacheVersion... exclude) {
+    Collection<GridCacheMvccCandidate> localCandidates(boolean reentries, GridCacheVersion... exclude) {
         return mvcc.localCandidates(reentries, exclude);
     }
 
@@ -793,7 +768,7 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     /**
      * @return Any MVCC owner.
      */
-    public GridCacheMvccCandidate anyOwner() {
+    GridCacheMvccCandidate anyOwner() {
         return mvcc.anyOwner();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
index b459dde..5c12f84 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridHashMapLoadTest.java
@@ -21,6 +21,7 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.testframework.junits.GridTestKernalContext;
@@ -86,10 +87,14 @@ public class GridHashMapLoadTest extends GridCommonAbstractTest {
                     long timeout,
                     @Nullable GridCacheVersion serOrder,
                     GridCacheVersion serReadVer,
-                    boolean keepBinary) {
+                    boolean read) {
                     return false;
                 }
 
+                @Override protected void checkThreadChain(GridCacheMvccCandidate owner) {
+                    // No-op.
+                }
+
                 @Override public void txUnlock(IgniteInternalTx tx) {
                     // No-op.
                 }


[05/19] ignite git commit: Web console beta-6.

Posted by sb...@apache.org.
Web console beta-6.


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

Branch: refs/heads/master
Commit: bf330251734018467fa3291fccf0414c9da7dd1b
Parents: 7a47a01
Author: Andrey Novikov <an...@gridgain.com>
Authored: Thu Nov 24 17:08:08 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Thu Nov 24 17:08:08 2016 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js        |  32 ++-
 modules/web-console/backend/app/browser.js      |  13 ++
 modules/web-console/backend/app/mongo.js        |   6 +
 .../backend/config/settings.json.sample         |   7 -
 modules/web-console/backend/index.js            |   6 +-
 modules/web-console/backend/middlewares/host.js |   7 +-
 modules/web-console/backend/routes/agent.js     |   4 +-
 modules/web-console/backend/services/agents.js  |   2 +-
 modules/web-console/backend/services/caches.js  |   2 +
 .../docker/compose/backend/.dockerignore        |   2 +
 .../docker/compose/backend/Dockerfile           |   6 +-
 .../web-console/docker/compose/backend/build.sh |   2 +-
 .../docker/compose/docker-compose.yml           |   5 -
 .../docker/compose/frontend/DockerfileBuild     |   4 +-
 .../docker/compose/frontend/build.sh            |   2 +-
 .../compose/frontend/nginx/web-console.conf     |   9 +
 .../web-console/docker/standalone/.dockerignore |   5 +
 .../web-console/docker/standalone/Dockerfile    |  10 +-
 modules/web-console/docker/standalone/build.sh  |   2 +-
 .../docker/standalone/docker-compose.yml        |   7 +-
 .../docker/standalone/nginx/web-console.conf    |   9 +
 .../frontend/app/data/pom-dependencies.json     |   2 +-
 .../directives/ui-ace-pojos/ui-ace-pojos.jade   |   2 +-
 .../frontend/app/filters/duration.filter.js     |   3 +
 .../helpers/jade/form/form-field-checkbox.jade  |   2 +-
 .../helpers/jade/form/form-field-datalist.jade  |   2 +-
 .../helpers/jade/form/form-field-dropdown.jade  |   2 +-
 .../helpers/jade/form/form-field-number.jade    |   2 +-
 .../helpers/jade/form/form-field-password.jade  |   2 +-
 .../app/helpers/jade/form/form-field-text.jade  |   2 +-
 .../frontend/app/helpers/jade/mixins.jade       |  24 +-
 .../modules/configuration/generator/Beans.js    |   6 +
 .../generator/ConfigurationGenerator.js         | 165 ++++++-------
 .../generator/JavaTransformer.service.js        |  15 +-
 .../generator/Properties.service.js             |  19 ++
 .../generator/SpringTransformer.service.js      |  22 +-
 .../generator/defaults/cache.provider.js        |   8 +
 .../generator/defaults/cluster.provider.js      |   2 +-
 .../app/modules/form/field/tooltip.directive.js |   2 +-
 .../app/modules/form/group/tooltip.directive.js |   2 +-
 .../app/modules/form/panel/field.directive.js   |   4 +-
 .../app/modules/sql/scan-filter-input.jade      |   2 +-
 .../frontend/app/modules/sql/sql.controller.js  |   6 +-
 .../configuration/caches/client-near-cache.jade |   2 +-
 .../configuration/caches/concurrency.jade       |   2 +-
 .../states/configuration/caches/general.jade    |   2 +-
 .../states/configuration/caches/memory.jade     |   2 +-
 .../configuration/caches/near-cache-client.jade |   2 +-
 .../configuration/caches/near-cache-server.jade |   2 +-
 .../configuration/caches/node-filter.jade       |   2 +-
 .../states/configuration/caches/query.jade      |   2 +-
 .../states/configuration/caches/rebalance.jade  |   2 +-
 .../states/configuration/caches/statistics.jade |   2 +-
 .../states/configuration/caches/store.jade      |  20 +-
 .../states/configuration/clusters/atomic.jade   |   2 +-
 .../configuration/clusters/attributes.jade      |   2 +-
 .../states/configuration/clusters/binary.jade   |   2 +-
 .../configuration/clusters/cache-key-cfg.jade   |   2 +-
 .../configuration/clusters/checkpoint.jade      |   8 +-
 .../configuration/clusters/checkpoint/fs.jade   |   2 +-
 .../configuration/clusters/checkpoint/jdbc.jade |  41 ++--
 .../configuration/clusters/checkpoint/s3.jade   | 229 ++++++++++---------
 .../configuration/clusters/collision.jade       |  12 +-
 .../clusters/collision/custom.jade              |   2 +-
 .../clusters/collision/fifo-queue.jade          |   2 +-
 .../clusters/collision/job-stealing.jade        |   2 +-
 .../clusters/collision/priority-queue.jade      |   2 +-
 .../configuration/clusters/communication.jade   |   2 +-
 .../configuration/clusters/connector.jade       |   2 +-
 .../configuration/clusters/deployment.jade      |   2 +-
 .../configuration/clusters/discovery.jade       |   2 +-
 .../states/configuration/clusters/events.jade   |   2 +-
 .../states/configuration/clusters/failover.jade |   2 +-
 .../states/configuration/clusters/general.jade  |   2 +-
 .../clusters/general/discovery/cloud.jade       |   2 +-
 .../clusters/general/discovery/google.jade      |   2 +-
 .../clusters/general/discovery/jdbc.jade        |   3 +-
 .../clusters/general/discovery/multicast.jade   |   2 +-
 .../clusters/general/discovery/s3.jade          |   2 +-
 .../clusters/general/discovery/shared.jade      |   2 +-
 .../clusters/general/discovery/vm.jade          |   2 +-
 .../clusters/general/discovery/zookeeper.jade   |   2 +-
 .../bounded-exponential-backoff.jade            |   2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |   2 +-
 .../retrypolicy/exponential-backoff.jade        |   2 +-
 .../zookeeper/retrypolicy/forever.jade          |   2 +-
 .../zookeeper/retrypolicy/n-times.jade          |   2 +-
 .../zookeeper/retrypolicy/one-time.jade         |   2 +-
 .../zookeeper/retrypolicy/until-elapsed.jade    |   2 +-
 .../states/configuration/clusters/igfs.jade     |   2 +-
 .../configuration/clusters/load-balancing.jade  |   2 +-
 .../states/configuration/clusters/logger.jade   |   2 +-
 .../configuration/clusters/logger/custom.jade   |   2 +-
 .../configuration/clusters/logger/log4j.jade    |   2 +-
 .../configuration/clusters/logger/log4j2.jade   |   2 +-
 .../configuration/clusters/marshaller.jade      |   2 +-
 .../states/configuration/clusters/metrics.jade  |   2 +-
 .../states/configuration/clusters/odbc.jade     |   2 +-
 .../states/configuration/clusters/ssl.jade      |   2 +-
 .../states/configuration/clusters/swap.jade     |   2 +-
 .../states/configuration/clusters/thread.jade   |   2 +-
 .../states/configuration/clusters/time.jade     |   2 +-
 .../configuration/clusters/transactions.jade    |   2 +-
 .../states/configuration/domains/general.jade   |   2 +-
 .../states/configuration/domains/query.jade     |  16 +-
 .../states/configuration/domains/store.jade     |  12 +-
 .../modules/states/configuration/igfs/dual.jade |   2 +-
 .../states/configuration/igfs/fragmentizer.jade |   2 +-
 .../states/configuration/igfs/general.jade      |   2 +-
 .../modules/states/configuration/igfs/ipc.jade  |   2 +-
 .../modules/states/configuration/igfs/misc.jade |   6 +-
 .../states/configuration/igfs/secondary.jade    |   2 +-
 .../configuration/summary/summary.controller.js |  12 +-
 .../frontend/controllers/clusters-controller.js |  10 +
 .../frontend/controllers/domains-controller.js  |   8 +-
 .../frontend/gulpfile.babel.js/tasks/jade.js    |   4 +-
 .../frontend/gulpfile.babel.js/tasks/test.js    |  92 --------
 .../gulpfile.babel.js/webpack/common.js         |   5 +-
 .../webpack/environments/development.js         |   5 +
 modules/web-console/frontend/package.json       |   2 +-
 .../stylesheets/_font-awesome-custom.scss       |  18 ++
 .../frontend/public/stylesheets/style.scss      |  13 +-
 .../frontend/views/configuration/caches.jade    |  22 +-
 .../frontend/views/configuration/clusters.jade  |  50 ++--
 .../views/configuration/domains-import.jade     |   4 +-
 .../frontend/views/configuration/domains.jade   |   8 +-
 .../frontend/views/configuration/igfs.jade      |  14 +-
 .../frontend/views/configuration/summary.jade   |   2 +-
 .../frontend/views/includes/header.jade         |  15 +-
 .../frontend/views/settings/profile.jade        |   2 +-
 modules/web-console/frontend/views/sql/sql.jade |   2 +-
 .../views/templates/agent-download.jade         |   2 +-
 .../frontend/views/templates/batch-confirm.jade |   2 +-
 .../frontend/views/templates/clone.jade         |   2 +-
 .../frontend/views/templates/confirm.jade       |   2 +-
 modules/web-console/web-agent/README.txt        |   4 +-
 .../web-agent/bin/ignite-web-agent.bat          |   7 +-
 .../web-agent/bin/ignite-web-agent.sh           |   6 +-
 .../console/agent/AgentConfiguration.java       |   5 +-
 .../ignite/console/agent/AgentLauncher.java     |   6 +-
 .../ignite/console/demo/AgentClusterDemo.java   |   1 +
 141 files changed, 667 insertions(+), 563 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/app/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/agent.js b/modules/web-console/backend/app/agent.js
index 6aa9a12..f74a3f2 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -297,6 +297,23 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo)
 
         /**
          * @param {Boolean} demo Is need run command on demo node.
+         * @param {Array.<String>} nids Node ids.
+         * @param {Number} since Metrics since.
+         * @returns {Promise}
+         */
+        queryDetailMetrics(demo, nids, since) {
+            const cmd = new Command(demo, 'exe')
+                .addParam('name', 'org.apache.ignite.internal.visor.compute.VisorGatewayTask')
+                .addParam('p1', nids)
+                .addParam('p2', 'org.apache.ignite.internal.visor.cache.VisorCacheQueryDetailMetricsCollectorTask')
+                .addParam('p3', 'java.lang.Long')
+                .addParam('p4', since);
+
+            return this.executeRest(cmd);
+        }
+
+        /**
+         * @param {Boolean} demo Is need run command on demo node.
          * @param {String} cacheName Cache name.
          * @returns {Promise}
          */
@@ -634,6 +651,19 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo)
                 });
         }
 
+        attachLegacy(srv) {
+            /**
+             * @type {socketIo.Server}
+             */
+            const io = socketio(srv);
+
+            io.on('connection', (socket) => {
+                socket.on('agent:auth', (data, cb) => {
+                    return cb('You are using an older version of the agent. Please reload agent archive');
+                });
+            });
+        }
+
         /**
          * @param {http.Server|https.Server} srv Server instance that we want to attach agent handler.
          */
@@ -646,7 +676,7 @@ module.exports.factory = function(_, fs, path, JSZip, socketio, settings, mongo)
             /**
              * @type {socketIo.Server}
              */
-            this._socket = socketio(this._server);
+            this._socket = socketio(this._server, {path: '/agents'});
 
             this._socket.on('connection', (socket) => {
                 socket.on('agent:auth', (data, cb) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/app/browser.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/browser.js b/modules/web-console/backend/app/browser.js
index 254851d..2710829 100644
--- a/modules/web-console/backend/app/browser.js
+++ b/modules/web-console/backend/app/browser.js
@@ -149,6 +149,19 @@ module.exports.factory = (_, socketio, agentMgr, configure) => {
                         .catch((err) => cb(_errorToJson(err)));
                 });
 
+                // Collect cache query metrics and return result to browser.
+                socket.on('node:query:metrics', (nids, since, cb) => {
+                    agentMgr.findAgent(accountId())
+                        .then((agent) => agent.queryDetailMetrics(demo, nids, since))
+                        .then((data) => {
+                            if (data.finished)
+                                return cb(null, data.result);
+
+                            cb(_errorToJson(data.error));
+                        })
+                        .catch((err) => cb(_errorToJson(err)));
+                });
+
                 // Return cache metadata from all nodes in grid.
                 socket.on('node:cache:metadata', (cacheName, cb) => {
                     agentMgr.findAgent(accountId())

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/app/mongo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/app/mongo.js b/modules/web-console/backend/app/mongo.js
index e12af2a..0f38eb2 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -197,6 +197,12 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
                     type: String,
                     enum: ['Generic', 'Oracle', 'DB2', 'SQLServer', 'MySQL', 'PostgreSQL', 'H2']
                 },
+                batchSize: Number,
+                maximumPoolSize: Number,
+                maximumWriteAttempts: Number,
+                parallelLoadCacheMinimumThreshold: Number,
+                hasher: String,
+                transformer: String,
                 sqlEscapeAll: Boolean
             },
             CacheJdbcBlobStoreFactory: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/config/settings.json.sample
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/config/settings.json.sample b/modules/web-console/backend/config/settings.json.sample
index 41f1152..71a64ea 100644
--- a/modules/web-console/backend/config/settings.json.sample
+++ b/modules/web-console/backend/config/settings.json.sample
@@ -10,13 +10,6 @@
     "mongodb": {
         "url": "mongodb://localhost/console"
     },
-    "agentServer": {
-        "port": 3001,
-        "ssl": false,
-        "key": "serve/keys/test.key",
-        "cert": "serve/keys/test.crt",
-        "keyPassphrase": "password"
-    },
     "mail": {
         "service": "",
         "sign": "Kind regards,<br>Apache Ignite Team",

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/index.js b/modules/web-console/backend/index.js
index 3a8ada9..9eeff35 100644
--- a/modules/web-console/backend/index.js
+++ b/modules/web-console/backend/index.js
@@ -82,9 +82,11 @@ Promise.all([injector('settings'), injector('app'), injector('agent-manager'), i
         server.on('listening', _onListening.bind(null, server.address()));
 
         app.listen(server);
+
+        agentMgr.attach(server);
         browserMgr.attach(server);
 
-        // Start agent server.
+        // Start legacy agent server.
         const agentServer = settings.agent.SSLOptions
             ? https.createServer(settings.agent.SSLOptions) : http.createServer();
 
@@ -92,7 +94,7 @@ Promise.all([injector('settings'), injector('app'), injector('agent-manager'), i
         agentServer.on('error', _onError.bind(null, settings.agent.port));
         agentServer.on('listening', _onListening.bind(null, agentServer.address()));
 
-        agentMgr.attach(agentServer);
+        agentMgr.attachLegacy(agentServer);
 
         // Used for automated test.
         if (process.send)

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/middlewares/host.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/middlewares/host.js b/modules/web-console/backend/middlewares/host.js
index 5ddd918..4c21da2 100644
--- a/modules/web-console/backend/middlewares/host.js
+++ b/modules/web-console/backend/middlewares/host.js
@@ -27,10 +27,11 @@ module.exports = {
                 if (req.headers.origin)
                     return req.headers.origin;
 
-                if (req.headers['x-forwarded-server'])
-                    return `${req.headers['x-forwarded-proto'] || 'http'}://${req.headers['x-forwarded-server']}`;
+                const proto = req.headers['x-forwarded-proto'] || req.protocol;
 
-                return `${req.protocol}://${req.get('host')}`;
+                const host = req.headers['x-forwarded-host'] || req.get('host');
+
+                return `${proto}://${host}`;
             };
 
             next();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/routes/agent.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/agent.js b/modules/web-console/backend/routes/agent.js
index 3f90fbd..477363f 100644
--- a/modules/web-console/backend/routes/agent.js
+++ b/modules/web-console/backend/routes/agent.js
@@ -36,9 +36,7 @@ module.exports.factory = function(_, express, agentsService) {
 
         /* Get grid topology. */
         router.get('/download/zip', (req, res) => {
-            const host = req.hostname.match(/:/g) ? req.hostname.slice(0, req.hostname.indexOf(':')) : req.hostname;
-
-            agentsService.getArchive(host, req.user.token)
+            agentsService.getArchive(req.origin(), req.user.token)
                 .then(({fileName, buffer}) => {
                     // Set the archive name.
                     res.attachment(fileName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/services/agents.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/agents.js b/modules/web-console/backend/services/agents.js
index 5c0b6a7..4931bf8 100644
--- a/modules/web-console/backend/services/agents.js
+++ b/modules/web-console/backend/services/agents.js
@@ -63,7 +63,7 @@ module.exports.factory = (_, fs, path, JSZip, settings, agentMgr, errors) => {
                             const prop = [];
 
                             prop.push('tokens=' + token);
-                            prop.push('server-uri=' + (settings.agent.SSLOptions ? 'https' : 'http') + '://' + host + ':' + settings.agent.port);
+                            prop.push(`server-uri=${host}`);
                             prop.push('#Uncomment following options if needed:');
                             prop.push('#node-uri=http://localhost:8080');
                             prop.push('#driver-folder=./jdbc-drivers');

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/backend/services/caches.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/caches.js b/modules/web-console/backend/services/caches.js
index e59d51d..646fbea 100644
--- a/modules/web-console/backend/services/caches.js
+++ b/modules/web-console/backend/services/caches.js
@@ -87,6 +87,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => {
      */
     const removeAllBySpaces = (spaceIds) => {
         return mongo.Cluster.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec()
+            .then(() => mongo.Cluster.update({space: {$in: spaceIds}}, {$pull: {checkpointSpi: {kind: 'Cache'}}}, {multi: true}).exec())
             .then(() => mongo.DomainModel.update({space: {$in: spaceIds}}, {caches: []}, {multi: true}).exec())
             .then(() => mongo.Cache.remove({space: {$in: spaceIds}}).exec());
     };
@@ -129,6 +130,7 @@ module.exports.factory = (_, mongo, spaceService, errors) => {
                 return Promise.reject(new errors.IllegalArgumentException('Cache id can not be undefined or null'));
 
             return mongo.Cluster.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec()
+                .then(() => mongo.Cluster.update({}, {$pull: {checkpointSpi: {kind: 'Cache', Cache: {cache: cacheId}}}}, {multi: true}).exec())
                 .then(() => mongo.DomainModel.update({caches: {$in: [cacheId]}}, {$pull: {caches: cacheId}}, {multi: true}).exec())
                 .then(() => mongo.Cache.remove({_id: cacheId}).exec())
                 .then(convertRemoveStatus);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/backend/.dockerignore
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/backend/.dockerignore b/modules/web-console/docker/compose/backend/.dockerignore
index 6fadfa5..05df665 100644
--- a/modules/web-console/docker/compose/backend/.dockerignore
+++ b/modules/web-console/docker/compose/backend/.dockerignore
@@ -1 +1,3 @@
+build/config/*.json
 build/node_modules
+build/test

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/backend/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/backend/Dockerfile b/modules/web-console/docker/compose/backend/Dockerfile
index b4f7c9d..e391ba4 100644
--- a/modules/web-console/docker/compose/backend/Dockerfile
+++ b/modules/web-console/docker/compose/backend/Dockerfile
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-FROM node:4
+FROM node:6
 
 RUN mkdir -p /opt/web-console-backend
 
@@ -23,8 +23,8 @@ WORKDIR /opt/web-console-backend
 
 COPY build .
 
-RUN npm -g update npm && npm install --no-optional
+RUN npm install --only=production --no-optional
 
-EXPOSE 3000 3001
+EXPOSE 3000
 
 CMD ["npm", "start"]

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/backend/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh
index f925bd7..d44efbd 100755
--- a/modules/web-console/docker/compose/backend/build.sh
+++ b/modules/web-console/docker/compose/backend/build.sh
@@ -51,7 +51,7 @@ cp -r $IGNITE_WEB_CONSOLE_BACKEND_DIR/. $BUILD_DIR
 cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/agent_dists/.
 
 echo "Step 4. Build docker image."
-docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION .
+docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest .
 
 echo "Step 5. Cleanup."
 rm -Rf $BUILD_DIR

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/docker-compose.yml
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/docker-compose.yml b/modules/web-console/docker/compose/docker-compose.yml
index a2c2f8b..8b9b86a 100644
--- a/modules/web-console/docker/compose/docker-compose.yml
+++ b/modules/web-console/docker/compose/docker-compose.yml
@@ -26,9 +26,6 @@ backend:
   links:
     # Link mongodb container as with mongodb hostname.
     - mongodb:mongodb
-  ports:
-    # Proxy 3001 port from docker container to 3001 port host machine. (HOST_PORT:DOCKER_PORT)
-    - 3001:3001
   # Restart on crash.
   restart: always  
   environment:
@@ -38,8 +35,6 @@ backend:
     - server_sessionSecret=CHANGE ME
     # URL for mongodb connection
     - mongodb_url=mongodb://mongodb/console
-    # Port for web-agent.
-    - agentServer_port=3001
     # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer
     - mail_service=
     - mail_sign=

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/frontend/DockerfileBuild
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/frontend/DockerfileBuild b/modules/web-console/docker/compose/frontend/DockerfileBuild
index 277991f..2d61f4c 100644
--- a/modules/web-console/docker/compose/frontend/DockerfileBuild
+++ b/modules/web-console/docker/compose/frontend/DockerfileBuild
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-FROM node:4
+FROM node:6
 
 RUN mkdir -p /opt/web-console-frontend
 
@@ -23,7 +23,7 @@ WORKDIR /opt/web-console-frontend
 
 COPY src .
 
-RUN npm update -g npm && npm install --no-optional
+RUN npm install --no-optional
 
 VOLUME /opt/web-console-frontend/build
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/frontend/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh
index 4dfa57a..6d52fb2 100755
--- a/modules/web-console/docker/compose/frontend/build.sh
+++ b/modules/web-console/docker/compose/frontend/build.sh
@@ -50,7 +50,7 @@ docker build -f=./DockerfileBuild -t $DOCKER_BUILD_IMAGE_NAME:latest .
 docker run -it -v $BUILD_DIR:/opt/web-console-frontend/build --name $DOCKER_BUILD_CONTAINER $DOCKER_BUILD_IMAGE_NAME
 
 echo "Step 2. Build NGINX container with SPA and proxy configuration"
-docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION .
+docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest .
 
 echo "Step 3. Cleanup"
 docker rm -f $DOCKER_BUILD_CONTAINER

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/compose/frontend/nginx/web-console.conf
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/frontend/nginx/web-console.conf b/modules/web-console/docker/compose/frontend/nginx/web-console.conf
index d80a7f9..323826e 100644
--- a/modules/web-console/docker/compose/frontend/nginx/web-console.conf
+++ b/modules/web-console/docker/compose/frontend/nginx/web-console.conf
@@ -53,6 +53,15 @@ server {
     proxy_pass http://backend-api;
   }
 
+  location /agents {
+    proxy_set_header Upgrade $http_upgrade;
+    proxy_set_header Connection "upgrade";
+    proxy_http_version 1.1;
+    proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for;
+    proxy_set_header Host $host;
+    proxy_pass http://backend-api;
+  }
+
   location = /50x.html {
     root $ignite_console_dir/error_page;
   }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/standalone/.dockerignore
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/.dockerignore b/modules/web-console/docker/standalone/.dockerignore
index 35b7244..c59189e 100644
--- a/modules/web-console/docker/standalone/.dockerignore
+++ b/modules/web-console/docker/standalone/.dockerignore
@@ -1,2 +1,7 @@
+build/frontend/build
 build/frontend/node_modules
+build/frontend/ignite_modules_temp
+build/frontend/test
+build/backend/config/*.json
 build/backend/node_modules
+build/backend/test

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/standalone/Dockerfile
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/Dockerfile b/modules/web-console/docker/standalone/Dockerfile
index 785f109..0bcd07d 100644
--- a/modules/web-console/docker/standalone/Dockerfile
+++ b/modules/web-console/docker/standalone/Dockerfile
@@ -18,7 +18,7 @@
 FROM ubuntu:14.04
 
 ENV NPM_CONFIG_LOGLEVEL info
-ENV NODE_VERSION 4.4.7
+ENV NODE_VERSION 6.6.0
 
 # Before package list update.
 RUN set -ex  && \
@@ -40,7 +40,7 @@ RUN apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv EA312927 && \
 
 # Update package list & install.
 RUN apt-get update && \
-    apt-get install -y nginx-light mongodb-org-server curl xz-utils
+    apt-get install -y nginx-light mongodb-org-server curl xz-utils git
 
 # Install Node JS.
 RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-x64.tar.xz"  && \
@@ -51,7 +51,7 @@ RUN curl -SLO "https://nodejs.org/dist/v$NODE_VERSION/node-v$NODE_VERSION-linux-
   rm "node-v$NODE_VERSION-linux-x64.tar.xz" SHASUMS256.txt.asc SHASUMS256.txt
 
 # Install global node packages.
-RUN npm upgrade -g npm && npm install -g pm2
+RUN npm install -g pm2
 
 # Install frontend & backend apps.
 RUN mkdir -p /opt/web-console
@@ -62,7 +62,7 @@ COPY build .
 
 # Install node modules.
 RUN cd /opt/web-console/frontend && npm install --no-optional && npm run build
-RUN cd /opt/web-console/backend && npm install --no-optional
+RUN cd /opt/web-console/backend && npm install --only=production --no-optional
 
 # Returns to base path.
 WORKDIR /opt/web-console
@@ -82,6 +82,6 @@ VOLUME ["/etc/nginx"]
 VOLUME ["/var/lib/mongodb"]
 VOLUME ["/opt/web-console/serve/agent_dists"]
 
-EXPOSE 80 3001
+EXPOSE 80
 
 ENTRYPOINT ["/opt/web-console/entrypoint.sh"]

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/standalone/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh
index 4365dec..5482086 100755
--- a/modules/web-console/docker/standalone/build.sh
+++ b/modules/web-console/docker/standalone/build.sh
@@ -53,7 +53,7 @@ cp -r $IGNITE_WEB_CONSOLE_DIR/backend/. $BUILD_DIR/backend
 cp $IGNITE_HOME/modules/web-console/web-agent/target/ignite-web-agent*.zip $BUILD_DIR/backend/agent_dists/.
 
 echo "Step 4. Build docker image."
-docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION .
+docker build -f=./Dockerfile -t $DOCKER_IMAGE_NAME:$RELEASE_VERSION -t $DOCKER_IMAGE_NAME:latest .
 
 echo "Step 5. Cleanup."
 rm -Rf $BUILD_DIR

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/standalone/docker-compose.yml
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/docker-compose.yml b/modules/web-console/docker/standalone/docker-compose.yml
index 561c88d..eb59e8a 100644
--- a/modules/web-console/docker/standalone/docker-compose.yml
+++ b/modules/web-console/docker/standalone/docker-compose.yml
@@ -18,10 +18,7 @@
 webconsole:
   image: ignite/web-console-standalone
   ports:
-    - 3080:80
-    - 3000:3000
-    - 3001:3001
-    - 27017:27017
+    - 80:80
   restart: always
   environment:
     # Port for serving frontend API
@@ -30,8 +27,6 @@ webconsole:
     - server_sessionSecret="CHANGE ME"
     # URL for mongodb connection
     - mongodb_url=mongodb://127.0.0.1/console
-    # Port for web-agent.
-    - agentServer_port=3001
     # Mail connection settings. Leave empty if no needed. See also settings, https://github.com/nodemailer/nodemailer
     - mail_service=""
     - mail_sign=""

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/docker/standalone/nginx/web-console.conf
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/nginx/web-console.conf b/modules/web-console/docker/standalone/nginx/web-console.conf
index 3de544f..3d83075 100644
--- a/modules/web-console/docker/standalone/nginx/web-console.conf
+++ b/modules/web-console/docker/standalone/nginx/web-console.conf
@@ -48,6 +48,15 @@ server {
     proxy_pass http://backend-api;
   }
 
+  location /agents {
+    proxy_set_header Upgrade $http_upgrade;
+    proxy_set_header Connection "upgrade";
+    proxy_http_version 1.1;
+    proxy_set_header X-Forwarded-For $proxy_add_x_forwarded_for;
+    proxy_set_header Host $host;
+    proxy_pass http://backend-api;
+  }
+
   location = /50x.html {
     root $ignite_console_dir/error_page;
   }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/data/pom-dependencies.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/pom-dependencies.json b/modules/web-console/frontend/app/data/pom-dependencies.json
index 7ab6c1b..acf2bc8 100644
--- a/modules/web-console/frontend/app/data/pom-dependencies.json
+++ b/modules/web-console/frontend/app/data/pom-dependencies.json
@@ -13,7 +13,7 @@
     "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.1"},
     "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.37"},
     "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4-1204-jdbc42"},
-    "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.3.175"},
+    "H2": {"groupId": "com.h2database", "artifactId": "h2", "version": "1.4.191"},
     "Oracle": {"groupId": "oracle", "artifactId": "jdbc", "version": "11.2", "jar": "ojdbc6.jar"},
     "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.19.26", "jar": "db2jcc4.jar"},
     "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.1", "jar": "sqljdbc41.jar"}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade
index ed1432b..581b8c1 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade
+++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.jade
@@ -15,7 +15,7 @@
     limitations under the License.
 
 mixin check-tooltip(message)
-    i.tipLabel.fa.fa-question-circle(bs-tooltip='"#{message}"')
+    i.tipLabel.icon-help(bs-tooltip='"#{message}"')
 
 .panel-details-noborder
     .details-row

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/filters/duration.filter.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/filters/duration.filter.js b/modules/web-console/frontend/app/filters/duration.filter.js
index deeedd7..0d7afe6 100644
--- a/modules/web-console/frontend/app/filters/duration.filter.js
+++ b/modules/web-console/frontend/app/filters/duration.filter.js
@@ -20,6 +20,9 @@ export default ['duration', [() => {
      * @param {Number} t Time in ms.
      */
     return (t) => {
+        if (t === 9223372036854775807)
+            return 'Infinite';
+
         const a = (i, suffix) => i && i !== '00' ? i + suffix + ' ' : '';
 
         const cd = 24 * 60 * 60 * 1000;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade
index ef5cb37..222ecfe 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-checkbox.jade
@@ -35,4 +35,4 @@ mixin form-field-checkbox(label, model, name, disabled, required, tip)
                         data-ignite-form-panel-field=''
                     )
             span #{label}
-            i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipLabel.icon-help(bs-tooltip='' data-title=tip)

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade
index 25e5805..4c1970e 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-datalist.jade
@@ -40,7 +40,7 @@ mixin form-field-datalist(label, model, name, disabled, required, placeholder, o
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipField.fa.icon-help(bs-tooltip='' data-title=tip)
 
             +form-field-feedback(name, 'required', errLbl + ' could not be empty!')
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
index f5d035d..298db52 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-dropdown.jade
@@ -41,7 +41,7 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipField.icon-help(bs-tooltip='' data-title=tip)
 
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
index 62b3e09..d48343c 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-number.jade
@@ -38,7 +38,7 @@ mixin ignite-form-field-number(label, model, name, disabled, required, placehold
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             +form-field-feedback(name, 'required', 'This field could not be empty')
             +form-field-feedback(name, 'min', 'Value is less than allowable minimum: '+ min || 0)

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade
index 51cc109..e5e7bc8 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-password.jade
@@ -36,7 +36,7 @@ mixin ignite-form-field-password(label, model, name, disabled, required, placeho
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
index 55b850d..136d23b 100644
--- a/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
+++ b/modules/web-console/frontend/app/helpers/jade/form/form-field-text.jade
@@ -36,7 +36,7 @@ mixin ignite-form-field-text(label, model, name, disabled, required, placeholder
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.fa.fa-question-circle(bs-tooltip='' data-title=tip)
+            i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/helpers/jade/mixins.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/helpers/jade/mixins.jade b/modules/web-console/frontend/app/helpers/jade/mixins.jade
index 9d46883..92af1b0 100644
--- a/modules/web-console/frontend/app/helpers/jade/mixins.jade
+++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade
@@ -433,10 +433,8 @@ mixin evictionPolicy(model, name, enabled, required, tip)
             {value: "SORTED", label: "Sorted"},\
             {value: undefined, label: "Not set"}\
         ]', tip)
-    span(ng-if=kind ng-init='__ = {};')
-        a.customize(ng-show='__.expanded' ng-click='__.expanded = false') Hide settings
-        a.customize(ng-hide='__.expanded' ng-click='__.expanded = true') Show settings
-        .panel-details(ng-if='__.expanded')
+    span(ng-show=kind)
+        +showHideLink('expanded', 'settings')
             .details-row
                 +number('Batch size', policy + '.batchSize', name + '+ "batchSize"', enabled, '1', '1',
                     'Number of entries to remove on shrink')
@@ -517,9 +515,9 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI
         .fieldSep !{divider}
         .input-tip
             if keyJavaBuiltInTypes
-                input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-escape='tableReset()')
+                input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-escape='tableReset(false)')
             else
-                input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder ignite-on-escape='tableReset()')
+                input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder=keyPlaceholder ignite-on-escape='tableReset(false)')
     .col-xs-6.col-sm-6.col-md-6
         -var btnVisible = 'tablePairSaveVisible(' + tbl + ', ' + index + ')'
         -var btnSave = 'tablePairSave(tablePairValid, backupItem, ' + tbl + ', ' + index + ')'
@@ -528,9 +526,9 @@ mixin table-pair-edit(tbl, prefix, keyPlaceholder, valPlaceholder, keyJavaBuiltI
         +btn-save(btnVisible, btnSave)
         .input-tip
             if valueJavaBuiltInTypes
-                input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+                input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder bs-typeahead container='body' ignite-retain-selection data-min-length='1' bs-options='javaClass for javaClass in javaBuiltInClasses' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
             else
-                input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+                input.form-control(id=valFocusId type='text' ng-model=valModel placeholder=valPlaceholder ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
 
 //- Mixin for DB dialect.
 mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placeholder)
@@ -553,3 +551,13 @@ mixin dialect(lbl, model, name, required, tipTitle, genericDialectName, placehol
             <li>PostgreSQL</li>\
             <li>H2 database</li>\
         </ul>')
+
+//- Mixin for show/hide links.
+mixin showHideLink(name, text)
+    span(ng-init='__ = {};')
+        a.customize(ng-show='__.#{name}' ng-click='__.#{name} = false') Hide #{text}
+        a.customize(ng-hide='__.#{name}' ng-click='__.#{name} = true; ui.loadPanel("#{name}");') Show #{text}
+        div(ng-if='ui.isPanelLoaded("#{name}")')
+            .panel-details(ng-show='__.#{name}')
+                if block
+                    block

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
index 546f38b..2750626 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
@@ -221,6 +221,12 @@ export class Bean extends EmptyBean {
         return this;
     }
 
+    propertyInt(name, value, hint) {
+        this.properties.push({clsName: 'PROPERTY_INT', name, value, hint});
+
+        return this;
+    }
+
     stringProperty(model, name = model, mapper) {
         return this._property(this.properties, 'java.lang.String', model, name, _.nonEmpty, mapper);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
index 869e3df..5887832 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -103,9 +103,9 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                 case 'DB2':
                     dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {})
                         .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_NAME')
-                        .property('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER')
-                        .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_JDBC_DRIVER_TYPE')
-                        .property('driverType', `${id}.jdbc.driver_type`, 'YOUR_DATABASE_NAME');
+                        .propertyInt('portNumber', `${id}.jdbc.port_number`, 'YOUR_JDBC_PORT_NUMBER')
+                        .property('databaseName', `${id}.jdbc.database_name`, 'YOUR_DATABASE_NAME')
+                        .propertyInt('driverType', `${id}.jdbc.driver_type`, 'YOUR_JDBC_DRIVER_TYPE');
 
                     break;
                 case 'SQLServer':
@@ -234,12 +234,14 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                     ipFinder.emptyBeanProperty('curator')
                         .stringProperty('zkConnectionString');
 
-                    if (_.get(src, 'retryPolicy.kind')) {
+                    const kind = _.get(src, 'retryPolicy.kind');
+
+                    if (kind) {
                         const policy = src.retryPolicy;
 
                         let retryPolicyBean;
 
-                        switch (policy.kind) {
+                        switch (kind) {
                             case 'ExponentialBackoff':
                                 retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null,
                                     policy.ExponentialBackoff, dflt.ExponentialBackoff)
@@ -289,6 +291,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                                 break;
                             default:
+                                // No-op.
                         }
 
                         if (retryPolicyBean)
@@ -301,6 +304,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     break;
                 default:
+                    // No-op.
             }
 
             if (ipFinder)
@@ -425,10 +429,10 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
         // Generate checkpoint configurations.
         static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) {
             const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => {
-                switch (spi.kind) {
+                switch (_.get(spi, 'kind')) {
                     case 'FS':
                         const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi',
-                            'checkpointSpi', spi.FS);
+                            'checkpointSpiFs', spi.FS);
 
                         fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths'))
                             .emptyBeanProperty('checkpointListener');
@@ -437,9 +441,11 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     case 'Cache':
                         const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi',
-                            'checkpointSpi', spi.Cache);
+                            'checkpointSpiCache', spi.Cache);
 
-                        const cache = _.find(caches, (c) => c._id === _.get(spi, 'Cache.cache') || c.cache._id === _.get(spi, 'Cache.cache'));
+                        const curCache = _.get(spi, 'Cache.cache');
+
+                        const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache));
 
                         if (cache)
                             cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name);
@@ -451,7 +457,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     case 'S3':
                         const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi',
-                            'checkpointSpi', spi.S3, clusterDflts.checkpointSpi.S3);
+                            'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3);
 
                         let credentialsBean = null;
 
@@ -459,8 +465,8 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                             case 'Basic':
                                 credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {});
 
-                                credentialsBean.constructorArgument('PROPERTY', 'checkpoint.s3.credentials.accessKey')
-                                    .constructorArgument('PROPERTY', 'checkpoint.s3.credentials.secretKey');
+                                credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY')
+                                    .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY');
 
                                 break;
 
@@ -645,7 +651,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     case 'JDBC':
                         const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi',
-                            'checkpointSpi', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
+                            'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
 
                         const id = jdbcBean.valueOf('dataSourceBean');
                         const dialect = _.get(spi.JDBC, 'dialect');
@@ -673,7 +679,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                         const clsName = _.get(spi, 'Custom.className');
 
                         if (clsName)
-                            return new Bean(clsName, 'checkpointSpi', spi.Cache);
+                            return new Bean(clsName, 'checkpointSpiCustom', spi.Cache);
 
                         return null;
 
@@ -691,7 +697,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
         static clusterCollision(collision, cfg = this.igniteConfigurationBean()) {
             let colSpi;
 
-            switch (collision.kind) {
+            switch (_.get(collision, 'kind')) {
                 case 'JobStealing':
                     colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi',
                         'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing);
@@ -727,18 +733,16 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     break;
                 case 'Custom':
-                    colSpi = new Bean(collision.Custom.class,
-                        'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue);
+                    if (_.nonNil(_.get(collision, 'Custom.class')))
+                        colSpi = new EmptyBean(collision.Custom.class);
 
                     break;
                 default:
                     return cfg;
             }
 
-            if (colSpi.isEmpty())
-                return cfg;
-
-            cfg.beanProperty('collisionSpi', colSpi);
+            if (_.nonNil(colSpi))
+                cfg.beanProperty('collisionSpi', colSpi);
 
             return cfg;
         }
@@ -907,7 +911,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
             _.forEach(cluster.failoverSpi, (spi) => {
                 let failoverSpi;
 
-                switch (spi.kind) {
+                switch (_.get(spi, 'kind')) {
                     case 'JobStealing':
                         failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi',
                             'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing);
@@ -955,26 +959,25 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
             _.forEach(cluster.loadBalancingSpi, (spi) => {
                 let loadBalancingSpi;
 
-                switch (spi.kind) {
+                switch (_.get(spi, 'kind')) {
                     case 'RoundRobin':
-                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpi', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin);
+                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin);
 
                         loadBalancingSpi.boolProperty('perTask');
 
                         break;
                     case 'Adaptive':
-                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpi', spi.Adaptive);
+                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive);
 
                         let probeBean;
 
-                        switch (spi.Adaptive.loadProbe.kind) {
+                        switch (_.get(spi, 'Adaptive.loadProbe.kind')) {
                             case 'Job':
                                 probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveJobCountLoadProbe', 'jobProbe', spi.Adaptive.loadProbe.Job, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.Job);
 
                                 probeBean.boolProperty('useAverage');
 
                                 break;
-
                             case 'CPU':
                                 probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.CPU);
 
@@ -983,14 +986,12 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                                     .intProperty('processorCoefficient');
 
                                 break;
-
                             case 'ProcessingTime':
                                 probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveProcessingTimeLoadProbe', 'timeProbe', spi.Adaptive.loadProbe.ProcessingTime, clusterDflts.loadBalancingSpi.Adaptive.loadProbe.ProcessingTime);
 
                                 probeBean.boolProperty('useAverage');
 
                                 break;
-
                             case 'Custom':
                                 const className = _.get(spi, 'Adaptive.loadProbe.Custom.className');
 
@@ -998,7 +999,6 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                                     probeBean = new Bean(className, 'probe', spi.Adaptive.loadProbe.Job.Custom);
 
                                 break;
-
                             default:
                                 // No-op.
                         }
@@ -1008,7 +1008,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                         break;
                     case 'WeightedRandom':
-                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpi', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom);
+                        loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.weightedrandom.WeightedRandomLoadBalancingSpi', 'loadBalancingSpiRandom', spi.WeightedRandom, clusterDflts.loadBalancingSpi.WeightedRandom);
 
                         loadBalancingSpi.intProperty('nodeWeight')
                             .boolProperty('useWeights');
@@ -1018,7 +1018,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                         const className = _.get(spi, 'Custom.className');
 
                         if (className)
-                            loadBalancingSpi = new Bean(className, 'loadBalancingSpi', spi.Custom);
+                            loadBalancingSpi = new Bean(className, 'loadBalancingSpiCustom', spi.Custom);
 
                         break;
                     default:
@@ -1030,19 +1030,16 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
             });
 
             if (spis.length)
-                cfg.arrayProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi');
+                cfg.varArgProperty('loadBalancingSpi', 'loadBalancingSpi', spis, 'org.apache.ignite.spi.loadbalancing.LoadBalancingSpi');
 
             return cfg;
         }
 
         // Generate logger group.
         static clusterLogger(logger, cfg = this.igniteConfigurationBean()) {
-            if (_.isNil(logger))
-                return cfg;
-
             let loggerBean;
 
-            switch (logger.kind) {
+            switch (_.get(logger, 'kind')) {
                 case 'Log4j':
                     if (logger.Log4j && (logger.Log4j.mode === 'Default' || logger.Log4j.mode === 'Path' && _.nonEmpty(logger.Log4j.path))) {
                         loggerBean = new Bean('org.apache.ignite.logger.log4j.Log4JLogger',
@@ -1087,6 +1084,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
                     break;
                 default:
+                    return cfg;
             }
 
             if (loggerBean)
@@ -1116,34 +1114,34 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
         // Generate marshaller group.
         static clusterMarshaller(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const marshaller = cluster.marshaller;
-
-            if (marshaller && marshaller.kind) {
-                let bean;
+            const kind = _.get(cluster.marshaller, 'kind');
+            const settings = _.get(cluster.marshaller, kind);
 
-                switch (marshaller.kind) {
-                    case 'OptimizedMarshaller':
-                        bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller',
-                            marshaller[marshaller.kind]);
+            if (_.isNil(settings))
+                return cfg;
 
-                        bean.intProperty('poolSize')
-                            .intProperty('requireSerializable');
+            let bean;
 
-                        break;
+            switch (kind) {
+                case 'OptimizedMarshaller':
+                    bean = new Bean('org.apache.ignite.marshaller.optimized.OptimizedMarshaller', 'marshaller', settings)
+                        .intProperty('poolSize')
+                        .intProperty('requireSerializable');
 
-                    case 'JdkMarshaller':
-                        bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller',
-                            marshaller[marshaller.kind]);
+                    break;
 
-                        break;
+                case 'JdkMarshaller':
+                    bean = new Bean('org.apache.ignite.marshaller.jdk.JdkMarshaller', 'marshaller', settings);
 
-                    default:
-                }
+                    break;
 
-                if (bean)
-                    cfg.beanProperty('marshaller', bean);
+                default:
+                    // No-op.
             }
 
+            if (bean)
+                cfg.beanProperty('marshaller', bean);
+
             cfg.intProperty('marshalLocalJobs')
                 .intProperty('marshallerCacheKeepAliveTime')
                 .intProperty('marshallerCacheThreadPoolSize', 'marshallerCachePoolSize');
@@ -1214,7 +1212,7 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
 
         // Generate swap group.
         static clusterSwap(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') {
+            if (_.get(cluster.swapSpaceSpi, 'kind') === 'FileSwapSpaceSpi') {
                 const bean = new Bean('org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi', 'swapSpaceSpi',
                     cluster.swapSpaceSpi.FileSwapSpaceSpi);
 
@@ -1451,14 +1449,20 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                 switch (kind) {
                     case 'CacheJdbcPojoStoreFactory':
                         bean = new Bean('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', 'cacheStoreFactory',
-                            storeFactory);
+                            storeFactory, cacheDflts.cacheStoreFactory.CacheJdbcPojoStoreFactory);
 
                         const jdbcId = bean.valueOf('dataSourceBean');
 
                         bean.dataSource(jdbcId, 'dataSourceBean', this.dataSourceBean(jdbcId, storeFactory.dialect))
                             .beanProperty('dialect', new EmptyBean(this.dialectClsName(storeFactory.dialect)));
 
-                        bean.boolProperty('sqlEscapeAll');
+                        bean.intProperty('batchSize')
+                            .intProperty('maximumPoolSize')
+                            .intProperty('maximumWriteAttempts')
+                            .intProperty('parallelLoadCacheMinimumThreshold')
+                            .emptyBeanProperty('hasher')
+                            .emptyBeanProperty('transformer')
+                            .boolProperty('sqlEscapeAll');
 
                         const setType = (typeBean, propName) => {
                             if (JavaTypes.nonBuiltInClass(typeBean.valueOf(propName)))
@@ -1555,31 +1559,35 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
         static cacheNodeFilter(cache, igfss, ccfg = this.cacheConfigurationBean(cache)) {
             const kind = _.get(cache, 'nodeFilter.kind');
 
-            if (kind && cache.nodeFilter[kind]) {
-                let bean = null;
+            const settings = _.get(cache.nodeFilter, kind);
 
-                switch (kind) {
-                    case 'IGFS':
-                        const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs);
+            if (_.isNil(settings))
+                return ccfg;
 
-                        if (foundIgfs) {
-                            bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs)
-                                .stringConstructorArgument('name');
-                        }
+            let bean = null;
 
-                        break;
-                    case 'Custom':
-                        bean = new Bean(cache.nodeFilter.Custom.className, 'nodeFilter');
+            switch (kind) {
+                case 'IGFS':
+                    const foundIgfs = _.find(igfss, {_id: settings.igfs});
 
-                        break;
-                    default:
-                        return ccfg;
-                }
+                    if (foundIgfs) {
+                        bean = new Bean('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate', 'nodeFilter', foundIgfs)
+                            .stringConstructorArgument('name');
+                    }
 
-                if (bean)
-                    ccfg.beanProperty('nodeFilter', bean);
+                    break;
+                case 'Custom':
+                    if (_.nonEmpty(settings.className))
+                        bean = new EmptyBean(settings.className);
+
+                    break;
+                default:
+                    // No-op.
             }
 
+            if (bean)
+                ccfg.beanProperty('nodeFilter', bean);
+
             return ccfg;
         }
 
@@ -1733,7 +1741,8 @@ export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'ig
                     .pathProperty('tokenDirectoryPath')
                     .intProperty('threadCount');
 
-                cfg.beanProperty('ipcEndpointConfiguration', bean);
+                if (bean.nonEmpty())
+                    cfg.beanProperty('ipcEndpointConfiguration', bean);
             }
 
             return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
index 64d43d8..b123ab5 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/JavaTransformer.service.js
@@ -395,17 +395,19 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                         return `${item}L`;
                     case 'java.io.Serializable':
                     case 'java.lang.String':
-                        return `"${item}"`;
+                        return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`;
                     case 'PATH':
                         return `"${item.replace(/\\/g, '\\\\')}"`;
                     case 'java.lang.Class':
                         return `${JavaTypes.shortClassName(item)}.class`;
                     case 'java.util.UUID':
                         return `UUID.fromString("${item}")`;
-                    case 'PROPERTY_CHAR':
-                        return `props.getProperty("${item}").toCharArray()`;
                     case 'PROPERTY':
                         return `props.getProperty("${item}")`;
+                    case 'PROPERTY_CHAR':
+                        return `props.getProperty("${item}").toCharArray()`;
+                    case 'PROPERTY_INT':
+                        return `Integer.parseInt(props.getProperty("${item}"))`;
                     default:
                         if (this._isBean(clsName)) {
                             if (item.isComplex())
@@ -744,6 +746,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                         break;
                     case 'PROPERTY':
                     case 'PROPERTY_CHAR':
+                    case 'PROPERTY_INT':
                         imports.push('java.io.InputStream', 'java.util.Properties');
 
                         break;
@@ -1075,7 +1078,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                 sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`);
 
                 _.forEach(_.tail(fields), (field, idx) => {
-                    sb.append(`${arg(field)}${idx !== fields.length - 1 ? ',' : ') {'}`);
+                    sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`);
                 });
 
                 _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`));
@@ -1262,8 +1265,8 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                         const valueFields = _.clone(domain.valueFields);
 
                         if (includeKeyFields) {
-                            _.forEach(domain.keyFields, ({fld}) => {
-                                if (!_.find(valueFields, {name: fld.name}))
+                            _.forEach(domain.keyFields, (fld) => {
+                                if (!_.find(valueFields, {javaFieldName: fld.javaFieldName}))
                                     valueFields.push(fld);
                             });
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
index 07f8dff..49b4aa6 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Properties.service.js
@@ -24,6 +24,23 @@ export default class PropertiesGenerator {
     _collectProperties(bean) {
         const props = [];
 
+        _.forEach(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'BEAN':
+                    props.push(...this._collectProperties(arg.value));
+
+                    break;
+                case 'PROPERTY':
+                case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
+                    props.push(`${arg.value}=${arg.hint}`);
+
+                    break;
+                default:
+                    // No-op.
+            }
+        });
+
         _.forEach(bean.properties, (prop) => {
             switch (prop.clsName) {
                 case 'DATA_SOURCE':
@@ -37,6 +54,7 @@ export default class PropertiesGenerator {
                     break;
                 case 'PROPERTY':
                 case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
                     props.push(`${prop.value}=${prop.hint}`);
 
                     break;
@@ -51,6 +69,7 @@ export default class PropertiesGenerator {
 
                     break;
                 default:
+                    // No-op.
             }
         });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
index 91a85fa..73df25e 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/SpringTransformer.service.js
@@ -20,6 +20,14 @@ import _ from 'lodash';
 import AbstractTransformer from './AbstractTransformer';
 import StringBuilder from './StringBuilder';
 
+const escapeXml = (str) => {
+    return str.replace(/&/g, '&amp;')
+        .replace(/"/g, '&quot;')
+        .replace(/'/g, '&apos;')
+        .replace(/>/g, '&gt;')
+        .replace(/</g, '&lt;');
+};
+
 export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => {
     return class SpringTransformer extends AbstractTransformer {
         static generator = generator;
@@ -84,11 +92,14 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                 switch (clsName) {
                     case 'PROPERTY':
                     case 'PROPERTY_CHAR':
+                    case 'PROPERTY_INT':
                         return `\${${item}}`;
                     case 'java.lang.Class':
                         return JavaTypes.fullClassName(item);
                     case 'long':
                         return `${item}L`;
+                    case 'java.lang.String':
+                        return escapeXml(item);
                     default:
                         return item;
                 }
@@ -99,9 +110,9 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
             return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.');
         }
 
-        static _setCollection(sb, prop, tag) {
+        static _setCollection(sb, prop) {
             sb.startBlock(`<property name="${prop.name}">`);
-            sb.startBlock(`<${tag}>`);
+            sb.startBlock('<list>');
 
             _.forEach(prop.items, (item, idx) => {
                 if (this._isBean(prop.typeClsName)) {
@@ -114,7 +125,7 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
                     sb.append(`<value>${item}</value>`);
             });
 
-            sb.endBlock(`</${tag}>`);
+            sb.endBlock('</list>');
             sb.endBlock('</property>');
         }
 
@@ -201,11 +212,8 @@ export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator'
 
                         break;
                     case 'ARRAY':
-                        this._setCollection(sb, prop, 'array');
-
-                        break;
                     case 'COLLECTION':
-                        this._setCollection(sb, prop, 'list');
+                        this._setCollection(sb, prop);
 
                         break;
                     case 'MAP':

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
index 5ff1e02..f50e493 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
@@ -37,6 +37,14 @@ const DFLT_CACHE = {
     sqlEscapeAll: false,
     storeKeepBinary: false,
     loadPreviousValue: false,
+    cacheStoreFactory: {
+        CacheJdbcPojoStoreFactory: {
+            batchSize: 512,
+            maximumWriteAttempts: 2,
+            parallelLoadCacheMinimumThreshold: 512,
+            sqlEscapeAll: false
+        }
+    },
     readThrough: false,
     writeThrough: false,
     writeBehindEnabled: false,

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
index 1be4b70..726581d 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
@@ -167,7 +167,7 @@ const DFLT_CLUSTER = {
     logger: {
         Log4j: {
             level: {
-                clsName: 'org.apache.logging.log4j.Level'
+                clsName: 'org.apache.log4j.Level'
             }
         },
         Log4j2: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js
index 5005280..4f440a1 100644
--- a/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js
+++ b/modules/web-console/frontend/app/modules/form/field/tooltip.directive.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-const template = '<i class="tipField fa fa-question-circle"></i>';
+const template = '<i class="tipField icon-help"></i>';
 
 export default ['igniteFormFieldTooltip', ['$tooltip', ($tooltip) => {
     const link = ($scope, $element, $attrs, [form, field], $transclude) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js
index 3e470e1..6027765 100644
--- a/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js
+++ b/modules/web-console/frontend/app/modules/form/group/tooltip.directive.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-const template = '<i class="group-legend-btn fa fa-question-circle"></i>';
+const template = '<i class="group-legend-btn icon-help"></i>';
 
 export default ['igniteFormGroupTooltip', ['$tooltip', ($tooltip) => {
     const link = ($scope, $element, $attrs, $ctrls, $transclude) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/form/panel/field.directive.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/form/panel/field.directive.js b/modules/web-console/frontend/app/modules/form/panel/field.directive.js
index 5dc7b07..cf8101a 100644
--- a/modules/web-console/frontend/app/modules/form/panel/field.directive.js
+++ b/modules/web-console/frontend/app/modules/form/panel/field.directive.js
@@ -40,8 +40,8 @@ export default ['igniteFormPanelField', ['$parse', 'IgniteLegacyTable', ($parse,
         else
             saveDefault();
 
-        scope.tableReset = () => {
-            if (!LegacyTable.tableSaveAndReset())
+        scope.tableReset = (trySave) => {
+            if (trySave === false || !LegacyTable.tableSaveAndReset())
                 LegacyTable.tableReset();
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
index addc5f3..0396727 100644
--- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
+++ b/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .modal(tabindex='-1' role='dialog')
     .modal-dialog

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/sql/sql.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.controller.js b/modules/web-console/frontend/app/modules/sql/sql.controller.js
index 0c2be01..4e6e372 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -1244,8 +1244,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         const _closeOldQuery = (paragraph) => {
-            if (paragraph.queryId)
-                return agentMonitor.queryClose(paragraph.resNodeId, paragraph.queryId);
+            const nid = paragraph.resNodeId;
+
+            if (paragraph.queryId && _.find($scope.caches, ({nodes}) => _.includes(nodes, nid)))
+                return agentMonitor.queryClose(nid, paragraph.queryId);
 
             return $q.when();
         };

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade
index 9d8ccbe..e3d1a81 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/client-near-cache.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'clientNearCache'
 -var model = 'backupItem.clientNearConfiguration'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade
index 37bd88d..ffcd568 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/concurrency.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'concurrency'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade
index e9ff143..14f3ab4 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/general.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade
index 724418f..f2d3e2b 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/memory.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'memory'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade
index ba538c2..56f7e64 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-client.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'clientNearCache'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade
index a96b947..9895281 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/near-cache-server.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'serverNearCache'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
index eb74736..b34aba0 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/caches/node-filter.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'nodeFilter'
 -var model = 'backupItem'


[13/19] ignite git commit: ignite-4344 Do not create offheap map on client nodes.

Posted by sb...@apache.org.
ignite-4344 Do not create offheap map on client nodes.


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

Branch: refs/heads/master
Commit: 066691098797be8c01daa0e8dc2ba94d4ad73561
Parents: c06e401
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 1 17:16:28 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 1 17:16:28 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../cache/OffheapCacheOnClientsTest.java        | 143 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 3 files changed, 150 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/06669109/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 0e0d769..0be2072 100755
--- 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
@@ -1286,10 +1286,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         U.startLifecycleAware(lifecycleAwares(cfg, cfgStore));
 
+        boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
+
         GridCacheAffinityManager affMgr = new GridCacheAffinityManager();
         GridCacheEventManager evtMgr = new GridCacheEventManager();
-        GridCacheSwapManager swapMgr = new GridCacheSwapManager(cfg.getCacheMode() == LOCAL ||
-            !GridCacheUtils.isNearEnabled(cfg));
+        GridCacheSwapManager swapMgr = new GridCacheSwapManager(
+            affNode && (cfg.getCacheMode() == LOCAL || !GridCacheUtils.isNearEnabled(cfg)));
         GridCacheEvictionManager evictMgr = new GridCacheEvictionManager();
         GridCacheQueryManager qryMgr = queryManager(cfg);
         CacheContinuousQueryManager contQryMgr = new CacheContinuousQueryManager();
@@ -1302,8 +1304,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         storeMgr.initialize(cfgStore, sesHolders);
 
-        boolean affNode = CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter());
-
         GridCacheContext<?, ?> cacheCtx = new GridCacheContext(
             ctx,
             sharedCtx,
@@ -1427,7 +1427,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
              * 7. GridCacheTtlManager.
              * ===============================================
              */
-            swapMgr = new GridCacheSwapManager(true);
+            swapMgr = new GridCacheSwapManager(affNode);
             evictMgr = new GridCacheEvictionManager();
             evtMgr = new GridCacheEventManager();
             pluginMgr = new CachePluginManager(ctx, cfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/06669109/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.java
new file mode 100644
index 0000000..90985b6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/OffheapCacheOnClientsTest.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.internal.processors.cache;
+
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMemoryMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.offheap.GridOffHeapProcessor;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMemoryMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class OffheapCacheOnClientsTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final String CACHE_NAME = "CACHE_NAME";
+
+    /** */
+    private boolean client;
+
+    /** */
+    private boolean forceSrvMode;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrid(0);
+
+        client = true;
+
+        startGrid(1);
+
+        forceSrvMode = true;
+
+        startGrid(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (client) {
+            cfg.setClientMode(true);
+
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(forceSrvMode);
+        }
+
+        return cfg;
+    }
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOffheapCacheOnClient() throws Exception {
+        try {
+            Ignite client = grid(1);
+
+            testStartCacheOnClient(client, OFFHEAP_TIERED);
+            testStartCacheOnClient(client, OFFHEAP_VALUES);
+            testStartCacheOnClient(client, ONHEAP_TIERED);
+
+            client = grid(2);
+
+            testStartCacheOnClient(client, OFFHEAP_TIERED);
+            testStartCacheOnClient(client, OFFHEAP_VALUES);
+            testStartCacheOnClient(client, ONHEAP_TIERED);
+        }
+        finally {
+            grid(0).destroyCache(CACHE_NAME);
+        }
+    }
+
+    /**
+     * @param client Node.
+     * @param memMode Memory mode.
+     * @throws Exception If failed.
+     */
+    private void testStartCacheOnClient(Ignite client, CacheMemoryMode memMode) throws Exception {
+        assertTrue(client.configuration().isClientMode());
+
+        try {
+            client.createCache(new CacheConfiguration(CACHE_NAME)
+                .setCacheMode(REPLICATED)
+                .setOffHeapMaxMemory(1024 * 1024)
+                .setMemoryMode(memMode));
+
+            IgniteCache<Integer, Integer> cache = client.cache(CACHE_NAME);
+
+            assertNotNull(cache);
+
+            cache.put(1, 1);
+            assertEquals((Integer)1, cache.get(1));
+
+            GridOffHeapProcessor offheap = ((IgniteKernal)client).cachex(CACHE_NAME).context().offheap();
+
+            assertNotNull(offheap);
+
+            ConcurrentMap offheapMaps = GridTestUtils.getFieldValue(offheap, "offheap");
+            assertNotNull(offheapMaps);
+
+            assertEquals(0,offheapMaps.size());
+        }
+        finally {
+            client.destroyCache(CACHE_NAME);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/06669109/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 6edfd09..f632f67 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
@@ -39,6 +39,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheEntryProcessorNode
 import org.apache.ignite.internal.processors.cache.IgniteCacheIncrementTxTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionMapUpdateTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheAndNodeStop;
+import org.apache.ignite.internal.processors.cache.OffheapCacheOnClientsTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLoadingConcurrentGridStartSelfTestAllowOverwrite;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLockReleaseNodeLeaveTest;
@@ -266,6 +267,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteNoCustomEventsOnNodeStart.class));
 
         suite.addTest(new TestSuite(CacheExchangeMessageDuplicatedStateTest.class));
+        suite.addTest(new TestSuite(OffheapCacheOnClientsTest.class));
 
         return suite;
     }


[07/19] ignite git commit: IGNITE-3958 Fixed "Client node should not start rest processor".

Posted by sb...@apache.org.
IGNITE-3958 Fixed "Client node should not start rest processor".


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

Branch: refs/heads/master
Commit: 9c6824b4f33fbdead64299d9e0c34365d5d4a570
Parents: 7d88c5b
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Nov 24 16:27:05 2016 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Nov 24 16:28:26 2016 +0300

----------------------------------------------------------------------
 .../rest/RestProcessorMultiStartSelfTest.java   | 48 +++++++++++++++++++-
 .../apache/ignite/IgniteSystemProperties.java   |  6 +++
 .../apache/ignite/internal/IgniteKernal.java    | 12 ++++-
 .../processors/rest/GridRestProcessor.java      | 15 ++++++
 4 files changed, 79 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9c6824b4/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java
index 7714dbf..5b88079 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorMultiStartSelfTest.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.processors.rest;
 
+import java.util.Map;
 import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -28,15 +30,26 @@ public class RestProcessorMultiStartSelfTest extends GridCommonAbstractTest {
     /** */
     private static final int GRID_CNT = 3;
 
+    /** */
+    private static boolean client = false;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         cfg.setConnectorConfiguration(new ConnectorConfiguration());
+        cfg.setClientMode(client);
 
         return cfg;
     }
 
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        client = false;
+    }
+
     /**
      * Test that multiple nodes can start with JETTY enabled.
      *
@@ -53,4 +66,37 @@ public class RestProcessorMultiStartSelfTest extends GridCommonAbstractTest {
             stopAllGrids();
         }
     }
-}
\ No newline at end of file
+
+    /**
+     * Test that multiple nodes can start with JETTY enabled.
+     *
+     * @throws Exception If failed.
+     */
+    public void testMultiStartWithClient() throws Exception {
+        try {
+            int clnIdx = GRID_CNT - 1;
+
+            for (int i = 0; i < clnIdx; i++) {
+                startGrid(i);
+
+                GridRestProcessor rest = grid(i).context().rest();
+
+                assertNotNull(rest);
+                assertFalse(((Map)GridTestUtils.getFieldValue(rest, "handlers")).isEmpty());
+            }
+
+            client = true;
+
+            startGrid(clnIdx);
+
+            GridRestProcessor rest = grid(GRID_CNT - 1).context().rest();
+
+            // Check that rest processor doesn't start.
+            assertNotNull(rest);
+            assertTrue(((Map)GridTestUtils.getFieldValue(rest, "handlers")).isEmpty());
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c6824b4/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 043c95a..de6cbed 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -110,6 +110,12 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_REST_MAX_TASK_RESULTS = "IGNITE_REST_MAX_TASK_RESULTS";
 
     /**
+     * This property allows to override default behavior that rest processor
+     * doesn't start on client node. If set {@code true} than rest processor will be started on client node.
+     */
+    public static final String IGNITE_REST_START_ON_CLIENT = "IGNITE_REST_START_ON_CLIENT";
+
+    /**
      * This property defines the maximum number of attempts to remap near get to the same
      * primary node. Remapping may be needed when topology is changed concurrently with
      * get operation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c6824b4/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 1963509..ef9c651 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
@@ -179,6 +179,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONFIG_URL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
@@ -1632,7 +1633,16 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     private boolean isRestEnabled() {
         assert cfg != null;
 
-        return cfg.getConnectorConfiguration() != null;
+        return cfg.getConnectorConfiguration() != null &&
+            // By default rest processor doesn't start on client nodes.
+            (!isClientNode() || (isClientNode() && IgniteSystemProperties.getBoolean(IGNITE_REST_START_ON_CLIENT)));
+    }
+
+    /**
+     * @return {@code True} if node client or daemon otherwise {@code false}.
+     */
+    private boolean isClientNode() {
+        return cfg.isClientMode() || cfg.isDaemon();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9c6824b4/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 6d20547..3f04785 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -75,6 +75,7 @@ import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.thread.IgniteThread;
 import org.jsr166.LongAdder8;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_AUTH_FAILED;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_FAILED;
 import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_SECURITY_CHECK_FAILED;
@@ -436,6 +437,13 @@ public class GridRestProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         if (isRestEnabled()) {
+            if (notStartOnClient()) {
+                U.quietAndInfo(log, "REST protocols do not start on client node. " +
+                    "To start the protocols on client node set '-DIGNITE_REST_START_ON_CLIENT=true' system property.");
+
+                return;
+            }
+
             // Register handlers.
             addHandler(new GridCacheCommandHandler(ctx));
             addHandler(new GridTaskCommandHandler(ctx));
@@ -471,6 +479,13 @@ public class GridRestProcessor extends GridProcessorAdapter {
         }
     }
 
+    /**
+     * @return {@code True} if rest processor should not start on client node.
+     */
+    private boolean notStartOnClient() {
+        return ctx.clientNode() && !IgniteSystemProperties.getBoolean(IGNITE_REST_START_ON_CLIENT);
+    }
+
     /** {@inheritDoc} */
     @Override public void onKernalStart() throws IgniteCheckedException {
         if (isRestEnabled()) {


[03/19] ignite git commit: Web console beta-6.

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade
index ab569e7..f5d6b7d 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/domains/store.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'store'
 -var model = 'backupItem'
@@ -43,15 +43,15 @@ mixin table-db-field-edit(tbl, prefix, focusId, index)
     .col-xs-3.col-sm-3.col-md-3
         .fieldSep /
         .input-tip
-            input.form-control(id=databaseNameId ignite-on-enter-focus-move=databaseTypeId type='text' ng-model=databaseNameModel placeholder='DB name' ignite-on-enter='#{javaNameModel} = #{javaNameModel} ? #{javaNameModel} : #{databaseNameModel}' ignite-on-escape='tableReset()')
+            input.form-control(id=databaseNameId ignite-on-enter-focus-move=databaseTypeId type='text' ng-model=databaseNameModel placeholder='DB name' ignite-on-enter='#{javaNameModel} = #{javaNameModel} ? #{javaNameModel} : #{databaseNameModel}' ignite-on-escape='tableReset(false)')
     .col-xs-3.col-sm-3.col-md-3
         .fieldSep /
         .input-tip
-            button.select-toggle.form-control(id=databaseTypeId ignite-on-enter-focus-move=javaNameId ng-model=databaseTypeModel data-placeholder='DB type' ng-class='{placeholder: !#{databaseTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJdbcTypes}}' ignite-on-escape='tableReset()' tabindex='0')
+            button.select-toggle.form-control(id=databaseTypeId ignite-on-enter-focus-move=javaNameId ng-model=databaseTypeModel data-placeholder='DB type' ng-class='{placeholder: !#{databaseTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJdbcTypes}}' ignite-on-escape='tableReset(false)' tabindex='0')
     .col-xs-3.col-sm-3.col-md-3
         .fieldSep /
         .input-tip
-            input.form-control(id=javaNameId ignite-on-enter-focus-move=javaTypeId type='text' ng-model=javaNameModel placeholder='Java name' ignite-on-escape='tableReset()')
+            input.form-control(id=javaNameId ignite-on-enter-focus-move=javaTypeId type='text' ng-model=javaNameModel placeholder='Java name' ignite-on-escape='tableReset(false)')
     .col-xs-3.col-sm-3.col-md-3
         -var btnVisible = 'tableDbFieldSaveVisible(' + tbl + ', ' + index +')'
         -var btnSave = 'tableDbFieldSave(' + tbl + ', ' + index +')'
@@ -59,12 +59,12 @@ mixin table-db-field-edit(tbl, prefix, focusId, index)
 
         +btn-save(btnVisible, btnSave)
         .input-tip
-            button.select-toggle.form-control(id=javaTypeId ng-model=javaTypeModel data-placeholder='Java type' ng-class='{placeholder: !#{javaTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJavaTypes}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()' tabindex='0')
+            button.select-toggle.form-control(id=javaTypeId ng-model=javaTypeModel data-placeholder='Java type' ng-class='{placeholder: !#{javaTypeModel}}' bs-select bs-options='item.value as item.label for item in {{supportedJavaTypes}}' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)' tabindex='0')
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")')
         ignite-form-panel-chevron
-        label Domain model for cache store
+        label(id='store-title') Domain model for cache store
         ignite-form-field-tooltip.tipLabel
             | Domain model properties for binding database with cache via POJO cache store#[br]
             | #[a(href="https://apacheignite.readme.io/docs/persistent-store" target="_blank") More info]

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade
index 1b3a00a..f6ac89f 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/dual.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'dualMode'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade
index edc2352..16f3749 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/fragmentizer.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'fragmentizer'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade
index d12a6e8..62cda77 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/general.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'general'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade
index f8da2f9..83c5de6 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/ipc.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'ipc'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade
index ca22dc5..dc0e9fc 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/misc.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'misc'
 -var model = 'backupItem'
@@ -32,7 +32,7 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index)
     .col-xs-8.col-sm-8.col-md-8
         .fieldSep /
         .input-tip
-            input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder='Path' ignite-on-escape='tableReset()')
+            input.form-control(id=keyFocusId ignite-on-enter-focus-move=valFocusId type='text' ng-model=keyModel placeholder='Path' ignite-on-escape='tableReset(false)')
     .col-xs-4.col-sm-4.col-md-4
         -var arg = keyModel + ', ' + valModel
         -var btnVisible = 'tablePairSaveVisible(tblPathModes, ' + index + ')'
@@ -40,7 +40,7 @@ mixin table-igfs-path-mode-edit(prefix, focusId, index)
         -var btnVisibleAndSave = btnVisible + ' && ' + btnSave
         +btn-save(btnVisible, btnSave)
         .input-tip
-            button.select-toggle.form-control(id=valFocusId bs-select ng-model=valModel data-placeholder='Mode' bs-options='item.value as item.label for item in igfsModes' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset()')
+            button.select-toggle.form-control(id=valFocusId bs-select ng-model=valModel data-placeholder='Mode' bs-options='item.value as item.label for item in igfsModes' tabindex='0' ignite-on-enter=btnVisibleAndSave ignite-on-escape='tableReset(false)')
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle='' ng-click='ui.loadPanel("#{form}")')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade
index 1143662..b605e77 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade
+++ b/modules/web-console/frontend/app/modules/states/configuration/igfs/secondary.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../../../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 -var form = 'secondaryFileSystem'
 -var model = 'backupItem'

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
index 4bfbf48..d739c43 100644
--- a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.controller.js
@@ -223,6 +223,10 @@ export default [
             return false;
         }
 
+        function escapeFileName(name) {
+            return name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+        }
+
         $scope.selectItem = (cluster) => {
             delete ctrl.cluster;
 
@@ -318,8 +322,8 @@ export default [
             const srcPath = 'src/main/java';
             const resourcesPath = 'src/main/resources';
 
-            const serverXml = `${cluster.name}-server.xml`;
-            const clientXml = `${cluster.name}-client.xml`;
+            const serverXml = `${escapeFileName(cluster.name)}-server.xml`;
+            const clientXml = `${escapeFileName(cluster.name)}-client.xml`;
 
             const metaPath = `${resourcesPath}/META-INF`;
 
@@ -329,7 +333,7 @@ export default [
             const cfgPath = `${srcPath}/config`;
 
             zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString());
-            zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString());
+            zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString());
 
             if (java.isDemoConfigured(cluster, $root.IgniteDemoMode)) {
                 zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup',
@@ -370,7 +374,7 @@ export default [
             $generatorOptional.optionalContent(zip, cluster);
 
             zip.generateAsync({type: 'blob', compression: 'DEFLATE', mimeType: 'application/octet-stream'})
-                .then((blob) => saver.saveAs(blob, cluster.name + '-project.zip'));
+                .then((blob) => saver.saveAs(blob, escapeFileName(cluster.name) + '-project.zip'));
         };
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/controllers/clusters-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/clusters-controller.js b/modules/web-console/frontend/controllers/clusters-controller.js
index f9096f7..f92a2f1 100644
--- a/modules/web-console/frontend/controllers/clusters-controller.js
+++ b/modules/web-console/frontend/controllers/clusters-controller.js
@@ -594,6 +594,13 @@ export default ['clustersController', [
             }));
         }
 
+        function checkODBC(item) {
+            if (_.get(item, 'odbc.odbcEnabled') && _.get(item, 'marshaller.kind'))
+                return ErrorPopover.show('odbcEnabledInput', 'ODBC can only be used with BinaryMarshaller', $scope.ui, 'odbcConfiguration');
+
+            return true;
+        }
+
         function checkSwapConfiguration(item) {
             const swapKind = item.swapSpaceSpi && item.swapSpaceSpi.kind;
 
@@ -674,6 +681,9 @@ export default ['clustersController', [
             if (!checkLoadBalancingConfiguration(item))
                 return false;
 
+            if (!checkODBC(item))
+                return false;
+
             if (!checkSwapConfiguration(item))
                 return false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/controllers/domains-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/domains-controller.js b/modules/web-console/frontend/controllers/domains-controller.js
index 0a79d82..2d7b875 100644
--- a/modules/web-console/frontend/controllers/domains-controller.js
+++ b/modules/web-console/frontend/controllers/domains-controller.js
@@ -95,7 +95,7 @@ export default ['domainsController', [
 
                         break;
 
-                    case 'indexes':
+                    case 'table-indexes':
                         if ($scope.tableIndexSaveVisible(field, index))
                             return $scope.tableIndexSave(field, index, stopEdit);
 
@@ -1355,6 +1355,9 @@ export default ['domainsController', [
             if (!LegacyUtils.domainForStoreConfigured(item) && !LegacyUtils.domainForQueryConfigured(item) && item.queryMetadata === 'Configuration')
                 return ErrorPopover.show('query-title', 'SQL query domain model should be configured', $scope.ui, 'query');
 
+            if (!LegacyUtils.domainForStoreConfigured(item) && item.generatePojo)
+                return ErrorPopover.show('store-title', 'Domain model for cache store should be configured when generation of POJO classes is enabled', $scope.ui, 'store');
+
             return true;
         }
 
@@ -1437,7 +1440,8 @@ export default ['domainsController', [
         }
 
         function _newNameIsValidJavaClass(newName) {
-            return LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-nameInput');
+            return !$scope.backupItem.generatePojo ||
+                LegacyUtils.isValidJavaClass('New name for value type', newName, false, 'copy-new-nameInput');
         }
 
         // Save domain model with new name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js
index fe8bb09..4935976 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/tasks/jade.js
@@ -19,10 +19,10 @@ import gulp from 'gulp';
 import ll from 'gulp-ll';
 import jade from 'gulp-jade';
 
-import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, destDir } from '../paths';
+import { jadeViewsPaths, jadeAppModulePaths, jadeModulePaths, destDir, rootDir } from '../paths';
 
 const jadeOptions = {
-    basedir: './',
+    basedir: rootDir,
     cache: true
 };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js b/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js
deleted file mode 100644
index ad4108d..0000000
--- a/modules/web-console/frontend/gulpfile.babel.js/tasks/test.js
+++ /dev/null
@@ -1,92 +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.
- */
-
-import gulp from 'gulp';
-import karmaBabelPreprocessor from 'karma-babel-preprocessor';
-import karmaPhantomjsLauncher from 'karma-phantomjs-launcher';
-import karmaWebpack from 'karma-webpack';
-import karmaJasmine from 'karma-jasmine';
-
-import {Server} from 'karma';
-
-import {rootDir} from '../paths';
-
-gulp.task('test', (cb) => {
-    new Server({
-        // Base path that will be used to resolve all patterns (eg. files, exclude).
-        basePath: rootDir,
-
-        // Frameworks to use available frameworks: https://npmjs.org/browse/keyword/karma-adapter
-        frameworks: ['jasmine'],
-
-        // List of files / patterns to load in the browser.
-        files: [
-            'test/**/*.test.js'
-        ],
-
-        plugins: [
-            karmaBabelPreprocessor,
-            karmaPhantomjsLauncher,
-            karmaWebpack,
-            karmaJasmine
-        ],
-
-        // Preprocess matching files before serving them to the browser
-        // available preprocessors: https://npmjs.org/browse/keyword/karma-preprocessor.
-        preprocessors: {
-            'test/**/*.js': ['webpack']
-        },
-
-        webpack: {
-            devtool: 'inline-source-map',
-            module: {
-                loaders: [
-                    {test: /\.js/, loaders: ['babel'], exclude: /node_modules/}
-                ]
-            },
-            resolve: {
-                extensions: ['', '.js']
-            }
-        },
-
-        // Test results reporter to use
-        // possible values: 'dots', 'progress'
-        // available reporters: https://npmjs.org/browse/keyword/karma-reporter.
-        reporters: ['progress'],
-
-        // web server port
-        port: 9876,
-
-        // enable / disable colors in the output (reporters and logs)
-        colors: true,
-
-        // enable / disable watching file and executing tests whenever any file changes
-        autoWatch: true,
-
-        // start these browsers
-        // available browser launchers: https://npmjs.org/browse/keyword/karma-launcher
-        browsers: ['PhantomJS'],
-
-        // Continuous Integration mode
-        // if true, Karma captures browsers, runs the tests and exits
-        singleRun: true,
-
-        // Concurrency level
-        // how many browser should be started simultaneous
-        concurrency: Infinity
-    }, cb).start();
-});

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
index f6c2d87..2463d24 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
@@ -77,7 +77,10 @@ export default () => {
                 './'
             ]
         },
-
+        jade: {
+            basedir: rootDir,
+            locals: {}
+        },
         // Modules resolvers.
         /* global require */
         module: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
index 1550dfa..cad9133 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/development.js
@@ -55,6 +55,11 @@ export default () => {
                     changeOrigin: true,
                     ws: true
                 },
+                '/agents': {
+                    target: 'http://localhost:3000',
+                    changeOrigin: true,
+                    ws: true
+                },
                 '/api/v1/*': {
                     target: 'http://localhost:3000',
                     changeOrigin: true,

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index fe8c295..b511ca1 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -95,7 +95,7 @@
     "html-loader": "^0.4.3",
     "html-webpack-plugin": "^2.21.0",
     "jade": "^1.11.0",
-    "jade-html-loader": "0.0.3",
+    "jade-html-loader": "git://github.com/courcelan/jade-html-loader",
     "jasmine-core": "^2.4.1",
     "json-loader": "^0.5.4",
     "karma": "^0.13.22",

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
index 1c8f325..15ee60c 100644
--- a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
+++ b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
@@ -30,3 +30,21 @@ $fa-font-path: '~font-awesome/fonts';
 @import '~font-awesome/scss/rotated-flipped';
 @import '~font-awesome/scss/stacked';
 @import '~font-awesome/scss/icons';
+
+.fa {
+  cursor: pointer;
+}
+
+.icon-help {
+  @extend .fa;
+  @extend .fa-question-circle-o;
+
+  cursor: default;
+}
+
+.icon-confirm {
+  @extend .fa;
+  @extend .fa-question-circle-o;
+
+  cursor: default;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/public/stylesheets/style.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/style.scss b/modules/web-console/frontend/public/stylesheets/style.scss
index 0f8f49a..172abf4 100644
--- a/modules/web-console/frontend/public/stylesheets/style.scss
+++ b/modules/web-console/frontend/public/stylesheets/style.scss
@@ -541,6 +541,12 @@ i.btn {
         margin-left: 0;
     }
 
+    button.btn[disabled="disabled"] {
+        i {
+            cursor: not-allowed;
+        }
+    }
+
     > .btn + .dropdown-toggle {
         margin-right: 0;
         padding: 3px 6px;
@@ -1350,9 +1356,6 @@ label {
     margin: 0 5px;
 }
 
-.fa {
-    cursor: pointer;
-}
 
 .fa-cursor-default {
     cursor: default !important;
@@ -1372,10 +1375,6 @@ label {
     margin-right: 5px;
 }
 
-.fa-question-circle {
-    cursor: default;
-}
-
 label.required:after {
     color: $brand-primary;
     content: ' *';

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/caches.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/caches.jade b/modules/web-console/frontend/views/configuration/caches.jade
index 74d5505..4a4cf2e 100644
--- a/modules/web-console/frontend/views/configuration/caches.jade
+++ b/modules/web-console/frontend/views/configuration/caches.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .docs-header
     h1 Configure Ignite Caches
@@ -36,19 +36,19 @@ include ../../app/helpers/jade/mixins.jade
             div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
                 form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
                     .panel-group
-                        include ../../app/modules/states/configuration/caches/general.jade
-                        include ../../app/modules/states/configuration/caches/memory.jade
-                        include ../../app/modules/states/configuration/caches/query.jade
-                        include ../../app/modules/states/configuration/caches/store.jade
+                        include /app/modules/states/configuration/caches/general.jade
+                        include /app/modules/states/configuration/caches/memory.jade
+                        include /app/modules/states/configuration/caches/query.jade
+                        include /app/modules/states/configuration/caches/store.jade
 
                         +advanced-options-toggle-default
 
                         div(ng-show='ui.expanded')
-                            include ../../app/modules/states/configuration/caches/concurrency.jade
-                            include ../../app/modules/states/configuration/caches/near-cache-client.jade
-                            include ../../app/modules/states/configuration/caches/near-cache-server.jade
-                            include ../../app/modules/states/configuration/caches/node-filter.jade
-                            include ../../app/modules/states/configuration/caches/rebalance.jade
-                            include ../../app/modules/states/configuration/caches/statistics.jade
+                            include /app/modules/states/configuration/caches/concurrency.jade
+                            include /app/modules/states/configuration/caches/near-cache-client.jade
+                            include /app/modules/states/configuration/caches/near-cache-server.jade
+                            include /app/modules/states/configuration/caches/node-filter.jade
+                            include /app/modules/states/configuration/caches/rebalance.jade
+                            include /app/modules/states/configuration/caches/statistics.jade
 
                             +advanced-options-toggle-default

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/clusters.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/clusters.jade b/modules/web-console/frontend/views/configuration/clusters.jade
index 8a3a194..7826464 100644
--- a/modules/web-console/frontend/views/configuration/clusters.jade
+++ b/modules/web-console/frontend/views/configuration/clusters.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .docs-header
     h1 Configure Ignite Clusters
@@ -36,33 +36,33 @@ include ../../app/helpers/jade/mixins.jade
             div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
                 form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
                     .panel-group
-                        include ../../app/modules/states/configuration/clusters/general.jade
+                        include /app/modules/states/configuration/clusters/general.jade
 
                         +advanced-options-toggle-default
 
                         div(ng-show='ui.expanded')
-                            include ../../app/modules/states/configuration/clusters/atomic.jade
-                            include ../../app/modules/states/configuration/clusters/binary.jade
-                            include ../../app/modules/states/configuration/clusters/cache-key-cfg.jade
-                            include ../../app/modules/states/configuration/clusters/checkpoint.jade
-                            include ../../app/modules/states/configuration/clusters/collision.jade
-                            include ../../app/modules/states/configuration/clusters/communication.jade
-                            include ../../app/modules/states/configuration/clusters/connector.jade
-                            include ../../app/modules/states/configuration/clusters/deployment.jade
-                            include ../../app/modules/states/configuration/clusters/discovery.jade
-                            include ../../app/modules/states/configuration/clusters/events.jade
-                            include ../../app/modules/states/configuration/clusters/failover.jade
-                            include ../../app/modules/states/configuration/clusters/igfs.jade
-                            include ../../app/modules/states/configuration/clusters/load-balancing.jade
-                            include ../../app/modules/states/configuration/clusters/logger.jade
-                            include ../../app/modules/states/configuration/clusters/marshaller.jade
-                            include ../../app/modules/states/configuration/clusters/metrics.jade
-                            include ../../app/modules/states/configuration/clusters/odbc.jade
-                            include ../../app/modules/states/configuration/clusters/ssl.jade
-                            include ../../app/modules/states/configuration/clusters/swap.jade
-                            include ../../app/modules/states/configuration/clusters/thread.jade
-                            include ../../app/modules/states/configuration/clusters/time.jade
-                            include ../../app/modules/states/configuration/clusters/transactions.jade
-                            include ../../app/modules/states/configuration/clusters/attributes.jade
+                            include /app/modules/states/configuration/clusters/atomic.jade
+                            include /app/modules/states/configuration/clusters/binary.jade
+                            include /app/modules/states/configuration/clusters/cache-key-cfg.jade
+                            include /app/modules/states/configuration/clusters/checkpoint.jade
+                            include /app/modules/states/configuration/clusters/collision.jade
+                            include /app/modules/states/configuration/clusters/communication.jade
+                            include /app/modules/states/configuration/clusters/connector.jade
+                            include /app/modules/states/configuration/clusters/deployment.jade
+                            include /app/modules/states/configuration/clusters/discovery.jade
+                            include /app/modules/states/configuration/clusters/events.jade
+                            include /app/modules/states/configuration/clusters/failover.jade
+                            include /app/modules/states/configuration/clusters/igfs.jade
+                            include /app/modules/states/configuration/clusters/load-balancing.jade
+                            include /app/modules/states/configuration/clusters/logger.jade
+                            include /app/modules/states/configuration/clusters/marshaller.jade
+                            include /app/modules/states/configuration/clusters/metrics.jade
+                            include /app/modules/states/configuration/clusters/odbc.jade
+                            include /app/modules/states/configuration/clusters/ssl.jade
+                            include /app/modules/states/configuration/clusters/swap.jade
+                            include /app/modules/states/configuration/clusters/thread.jade
+                            include /app/modules/states/configuration/clusters/time.jade
+                            include /app/modules/states/configuration/clusters/transactions.jade
+                            include /app/modules/states/configuration/clusters/attributes.jade
 
                             +advanced-options-toggle-default

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/domains-import.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/domains-import.jade b/modules/web-console/frontend/views/configuration/domains-import.jade
index 99014bc..e4f95bc 100644
--- a/modules/web-console/frontend/views/configuration/domains-import.jade
+++ b/modules/web-console/frontend/views/configuration/domains-import.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 mixin chk(mdl, change, tip)
     input(type='checkbox' ng-model=mdl ng-change=change bs-tooltip='' data-title=tip data-trigger='hover' data-placement='top')
@@ -133,7 +133,7 @@ mixin td-ellipses-lbl(w, lbl)
                                                     td
                     .settings-row
                         label Defaults to be applied for filtered tables
-                        i.tipLabel.fa.fa-question-circle(bs-tooltip='' data-title='Select and apply options for caches generation')
+                        i.tipLabel.icon-help(bs-tooltip='' data-title='Select and apply options for caches generation')
                     .settings-row
                         .col-sm-11
                             .col-sm-6(style='padding-right: 5px')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/domains.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/domains.jade b/modules/web-console/frontend/views/configuration/domains.jade
index 5912343..14e9ca5 100644
--- a/modules/web-console/frontend/views/configuration/domains.jade
+++ b/modules/web-console/frontend/views/configuration/domains.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .docs-header
     h1 Configure Domain Model And SQL Queries
@@ -61,6 +61,6 @@ include ../../app/helpers/jade/mixins.jade
             div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
                 form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
                     .panel-group
-                        include ../../app/modules/states/configuration/domains/general.jade
-                        include ../../app/modules/states/configuration/domains/query.jade
-                        include ../../app/modules/states/configuration/domains/store.jade
+                        include /app/modules/states/configuration/domains/general.jade
+                        include /app/modules/states/configuration/domains/query.jade
+                        include /app/modules/states/configuration/domains/store.jade

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/igfs.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/igfs.jade b/modules/web-console/frontend/views/configuration/igfs.jade
index 16e9f28..10c902b 100644
--- a/modules/web-console/frontend/views/configuration/igfs.jade
+++ b/modules/web-console/frontend/views/configuration/igfs.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .docs-header
     h1 Configure Ignite In-memory File Systems
@@ -37,15 +37,15 @@ include ../../app/helpers/jade/mixins.jade
             div(bs-collapse='' data-allow-multiple='true' ng-model='ui.activePanels')
                 form.form-horizontal(name='ui.inputForm' novalidate ng-if='contentVisible()')
                     .panel-group
-                        include ../../app/modules/states/configuration/igfs/general.jade
+                        include /app/modules/states/configuration/igfs/general.jade
 
                         +advanced-options-toggle-default
 
                         div(ng-show='ui.expanded')
-                            include ../../app/modules/states/configuration/igfs/secondary.jade
-                            include ../../app/modules/states/configuration/igfs/ipc.jade
-                            include ../../app/modules/states/configuration/igfs/fragmentizer.jade
-                            include ../../app/modules/states/configuration/igfs/dual.jade
-                            include ../../app/modules/states/configuration/igfs/misc.jade
+                            include /app/modules/states/configuration/igfs/secondary.jade
+                            include /app/modules/states/configuration/igfs/ipc.jade
+                            include /app/modules/states/configuration/igfs/fragmentizer.jade
+                            include /app/modules/states/configuration/igfs/dual.jade
+                            include /app/modules/states/configuration/igfs/misc.jade
 
                             +advanced-options-toggle-default

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/configuration/summary.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/configuration/summary.jade b/modules/web-console/frontend/views/configuration/summary.jade
index d041fea..9a6e553 100644
--- a/modules/web-console/frontend/views/configuration/summary.jade
+++ b/modules/web-console/frontend/views/configuration/summary.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 mixin hard-link(ref, txt)
     a(style='color:#ec1c24' href=ref target='_blank') #{txt}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/includes/header.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/includes/header.jade b/modules/web-console/frontend/views/includes/header.jade
index 9ef09aa..99bcea9 100644
--- a/modules/web-console/frontend/views/includes/header.jade
+++ b/modules/web-console/frontend/views/includes/header.jade
@@ -24,22 +24,23 @@ header#header.header
             td(ng-if='$root.user' style='padding-top: 20px')
                 ul.nav.navbar-nav(ignite-sidebar ignite-navbar)
                     li(ng-class='{active: $state.includes("base.configuration")}' bs-dropdown='sidebar.items' data-placement='bottom-right' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()')
-                        a.dropdown-toggle Configuration
+                        a.dropdown-toggle Configure
                             span.caret
 
                 ul.nav.navbar-nav(ng-controller='notebookController')
                     li.sql-notebooks(ng-if='IgniteDemoMode' ng-class='{active: $state.includes("base.sql")}')
-                        a(ui-sref='base.sql.demo') SQL
+                        a(ui-sref='base.sql.demo') Queries
 
                     li.sql-notebooks(ng-if='!IgniteDemoMode && !notebooks.length' ng-class='{active: $state.includes("base.sql")}')
-                        a(ng-click='createNotebook()') SQL
+                        a(ng-click='createNotebook()') Queries
 
-                    li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebooks' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup="true" aria-expanded="false")
-                        a.dropdown-toggle SQL
+                    li.sql-notebooks(ng-if='!IgniteDemoMode && notebooks.length' ng-class='{active: $state.includes("base.sql")}' bs-dropdown='notebooks' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup='true' aria-expanded='false')
+                        a.dropdown-toggle Queries
                             span.caret
 
-                    li(ui-sref-active='active'  ng-repeat='item in navbar.items')
-                        a(ui-sref='{{::item.sref}}') {{::item.text}}
+                    li(ui-sref-active='active' ng-repeat='item in navbar.items' ng-class='{active: $state.includes("base.monitoring")}' bs-dropdown='item.children' data-placement='bottom-left' data-trigger='hover focus' data-container='self' data-animation='' ng-click='$event.stopPropagation()' aria-haspopup='true' aria-expanded='false')
+                        a.dropdown-toggle {{::item.text}}
+                            span.caret
 
                 a(ng-controller='demoController')
                     button.btn.btn-info(ng-if='IgniteDemoMode' ng-click='closeDemo()') Close demo

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/settings/profile.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/settings/profile.jade b/modules/web-console/frontend/views/settings/profile.jade
index bc4965a..6f91f8f 100644
--- a/modules/web-console/frontend/views/settings/profile.jade
+++ b/modules/web-console/frontend/views/settings/profile.jade
@@ -56,7 +56,7 @@ mixin lbl(txt)
                             label {{user.token || 'No security token. Regenerate please.'}}
                             i.tipLabel.fa.fa-refresh(ng-click='generateToken()' bs-tooltip='' data-title='Generate random security token')
                             i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{user.token}}' bs-tooltip='' data-title='Copy security token to clipboard')
-                            i.tipLabel.fa.fa-question-circle(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent')
+                            i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent')
                     .details-row
                         .advanced-options
                             i.fa(

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/sql/sql.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/sql.jade b/modules/web-console/frontend/views/sql/sql.jade
index 4a596b9..e3f6461 100644
--- a/modules/web-console/frontend/views/sql/sql.jade
+++ b/modules/web-console/frontend/views/sql/sql.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/directives/ui-grid-settings/ui-grid-settings.jade
+include /app/directives/ui-grid-settings/ui-grid-settings.jade
 
 mixin btn-toolbar(btn, click, tip, focusId)
     i.btn.btn-default.fa(class=btn ng-click=click bs-tooltip='' data-title=tip ignite-on-click-focus=focusId data-trigger='hover' data-placement='bottom')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/templates/agent-download.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/agent-download.jade b/modules/web-console/frontend/views/templates/agent-download.jade
index a6da3d1..20b6b03 100644
--- a/modules/web-console/frontend/views/templates/agent-download.jade
+++ b/modules/web-console/frontend/views/templates/agent-download.jade
@@ -36,7 +36,7 @@
                 .details-row(ng-show='agentLoad.showToken')
                     label.labelField Security token: {{user.token}}
                     i.tipLabel.fa.fa-clipboard(ignite-copy-to-clipboard='{{user.token}}' bs-tooltip='' data-title='Copy security token to clipboard')
-                    i.tipLabel.fa.fa-question-circle(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent')
+                    i.tipLabel.icon-help(ng-if=lines bs-tooltip='' data-title='The security token is used for authorization of web agent')
             .agent-download(ng-if='hasAgents')
                 p Connection to Ignite Web Agent is established, but agent failed to connect to Ignite Node
                 p Please check the following:

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/templates/batch-confirm.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/batch-confirm.jade b/modules/web-console/frontend/views/templates/batch-confirm.jade
index c5d3775..0b74a4e 100644
--- a/modules/web-console/frontend/views/templates/batch-confirm.jade
+++ b/modules/web-console/frontend/views/templates/batch-confirm.jade
@@ -20,7 +20,7 @@
             .modal-header
                 button.close(ng-click='cancel()' aria-hidden='true') &times;
                 h4.modal-title 
-                    i.fa.fa-question
+                    i.icon-confirm
                     | Confirmation
             .modal-body(ng-show='content')
                 p(ng-bind-html='content' style='text-align: center')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/templates/clone.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/clone.jade b/modules/web-console/frontend/views/templates/clone.jade
index 6cf93ca..99ec58b 100644
--- a/modules/web-console/frontend/views/templates/clone.jade
+++ b/modules/web-console/frontend/views/templates/clone.jade
@@ -14,7 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-include ../../app/helpers/jade/mixins.jade
+include /app/helpers/jade/mixins.jade
 
 .modal(tabindex='-1' role='dialog')
     .modal-dialog

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/frontend/views/templates/confirm.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/confirm.jade b/modules/web-console/frontend/views/templates/confirm.jade
index f9f966b..0031498 100644
--- a/modules/web-console/frontend/views/templates/confirm.jade
+++ b/modules/web-console/frontend/views/templates/confirm.jade
@@ -20,7 +20,7 @@
             .modal-header
                 button.close(ng-click='confirmCancel()' aria-hidden='true') &times;
                 h4.modal-title 
-                    i.fa.fa-question-circle-o
+                    i.icon-confirm
                     | Confirmation
             .modal-body(ng-show='content')
                 p(ng-bind-html='content' style='text-align: center;')

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/README.txt
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/README.txt b/modules/web-console/web-agent/README.txt
index c6e625b..cc0c80f 100644
--- a/modules/web-console/web-agent/README.txt
+++ b/modules/web-console/web-agent/README.txt
@@ -24,7 +24,7 @@ Configuration file:
 
   Example configuration file:
     tokens=1a2b3c4d5f,2j1s134d12
-    serverURI=https://console.example.com:3001
+    serverURI=https://console.example.com
 
 Security tokens:
   1) By default security token of current user will be included into "default.properties" inside downloaded "ignite-web-agent-x.x.x.zip".
@@ -47,7 +47,7 @@ Options:
      http://localhost:8080
   -s, --server-uri
      URI for connect to Ignite Web Console via web-socket protocol, default
-     value: http://localhost:3001
+     value: http://localhost:3000
   -t, --tokens
      User's security tokens
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/bin/ignite-web-agent.bat
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.bat b/modules/web-console/web-agent/bin/ignite-web-agent.bat
index f16eb35..8291b55 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.bat
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.bat
@@ -53,7 +53,12 @@ goto error_finish
 ::
 :: ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE
 ::
-if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m
+"%JAVA_HOME%\bin\java.exe" -version 2>&1 | findstr "1\.[7]\." > nul
+if %ERRORLEVEL% equ 0 (
+    if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m
+) else (
+    if "%JVM_OPTS%" == "" set JVM_OPTS=-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m
+)
 
 "%JAVA_HOME%\bin\java.exe" %JVM_OPTS% -cp "*" org.apache.ignite.console.agent.AgentLauncher  %*
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/bin/ignite-web-agent.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/bin/ignite-web-agent.sh b/modules/web-console/web-agent/bin/ignite-web-agent.sh
index 3f2c2bc..2e9f041 100644
--- a/modules/web-console/web-agent/bin/ignite-web-agent.sh
+++ b/modules/web-console/web-agent/bin/ignite-web-agent.sh
@@ -81,7 +81,11 @@ cd $DIR
 # ADD YOUR/CHANGE ADDITIONAL OPTIONS HERE
 #
 if [ -z "$JVM_OPTS" ] ; then
-    JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m"
+    if [[ `"$JAVA" -version 2>&1 | egrep "1\.[7]\."` ]]; then
+        JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxPermSize=256m"
+    else
+        JVM_OPTS="-Xms1g -Xmx1g -server -XX:+AggressiveOpts -XX:MaxMetaspaceSize=256m"
+    fi
 fi
 
 "$JAVA" ${JVM_OPTS} -cp "*" org.apache.ignite.console.agent.AgentLauncher "$@"

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java
index d4787cc..8f70100 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentConfiguration.java
@@ -31,9 +31,6 @@ import java.util.Properties;
  * Agent configuration.
  */
 public class AgentConfiguration {
-    /** Default server port. */
-    public static final int DFLT_SERVER_PORT = 3001;
-
     /** Default Ignite node HTTP port. */
     public static final int DFLT_NODE_PORT = 8080;
 
@@ -41,7 +38,7 @@ public class AgentConfiguration {
     public static final String DFLT_CFG_PATH = "default.properties";
 
     /** Default server URI. */
-    private static final String DFLT_SERVER_URI = "http://localhost:3001";
+    private static final String DFLT_SERVER_URI = "http://localhost:3000";
 
     /** Default Ignite node HTTP URI. */
     private static final String DFLT_NODE_URI = "http://localhost:8080";

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
index 810fad4..0c03d77 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
@@ -50,7 +50,6 @@ import static io.socket.client.Socket.EVENT_CONNECT_ERROR;
 import static io.socket.client.Socket.EVENT_DISCONNECT;
 import static io.socket.client.Socket.EVENT_ERROR;
 import static io.socket.client.Socket.EVENT_RECONNECTING;
-import static org.apache.ignite.console.agent.AgentConfiguration.DFLT_SERVER_PORT;
 
 /**
  * Control Center Agent launcher.
@@ -227,11 +226,10 @@ public class AgentLauncher {
 
             URI uri = URI.create(cfg.serverUri());
 
-            if (uri.getPort() == -1)
-                uri = URI.create(cfg.serverUri() + ':' + DFLT_SERVER_PORT);
-
             IO.Options opts = new IO.Options();
 
+            opts.path = "/agents";
+
             opts.reconnectionDelay = RECONNECT_INTERVAL;
 
             // Workaround for use self-signed certificate

http://git-wip-us.apache.org/repos/asf/ignite/blob/bf330251/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
index 2fb9f56..489e762 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/demo/AgentClusterDemo.java
@@ -134,6 +134,7 @@ public class AgentClusterDemo {
         CacheConfiguration<K, V> ccfg = new CacheConfiguration<>(name);
 
         ccfg.setAffinity(new RendezvousAffinityFunction(false, 32));
+        ccfg.setQueryDetailMetricsSize(10);
         ccfg.setStartSize(100);
         ccfg.setStatisticsEnabled(true);
 


[06/19] ignite git commit: Web console beta-6. Minor fix.

Posted by sb...@apache.org.
Web console beta-6. Minor fix.


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

Branch: refs/heads/master
Commit: 7d88c5bfe7d6f130974fab1ed4266fff859afd3d
Parents: bf33025
Author: Andrey Novikov <an...@gridgain.com>
Authored: Thu Nov 24 17:59:33 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Thu Nov 24 17:59:33 2016 +0700

----------------------------------------------------------------------
 modules/web-console/docker/compose/backend/build.sh  | 2 +-
 modules/web-console/docker/compose/frontend/build.sh | 4 ++--
 modules/web-console/docker/standalone/build.sh       | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d88c5bf/modules/web-console/docker/compose/backend/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/backend/build.sh b/modules/web-console/docker/compose/backend/build.sh
index d44efbd..a10b706 100755
--- a/modules/web-console/docker/compose/backend/build.sh
+++ b/modules/web-console/docker/compose/backend/build.sh
@@ -27,7 +27,7 @@ WORK_DIR=`cd "$(dirname "$0")"; pwd`
 BUILD_DIR="$WORK_DIR/build"
 
 IGNITE_WEB_CONSOLE_BACKEND_DIR="$IGNITE_HOME/modules/web-console/backend"
-DOCKER_IMAGE_NAME="ignite/web-console-backend"
+DOCKER_IMAGE_NAME="apacheignite/web-console-backend"
 
 echo "Receiving version..."
 VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'`

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d88c5bf/modules/web-console/docker/compose/frontend/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/compose/frontend/build.sh b/modules/web-console/docker/compose/frontend/build.sh
index 6d52fb2..c807a86 100755
--- a/modules/web-console/docker/compose/frontend/build.sh
+++ b/modules/web-console/docker/compose/frontend/build.sh
@@ -28,8 +28,8 @@ SOURCE_DIR=$WORK_DIR/src
 BUILD_DIR=$WORK_DIR/build
 
 DOCKER_BUILD_CONTAINER=web-console-frontend-builder
-DOCKER_BUILD_IMAGE_NAME=ignite/$DOCKER_BUILD_CONTAINER
-DOCKER_IMAGE_NAME=ignite/web-console-frontend
+DOCKER_BUILD_IMAGE_NAME=apacheignite/$DOCKER_BUILD_CONTAINER
+DOCKER_IMAGE_NAME=apacheignite/web-console-frontend
 
 echo "Receiving version..."
 VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'`

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d88c5bf/modules/web-console/docker/standalone/build.sh
----------------------------------------------------------------------
diff --git a/modules/web-console/docker/standalone/build.sh b/modules/web-console/docker/standalone/build.sh
index 5482086..c32dc40 100755
--- a/modules/web-console/docker/standalone/build.sh
+++ b/modules/web-console/docker/standalone/build.sh
@@ -27,7 +27,7 @@ WORK_DIR=`cd "$(dirname "$0")"; pwd`
 BUILD_DIR="$WORK_DIR/build"
 
 IGNITE_WEB_CONSOLE_DIR="$IGNITE_HOME/modules/web-console"
-DOCKER_IMAGE_NAME="ignite/web-console-standalone"
+DOCKER_IMAGE_NAME="apacheignite/web-console-standalone"
 
 echo "Receiving version..."
 VERSION=`cd $IGNITE_HOME && mvn org.apache.maven.plugins:maven-help-plugin:evaluate -Dexpression=project.version| grep -Ev '(^\[|Download\w+:)'`


[14/19] ignite git commit: Improved exception handling.

Posted by sb...@apache.org.
Improved exception handling.


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

Branch: refs/heads/master
Commit: e9ace7730773a6d4a1d30b271854f1fe8a7ba632
Parents: 0666910
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 2 16:06:41 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 2 16:06:41 2016 +0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java | 4 ++--
 .../ignite/marshaller/optimized/OptimizedMarshaller.java     | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e9ace773/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
index 54172dc..06b7109 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/jdk/JdkMarshaller.java
@@ -121,8 +121,8 @@ public class JdkMarshaller extends AbstractNodeNameAwareMarshaller {
         }
         catch (ClassNotFoundException e) {
             throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " +
-                clsLdr, e);
+                "(make sure same versions of all classes are available on all nodes or enable peer-class-loading) " +
+                "[clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e);
         }
         catch (Exception e) {
             throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/e9ace773/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
index 467dddf..6d57864 100644
--- a/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/apache/ignite/marshaller/optimized/OptimizedMarshaller.java
@@ -219,8 +219,8 @@ public class OptimizedMarshaller extends AbstractNodeNameAwareMarshaller {
         }
         catch (ClassNotFoundException e) {
             throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                "(make sure same versions of all classes are available on all nodes or enable peer-class-loading): " +
-                clsLdr, e);
+                "(make sure same versions of all classes are available on all nodes or enable peer-class-loading) " +
+                "[clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e);
         }
         catch (Exception e) {
             throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);
@@ -248,8 +248,8 @@ public class OptimizedMarshaller extends AbstractNodeNameAwareMarshaller {
         }
         catch (ClassNotFoundException e) {
             throw new IgniteCheckedException("Failed to find class with given class loader for unmarshalling " +
-                "(make sure same version of all classes are available on all nodes or enable peer-class-loading): " +
-                clsLdr, e);
+                "(make sure same version of all classes are available on all nodes or enable peer-class-loading)" +
+                " [clsLdr=" + clsLdr + ", cls=" + e.getMessage() + "]", e);
         }
         catch (Exception e) {
             throw new IgniteCheckedException("Failed to deserialize object with given class loader: " + clsLdr, e);


[18/19] ignite git commit: ignite-4285 For serializable txs allow multiple threads to get read lock for the same key

Posted by sb...@apache.org.
ignite-4285 For serializable txs allow multiple threads to get read lock for the same key


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

Branch: refs/heads/master
Commit: 33dda46061aae73e5c138851cfdd5f49a1c254cb
Parents: 12bdd6a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 2 12:13:38 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 2 12:15:14 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/CacheLockCandidates.java   |  42 ++
 .../cache/CacheLockCandidatesList.java          |  71 +++
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../processors/cache/GridCacheMapEntry.java     | 117 +++-
 .../processors/cache/GridCacheMvcc.java         | 376 ++++++++----
 .../processors/cache/GridCacheMvccCallback.java |   4 +-
 .../cache/GridCacheMvccCandidate.java           |  80 ++-
 .../processors/cache/GridCacheMvccManager.java  |  19 +-
 .../distributed/GridDistributedCacheEntry.java  | 303 +++-------
 .../distributed/dht/GridDhtCacheEntry.java      |  32 +-
 .../distributed/dht/GridDhtLockFuture.java      |  34 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../distributed/near/GridNearCacheEntry.java    |  44 +-
 .../distributed/near/GridNearLockFuture.java    |   3 +-
 .../near/GridNearTransactionalCache.java        |   5 +-
 .../cache/local/GridLocalCacheEntry.java        | 173 ++----
 .../cache/local/GridLocalLockFuture.java        |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   6 +-
 .../CacheSerializableTransactionsTest.java      | 604 ++++++++++++++++++-
 .../cache/GridCacheMvccFlagsTest.java           |   8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java | 334 ++++++++--
 .../processors/cache/GridCacheMvccSelfTest.java | 212 +++----
 .../processors/cache/GridCacheTestEntryEx.java  |  77 +--
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   7 +-
 26 files changed, 1721 insertions(+), 849 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java
new file mode 100644
index 0000000..9cf16f4
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidates.java
@@ -0,0 +1,42 @@
+/*
+ * 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.internal.processors.cache.version.GridCacheVersion;
+
+/**
+ *
+ */
+public interface CacheLockCandidates {
+    /**
+     * @param idx Candidate index.
+     * @return Candidate.
+     */
+    public GridCacheMvccCandidate candidate(int idx);
+
+    /**
+     * @return Number of candidates.
+     */
+    public int size();
+
+    /**
+     * @param ver Candidate version.
+     * @return {@code True} if contains candidate with given version.
+     */
+    public boolean hasCandidate(GridCacheVersion ver);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java
new file mode 100644
index 0000000..e026bce
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesList.java
@@ -0,0 +1,71 @@
+/*
+ * 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 java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ *
+ */
+class CacheLockCandidatesList implements CacheLockCandidates {
+    /** */
+    @GridToStringInclude
+    private List<GridCacheMvccCandidate> list = new ArrayList<>();
+
+    /**
+     * @param cand Candidate to add.
+     */
+    void add(GridCacheMvccCandidate cand) {
+        assert !hasCandidate(cand.version()) : cand;
+
+        list.add(cand);
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheMvccCandidate candidate(int idx) {
+        assert idx < list.size() : idx;
+
+        return list.get(idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        return list.size();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasCandidate(GridCacheVersion ver) {
+        for (int i = 0; i < list.size(); i++) {
+            GridCacheMvccCandidate cand = list.get(i);
+
+            if (cand.version().equals(ver))
+                return true;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheLockCandidatesList.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 176fe77..d8194fc 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
@@ -565,6 +565,7 @@ public interface GridCacheEntryEx {
      * @param timeout Timeout for lock acquisition.
      * @param serOrder Version for serializable transactions ordering.
      * @param serReadVer Optional read entry version for optimistic serializable transaction.
+     * @param read Read lock flag.
      * @return {@code True} if lock was acquired, {@code false} otherwise.
      * @throws GridCacheEntryRemovedException If this entry is obsolete.
      * @throws GridDistributedLockCancelledException If lock has been cancelled.
@@ -573,7 +574,7 @@ public interface GridCacheEntryEx {
         long timeout,
         @Nullable GridCacheVersion serOrder,
         @Nullable GridCacheVersion serReadVer,
-        boolean keepBinary
+        boolean read
     ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 aec28bb..31baeda 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
@@ -79,9 +79,11 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_EXPIRED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_LOCKED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_READ;
 import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_REMOVED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_UNLOCKED;
 import static org.apache.ignite.internal.processors.cache.GridCacheOperation.DELETE;
 import static org.apache.ignite.internal.processors.dr.GridDrType.DR_NONE;
 
@@ -827,8 +829,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         if (readThrough && !cctx.readThrough())
             readThrough = false;
 
-        GridCacheMvccCandidate owner;
-
         CacheObject ret;
 
         GridCacheVersion startVer;
@@ -841,10 +841,6 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
         synchronized (this) {
             checkObsolete();
 
-            GridCacheMvcc mvcc = mvccExtras();
-
-            owner = mvcc == null ? null : mvcc.anyOwner();
-
             boolean valid = valid(tx != null ? tx.topologyVersion() : cctx.affinity().affinityTopologyVersion());
 
             CacheObject val;
@@ -899,11 +895,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
             if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
                 transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo);
 
+                GridCacheMvcc mvcc = mvccExtras();
+
                 cctx.events().addEvent(
                     partition(),
                     key,
                     tx,
-                    owner,
+                    mvcc != null ? mvcc.anyOwner() : null,
                     EVT_CACHE_OBJECT_READ,
                     ret,
                     ret != null,
@@ -1010,11 +1008,13 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (evt && cctx.events().isRecordable(EVT_CACHE_OBJECT_READ)) {
                     transformClo = EntryProcessorResourceInjectorProxy.unwrap(transformClo);
 
+                    GridCacheMvcc mvcc = mvccExtras();
+
                     cctx.events().addEvent(
                         partition(),
                         key,
                         tx,
-                        owner,
+                        mvcc != null ? mvcc.anyOwner() : null,
                         EVT_CACHE_OBJECT_READ,
                         ret,
                         ret != null,
@@ -3391,14 +3391,14 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized boolean hasValue() {
+    @Override public final synchronized boolean hasValue() {
         return hasValueUnlocked();
     }
 
     /**
      * @return {@code True} if this entry has value.
      */
-    protected boolean hasValueUnlocked() {
+    protected final boolean hasValueUnlocked() {
         assert Thread.holdsLock(this);
 
         return val != null || hasOffHeapPointer();
@@ -4318,7 +4318,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer)
+    @Override public final GridCacheBatchSwapEntry evictInBatchInternal(GridCacheVersion obsoleteVer)
         throws IgniteCheckedException {
         assert Thread.holdsLock(this);
         assert cctx.isSwapOrOffheapEnabled();
@@ -4385,7 +4385,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @param filter Entry filter.
      * @return {@code True} if entry is visitable.
      */
-    public boolean visitable(CacheEntryPredicate[] filter) {
+    public final boolean visitable(CacheEntryPredicate[] filter) {
         boolean rmv = false;
 
         try {
@@ -4440,7 +4440,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public boolean deleted() {
+    @Override public final boolean deleted() {
         if (!cctx.deferredDelete())
             return false;
 
@@ -4450,7 +4450,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized boolean obsoleteOrDeleted() {
+    @Override public final synchronized boolean obsoleteOrDeleted() {
         return obsoleteVersionExtras() != null ||
             (cctx.deferredDelete() && (deletedUnlocked() || !hasValueUnlocked()));
     }
@@ -4459,7 +4459,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @return {@code True} if deleted.
      */
     @SuppressWarnings("SimplifiableIfStatement")
-    protected boolean deletedUnlocked() {
+    protected final boolean deletedUnlocked() {
         assert Thread.holdsLock(this);
 
         if (!cctx.deferredDelete())
@@ -4471,7 +4471,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * @param deleted {@code True} if deleted.
      */
-    protected void deletedUnlocked(boolean deleted) {
+    protected final void deletedUnlocked(boolean deleted) {
         assert Thread.holdsLock(this);
         assert cctx.deferredDelete();
 
@@ -4508,7 +4508,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * @return MVCC.
      */
-    @Nullable protected GridCacheMvcc mvccExtras() {
+    @Nullable protected final GridCacheMvcc mvccExtras() {
         return extras != null ? extras.mvcc() : null;
     }
 
@@ -4516,7 +4516,7 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
      * @return All MVCC local and non near candidates.
      */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    @Nullable public synchronized List<GridCacheMvccCandidate> mvccAllLocal() {
+    @Nullable public final synchronized List<GridCacheMvccCandidate> mvccAllLocal() {
         GridCacheMvcc mvcc = extras != null ? extras.mvcc() : null;
 
         if (mvcc == null)
@@ -4542,21 +4542,22 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /**
      * @param mvcc MVCC.
      */
-    protected void mvccExtras(@Nullable GridCacheMvcc mvcc) {
+    protected final void mvccExtras(@Nullable GridCacheMvcc mvcc) {
         extras = (extras != null) ? extras.mvcc(mvcc) : mvcc != null ? new GridCacheMvccEntryExtras(mvcc) : null;
     }
 
     /**
      * @return Obsolete version.
      */
-    @Nullable protected GridCacheVersion obsoleteVersionExtras() {
+    @Nullable protected final GridCacheVersion obsoleteVersionExtras() {
         return extras != null ? extras.obsoleteVersion() : null;
     }
 
     /**
      * @param obsoleteVer Obsolete version.
+     * @param ext Extras.
      */
-    protected void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, GridCacheObsoleteEntryExtras ext) {
+    private void obsoleteVersionExtras(@Nullable GridCacheVersion obsoleteVer, GridCacheObsoleteEntryExtras ext) {
         extras = (extras != null) ?
             extras.obsoleteVersion(obsoleteVer) :
             obsoleteVer != null ?
@@ -4565,6 +4566,80 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     }
 
     /**
+     * @param prevOwners Previous owners.
+     * @param owners Current owners.
+     * @param val Entry value.
+     */
+    protected final void checkOwnerChanged(@Nullable CacheLockCandidates prevOwners,
+        @Nullable CacheLockCandidates owners,
+        CacheObject val) {
+        assert !Thread.holdsLock(this);
+
+        if (prevOwners != null && owners == null) {
+            cctx.mvcc().callback().onOwnerChanged(this, null);
+
+            if (cctx.events().isRecordable(EVT_CACHE_OBJECT_UNLOCKED)) {
+                boolean hasVal = hasValue();
+
+                GridCacheMvccCandidate cand = prevOwners.candidate(0);
+
+                cctx.events().addEvent(partition(),
+                    key,
+                    cand.nodeId(),
+                    cand,
+                    EVT_CACHE_OBJECT_UNLOCKED,
+                    val,
+                    hasVal,
+                    val,
+                    hasVal,
+                    null,
+                    null,
+                    null,
+                    true);
+            }
+        }
+
+        if (owners != null) {
+            for (int i = 0; i < owners.size(); i++) {
+                GridCacheMvccCandidate owner = owners.candidate(i);
+
+                boolean locked = prevOwners == null || !prevOwners.hasCandidate(owner.version());
+
+                if (locked) {
+                    cctx.mvcc().callback().onOwnerChanged(this, owner);
+
+                    if (owner.local())
+                        checkThreadChain(owner);
+
+                    if (cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED)) {
+                        boolean hasVal = hasValue();
+
+                        // Event notification.
+                        cctx.events().addEvent(partition(),
+                            key,
+                            owner.nodeId(),
+                            owner,
+                            EVT_CACHE_OBJECT_LOCKED,
+                            val,
+                            hasVal,
+                            val,
+                            hasVal,
+                            null,
+                            null,
+                            null,
+                            true);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * @param owner Starting candidate in the chain.
+     */
+    protected abstract void checkThreadChain(GridCacheMvccCandidate owner);
+
+    /**
      * Updates metrics.
      *
      * @param op Operation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
index 507a2c9..498584c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvcc.java
@@ -107,7 +107,7 @@ public final class GridCacheMvcc {
     /**
      * @return Any owner.
      */
-    @Nullable public GridCacheMvccCandidate anyOwner() {
+    @Nullable GridCacheMvccCandidate anyOwner() {
         GridCacheMvccCandidate owner = localOwner();
 
         if (owner == null)
@@ -117,10 +117,22 @@ public final class GridCacheMvcc {
     }
 
     /**
+     * @return All owners.
+     */
+    @Nullable public CacheLockCandidates allOwners() {
+        CacheLockCandidates owners = localOwners();
+
+        if (owners == null)
+            owners = remoteOwner();
+
+        return owners;
+    }
+
+    /**
      * @return Remote candidate only if it's first in the list and is marked
      *      as <tt>'used'</tt>.
      */
-    @Nullable public GridCacheMvccCandidate remoteOwner() {
+    @Nullable private GridCacheMvccCandidate remoteOwner() {
         if (rmts != null) {
             assert !rmts.isEmpty();
 
@@ -133,10 +145,58 @@ public final class GridCacheMvcc {
     }
 
     /**
+     * @return All local owners.
+     */
+    @Nullable public CacheLockCandidates localOwners() {
+        if (locs != null) {
+            assert !locs.isEmpty();
+
+            CacheLockCandidates owners = null;
+
+            GridCacheMvccCandidate first = locs.getFirst();
+
+            if (first.read()) {
+                for (GridCacheMvccCandidate cand : locs) {
+                    if (cand.owner()) {
+                        assert cand.read() : this;
+
+                        if (owners != null) {
+                            CacheLockCandidatesList list;
+
+                            if (owners.size() == 1) {
+                                GridCacheMvccCandidate owner = owners.candidate(0);
+
+                                owners = list = new CacheLockCandidatesList();
+
+                                ((CacheLockCandidatesList)owners).add(owner);
+                            }
+                            else
+                                list = ((CacheLockCandidatesList)owners);
+
+                            list.add(cand);
+                        }
+                        else
+                            owners = cand;
+                    }
+
+                    if (!cand.read())
+                        break;
+                }
+            }
+            else if (first.owner())
+                owners = first;
+
+            return owners;
+        }
+
+        return null;
+    }
+
+    /**
      * @return Local candidate only if it's first in the list and is marked
      *      as <tt>'owner'</tt>.
      */
-    @Nullable public GridCacheMvccCandidate localOwner() {
+    @Nullable GridCacheMvccCandidate localOwner() {
         if (locs != null) {
             assert !locs.isEmpty();
 
@@ -186,6 +246,29 @@ public final class GridCacheMvcc {
     }
 
     /**
+     * @param cand Existing candidate.
+     * @param newCand New candidate.
+     * @return {@code False} if new candidate can not be added.
+     */
+    private boolean compareSerializableVersion(GridCacheMvccCandidate cand, GridCacheMvccCandidate newCand) {
+        assert cand.serializable() && newCand.serializable();
+
+        GridCacheVersion candOrder = cand.serializableOrder();
+
+        assert candOrder != null : cand;
+
+        GridCacheVersion newCandOrder = newCand.serializableOrder();
+
+        assert newCandOrder != null : newCand;
+
+        int cmp = SER_VER_COMPARATOR.compare(candOrder, newCandOrder);
+
+        assert cmp != 0;
+
+        return cmp < 0;
+    }
+
+    /**
      * @param cand Candidate to add.
      * @return {@code False} if failed to add candidate and transaction should be cancelled.
      */
@@ -200,25 +283,34 @@ public final class GridCacheMvcc {
             if (!cand.nearLocal()) {
                 if (!locs.isEmpty()) {
                     if (cand.serializable()) {
-                        GridCacheMvccCandidate last = locs.getLast();
-
-                        if (!last.serializable())
-                            return false;
-
-                        GridCacheVersion lastOrder = last.serializableOrder();
+                        Iterator<GridCacheMvccCandidate> it = locs.descendingIterator();
 
-                        assert lastOrder != null : last;
+                        if (cand.read()) {
+                            while (it.hasNext()) {
+                                GridCacheMvccCandidate c = it.next();
 
-                        GridCacheVersion candOrder = cand.serializableOrder();
+                                if (!c.serializable())
+                                    return false;
 
-                        assert candOrder != null : cand;
-
-                        int cmp = SER_VER_COMPARATOR.compare(lastOrder, candOrder);
+                                if (!c.read()) {
+                                    if (compareSerializableVersion(c, cand))
+                                        break;
+                                    else
+                                        return false;
+                                }
+                            }
+                        }
+                        else {
+                            while (it.hasNext()) {
+                                GridCacheMvccCandidate c = it.next();
 
-                        assert cmp != 0;
+                                if (!c.serializable() || !compareSerializableVersion(c, cand))
+                                    return false;
 
-                        if (cmp > 0)
-                            return false;
+                                if (!c.read())
+                                    break;
+                            }
+                        }
 
                         locs.addLast(cand);
 
@@ -284,12 +376,12 @@ public final class GridCacheMvcc {
         }
         // Remote.
         else {
-            assert !cand.serializable() : cand;
+            assert !cand.serializable() && !cand.read() : cand;
 
             if (rmts == null)
                 rmts = new LinkedList<>();
 
-            assert !cand.owner() || localOwner() == null : "Cannot have local and remote owners " +
+            assert !cand.owner() || localOwners() == null : "Cannot have local and remote owners " +
                 "at the same time [cand=" + cand + ", locs=" + locs + ", rmts=" + rmts + ']';
 
             GridCacheMvccCandidate cur = candidate(rmts, cand.version());
@@ -398,9 +490,8 @@ public final class GridCacheMvcc {
      * @param baseVer Base version.
      * @param committedVers Committed versions relative to base.
      * @param rolledbackVers Rolled back versions relative to base.
-     * @return Lock owner.
      */
-    @Nullable public GridCacheMvccCandidate orderCompleted(GridCacheVersion baseVer,
+    public void orderCompleted(GridCacheVersion baseVer,
         Collection<GridCacheVersion> committedVers, Collection<GridCacheVersion> rolledbackVers) {
         assert baseVer != null;
 
@@ -415,10 +506,13 @@ public final class GridCacheMvcc {
                 if (!cur.version().equals(baseVer) && committedVers.contains(cur.version())) {
                     cur.setOwner();
 
-                    assert localOwner() == null || localOwner().nearLocal(): "Cannot not have local owner and " +
+                    assert localOwners() == null || localOwner().nearLocal(): "Cannot not have local owner and " +
                         "remote completed transactions at the same time [baseVer=" + baseVer +
-                        ", committedVers=" + committedVers + ", rolledbackVers=" + rolledbackVers +
-                        ", localOwner=" + localOwner() + ", locs=" + locs + ", rmts=" + rmts + ']';
+                        ", committedVers=" + committedVers +
+                        ", rolledbackVers=" + rolledbackVers +
+                        ", localOwner=" + localOwner() +
+                        ", locs=" + locs +
+                        ", rmts=" + rmts + ']';
 
                     if (maxIdx < 0)
                         maxIdx = it.nextIndex();
@@ -462,8 +556,6 @@ public final class GridCacheMvcc {
                     rmts = null;
             }
         }
-
-        return anyOwner();
     }
 
     /**
@@ -471,11 +563,10 @@ public final class GridCacheMvcc {
      *
      * @param baseVer Base version.
      * @param owned Owned list.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate markOwned(GridCacheVersion baseVer, GridCacheVersion owned) {
+    public void markOwned(GridCacheVersion baseVer, GridCacheVersion owned) {
         if (owned == null)
-            return anyOwner();
+            return;
 
         if (rmts != null) {
             GridCacheMvccCandidate baseCand = candidate(rmts, baseVer);
@@ -483,8 +574,6 @@ public final class GridCacheMvcc {
             if (baseCand != null)
                 baseCand.ownerVersion(owned);
         }
-
-        return anyOwner();
     }
 
     /**
@@ -495,6 +584,7 @@ public final class GridCacheMvcc {
      * @param reenter Reentry flag ({@code true} if reentry is allowed).
      * @param tx Transaction flag.
      * @param implicitSingle Implicit transaction flag.
+     * @param read Read lock flag.
      * @return New lock candidate if lock was added, or current owner if lock was reentered,
      *      or <tt>null</tt> if lock was owned by another thread and timeout is negative.
      */
@@ -505,7 +595,8 @@ public final class GridCacheMvcc {
         long timeout,
         boolean reenter,
         boolean tx,
-        boolean implicitSingle) {
+        boolean implicitSingle,
+        boolean read) {
         return addLocal(
             parent,
             /*nearNodeId*/null,
@@ -517,7 +608,8 @@ public final class GridCacheMvcc {
             reenter,
             tx,
             implicitSingle,
-            /*dht-local*/false
+            /*dht-local*/false,
+            /*read*/read
         );
     }
 
@@ -533,6 +625,7 @@ public final class GridCacheMvcc {
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
      * @param dhtLoc DHT local flag.
+     * @param read Read lock flag.
      * @return New lock candidate if lock was added, or current owner if lock was reentered,
      *      or <tt>null</tt> if lock was owned by another thread and timeout is negative.
      */
@@ -547,7 +640,8 @@ public final class GridCacheMvcc {
         boolean reenter,
         boolean tx,
         boolean implicitSingle,
-        boolean dhtLoc) {
+        boolean dhtLoc,
+        boolean read) {
         if (log.isDebugEnabled())
             log.debug("Adding local candidate [mvcc=" + this + ", parent=" + parent + ", threadId=" + threadId +
                 ", ver=" + ver + ", timeout=" + timeout + ", reenter=" + reenter + ", tx=" + tx + "]");
@@ -582,14 +676,14 @@ public final class GridCacheMvcc {
             nearVer,
             threadId,
             ver,
-            timeout,
             /*local*/true,
             /*reenter*/false,
             tx,
             implicitSingle,
             /*near-local*/false,
             dhtLoc,
-            serOrder
+            serOrder,
+            read
         );
 
         if (serOrder == null) {
@@ -613,7 +707,6 @@ public final class GridCacheMvcc {
      * @param otherNodeId Other node ID.
      * @param threadId Thread ID.
      * @param ver Lock version.
-     * @param timeout Lock acquire timeout.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
      * @param nearLoc Near local flag.
@@ -625,7 +718,6 @@ public final class GridCacheMvcc {
         @Nullable UUID otherNodeId,
         long threadId,
         GridCacheVersion ver,
-        long timeout,
         boolean tx,
         boolean implicitSingle,
         boolean nearLoc) {
@@ -636,14 +728,14 @@ public final class GridCacheMvcc {
             null,
             threadId,
             ver,
-            timeout,
             /*local*/false,
             /*reentry*/false,
             tx,
             implicitSingle,
             nearLoc,
             false,
-            null
+            null,
+            /*read*/false
         );
 
         addRemote(cand);
@@ -659,9 +751,9 @@ public final class GridCacheMvcc {
      * @param otherNodeId Other node ID.
      * @param threadId Thread ID.
      * @param ver Lock version.
-     * @param timeout Lock acquire timeout.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
+     * @param read Read lock flag.
      * @return Add remote candidate.
      */
     public GridCacheMvccCandidate addNearLocal(GridCacheEntryEx parent,
@@ -669,23 +761,23 @@ public final class GridCacheMvcc {
         @Nullable UUID otherNodeId,
         long threadId,
         GridCacheVersion ver,
-        long timeout,
         boolean tx,
-        boolean implicitSingle) {
+        boolean implicitSingle,
+        boolean read) {
         GridCacheMvccCandidate cand = new GridCacheMvccCandidate(parent,
             nodeId,
             otherNodeId,
             null,
             threadId,
             ver,
-            timeout,
             /*local*/true,
             /*reentry*/false,
             tx,
             implicitSingle,
             /*near loc*/true,
             /*dht loc*/false,
-            null);
+            null,
+            /*read*/read);
 
         add0(cand);
 
@@ -695,7 +787,7 @@ public final class GridCacheMvcc {
     /**
      * @param cand Remote candidate.
      */
-    public void addRemote(GridCacheMvccCandidate cand) {
+    private void addRemote(GridCacheMvccCandidate cand) {
         assert !cand.local();
 
         if (log.isDebugEnabled())
@@ -710,11 +802,11 @@ public final class GridCacheMvcc {
      * @param ver Lock version to acquire or set to ready.
      * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate readyLocal(GridCacheVersion ver) {
+    @Nullable public CacheLockCandidates readyLocal(GridCacheVersion ver) {
         GridCacheMvccCandidate cand = candidate(ver);
 
         if (cand == null)
-            return anyOwner();
+            return allOwners();
 
         assert cand.local();
 
@@ -725,14 +817,14 @@ public final class GridCacheMvcc {
      * @param cand Local candidate added in any of the {@code addLocal(..)} methods.
      * @return Current lock owner.
      */
-    @Nullable public GridCacheMvccCandidate readyLocal(GridCacheMvccCandidate cand) {
+    @Nullable public CacheLockCandidates readyLocal(GridCacheMvccCandidate cand) {
         assert cand.local();
 
         cand.setReady();
 
         reassign();
 
-        return anyOwner();
+        return allOwners();
     }
 
     /**
@@ -751,9 +843,12 @@ public final class GridCacheMvcc {
      * @param pending Pending dht versions that are not owned and which version is less then mapped.
      * @return Lock owner after reassignment.
      */
-    @Nullable public GridCacheMvccCandidate readyNearLocal(GridCacheVersion ver, GridCacheVersion mappedVer,
-        Collection<GridCacheVersion> committedVers, Collection<GridCacheVersion> rolledBackVers,
-        Collection<GridCacheVersion> pending) {
+    @Nullable public CacheLockCandidates readyNearLocal(GridCacheVersion ver,
+        GridCacheVersion mappedVer,
+        Collection<GridCacheVersion> committedVers,
+        Collection<GridCacheVersion> rolledBackVers,
+        Collection<GridCacheVersion> pending)
+    {
         GridCacheMvccCandidate cand = candidate(locs, ver);
 
         if (cand != null) {
@@ -785,7 +880,7 @@ public final class GridCacheMvcc {
                     if (c.owner())
                         continue;
 
-                    assert !c.ready() :
+                    assert !c.ready() || (c.read() && cand.read()):
                         "Cannot have more then one ready near-local candidate [c=" + c + ", cand=" + cand +
                             ", mvcc=" + this + ']';
 
@@ -819,7 +914,7 @@ public final class GridCacheMvcc {
             reassign();
         }
 
-        return anyOwner();
+        return allOwners();
     }
 
     /**
@@ -831,7 +926,7 @@ public final class GridCacheMvcc {
      * @param rolledback Rolledback versions.
      * @return Lock owner.
      */
-    @Nullable public GridCacheMvccCandidate doneRemote(
+    @Nullable public CacheLockCandidates doneRemote(
         GridCacheVersion ver,
         Collection<GridCacheVersion> pending,
         Collection<GridCacheVersion> committed,
@@ -879,7 +974,7 @@ public final class GridCacheMvcc {
             }
         }
 
-        return anyOwner();
+        return allOwners();
     }
 
     /**
@@ -942,19 +1037,39 @@ public final class GridCacheMvcc {
         if (locs != null) {
             boolean first = true;
 
-            for (ListIterator<GridCacheMvccCandidate> it = locs.listIterator(); it.hasNext(); ) {
+            ListIterator<GridCacheMvccCandidate> it = locs.listIterator();
+
+            while (it.hasNext()) {
                 GridCacheMvccCandidate cand = it.next();
 
-                if (first && cand.serializable()) {
-                    if (cand.owner() || !cand.ready())
+                if (first) {
+                    if (cand.read()) {
+                        if (cand.ready() && !cand.owner())
+                            cand.setOwner();
+
+                        while (it.hasNext()) {
+                            cand = it.next();
+
+                            if (!cand.read())
+                                break;
+
+                            if (cand.ready() && !cand.owner())
+                                cand.setOwner();
+                        }
+
                         return;
+                    }
+                    else if (cand.serializable()) {
+                        if (cand.owner() || !cand.ready())
+                            return;
 
-                    cand.setOwner();
+                        cand.setOwner();
 
-                    return;
-                }
+                        return;
+                    }
 
-                first = false;
+                    first = false;
+                }
 
                 if (cand.owner())
                     return;
@@ -1036,6 +1151,8 @@ public final class GridCacheMvcc {
                     }
 
                     if (assigned) {
+                        assert !cand.serializable() : cand;
+
                         it.remove();
 
                         // Owner must be first in the list.
@@ -1066,15 +1183,16 @@ public final class GridCacheMvcc {
      *
      * @return Owner.
      */
-    @Nullable public GridCacheMvccCandidate recheck() {
+    @Nullable public CacheLockCandidates recheck() {
         reassign();
 
-        return anyOwner();
+        return allOwners();
     }
 
     /**
      * Local local release.
-     * @return Removed lock candidate or <tt>null</tt> if candidate was not removed.
+     *
+     * @return Removed candidate.
      */
     @Nullable public GridCacheMvccCandidate releaseLocal() {
         return releaseLocal(Thread.currentThread().getId());
@@ -1084,32 +1202,45 @@ public final class GridCacheMvcc {
      * Local release.
      *
      * @param threadId ID of the thread.
-     * @return Current owner.
+     * @return Removed candidate.
      */
     @Nullable public GridCacheMvccCandidate releaseLocal(long threadId) {
-        GridCacheMvccCandidate owner = localOwner();
+        CacheLockCandidates owners = localOwners();
 
-        if (owner == null || owner.threadId() != threadId)
-            // Release had no effect.
-            return owner;
+        // Release had no effect.
+        if (owners == null)
+            return null;
 
-        owner.setUsed();
+        GridCacheMvccCandidate owner = null;
 
-        remove0(owner.version(), true);
+        for (int i = 0; i < owners.size(); i++) {
+            GridCacheMvccCandidate owner0 = owners.candidate(i);
 
-        return anyOwner();
+            if (owner0.threadId() == threadId) {
+                owner = owner0;
+
+                break;
+            }
+        }
+
+        if (owner != null) {
+            owner.setUsed();
+
+            remove0(owner.version(), true);
+
+            return owner;
+        }
+        else
+            return null;
     }
 
     /**
      * Removes lock even if it is not owner.
      *
      * @param ver Lock version.
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate remove(GridCacheVersion ver) {
+    public void remove(GridCacheVersion ver) {
         remove0(ver, false);
-
-        return anyOwner();
     }
 
     /**
@@ -1118,7 +1249,7 @@ public final class GridCacheMvcc {
      * @param nodeId Node ID.
      * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate removeExplicitNodeCandidates(UUID nodeId) {
+    @Nullable public CacheLockCandidates removeExplicitNodeCandidates(UUID nodeId) {
         if (rmts != null) {
             for (Iterator<GridCacheMvccCandidate> it = rmts.iterator(); it.hasNext(); ) {
                 GridCacheMvccCandidate cand = it.next();
@@ -1153,7 +1284,7 @@ public final class GridCacheMvcc {
 
         reassign();
 
-        return anyOwner();
+        return allOwners();
     }
 
     /**
@@ -1177,7 +1308,7 @@ public final class GridCacheMvcc {
      * @param threadId Thread ID.
      * @return Candidate or <tt>null</tt> if there is no candidate for given ID.
      */
-    @Nullable public GridCacheMvccCandidate localCandidate(long threadId) {
+    @Nullable GridCacheMvccCandidate localCandidate(long threadId) {
         // Don't return reentries.
         return localCandidate(threadId, false);
     }
@@ -1187,7 +1318,7 @@ public final class GridCacheMvcc {
      * @param threadId Thread ID.
      * @return Remote candidate.
      */
-    @Nullable public GridCacheMvccCandidate remoteCandidate(UUID nodeId, long threadId) {
+    @Nullable GridCacheMvccCandidate remoteCandidate(UUID nodeId, long threadId) {
         if (rmts != null)
             for (GridCacheMvccCandidate c : rmts)
                 if (c.nodeId().equals(nodeId) && c.threadId() == threadId)
@@ -1217,7 +1348,7 @@ public final class GridCacheMvcc {
      * @param ver Version.
      * @return {@code True} if candidate with given version exists.
      */
-    public boolean hasCandidate(GridCacheVersion ver) {
+    boolean hasCandidate(GridCacheVersion ver) {
         return candidate(ver) != null;
     }
 
@@ -1284,40 +1415,24 @@ public final class GridCacheMvcc {
     }
 
     /**
-     * @return {@code True} if lock is owner by current thread.
-     */
-    public boolean isLocallyOwnedByCurrentThread() {
-        return isLocallyOwnedByThread(Thread.currentThread().getId(), true);
-    }
-
-    /**
      * @param threadId Thread ID to check.
      * @param exclude Versions to ignore.
      * @return {@code True} if lock is owned by the thread with given ID.
      */
-    public boolean isLocallyOwnedByThread(long threadId, boolean allowDhtLoc, GridCacheVersion... exclude) {
-        GridCacheMvccCandidate owner = localOwner();
-
-        return owner != null && owner.threadId() == threadId && owner.nodeId().equals(cctx.nodeId()) &&
-            (allowDhtLoc || !owner.dhtLocal()) && !U.containsObjectArray(exclude, owner.version());
-    }
+    boolean isLocallyOwnedByThread(long threadId, boolean allowDhtLoc, GridCacheVersion... exclude) {
+        CacheLockCandidates owners = localOwners();
 
-    /**
-     * @param threadId Thread ID.
-     * @param nodeId Node ID.
-     * @return {@code True} if lock is held by given thread and node IDs.
-     */
-    public boolean isLockedByThread(long threadId, UUID nodeId) {
-        GridCacheMvccCandidate owner = anyOwner();
+        if (owners != null) {
+            for (int i = 0; i < owners.size(); i++) {
+                GridCacheMvccCandidate owner = owners.candidate(i);
 
-        return owner != null && owner.threadId() == threadId && owner.nodeId().equals(nodeId);
-    }
+                if (owner.threadId() == threadId && owner.nodeId().equals(cctx.nodeId()) &&
+                    (allowDhtLoc || !owner.dhtLocal()) && !U.containsObjectArray(exclude, owner.version()))
+                    return true;
+            }
+        }
 
-    /**
-     * @return {@code True} if lock is owned by any thread or node.
-     */
-    public boolean isOwnedByAny() {
-        return anyOwner() != null;
+        return false;
     }
 
     /**
@@ -1325,10 +1440,10 @@ public final class GridCacheMvcc {
      * @param lockVer ID of lock candidate.
      * @return {@code True} if candidate is owner.
      */
-    public boolean isLocallyOwned(GridCacheVersion lockVer) {
-        GridCacheMvccCandidate owner = localOwner();
+    boolean isLocallyOwned(GridCacheVersion lockVer) {
+        CacheLockCandidates owners = localOwners();
 
-        return owner != null && owner.version().equals(lockVer);
+        return owners != null && owners.hasCandidate(lockVer);
     }
 
     /**
@@ -1336,30 +1451,25 @@ public final class GridCacheMvcc {
      * @param threadId Thread ID.
      * @return {@code True} if locked by lock ID or thread ID.
      */
-    public boolean isLocallyOwnedByIdOrThread(GridCacheVersion lockVer, long threadId) {
-        GridCacheMvccCandidate owner = localOwner();
+    boolean isLocallyOwnedByIdOrThread(GridCacheVersion lockVer, long threadId) {
+        CacheLockCandidates owners = localOwners();
 
-        return owner != null && (owner.version().equals(lockVer) || owner.threadId() == threadId);
-    }
+        if (owners != null) {
+            for (int i = 0; i < owners.size(); i++) {
+                GridCacheMvccCandidate owner = owners.candidate(i);
 
-    /**
-     * @return First remote entry or <tt>null</tt>.
-     */
-    @Nullable public GridCacheMvccCandidate firstRemote() {
-        return rmts == null ? null : rmts.getFirst();
-    }
+                if ((owner.version().equals(lockVer) || owner.threadId() == threadId))
+                    return true;
+            }
+        }
 
-    /**
-     * @return First local entry or <tt>null</tt>.
-     */
-    @Nullable public GridCacheMvccCandidate firstLocal() {
-        return locs == null ? null : locs.getFirst();
+        return false;
     }
 
     /**
      * @return Local MVCC candidates.
      */
-    @Nullable public List<GridCacheMvccCandidate> allLocal() {
+    @Nullable List<GridCacheMvccCandidate> allLocal() {
         return locs;
     }
 
@@ -1367,10 +1477,10 @@ public final class GridCacheMvcc {
      * @param ver Version to check for ownership.
      * @return {@code True} if lock is owned by the specified version.
      */
-    public boolean isOwnedBy(GridCacheVersion ver) {
-        GridCacheMvccCandidate cand = anyOwner();
+    boolean isOwnedBy(GridCacheVersion ver) {
+        CacheLockCandidates owners = allOwners();
 
-        return cand != null && cand.version().equals(ver);
+        return owners != null && owners.hasCandidate(ver);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java
index fc1faf7..2ba41f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCallback.java
@@ -37,11 +37,9 @@ public interface GridCacheMvccCallback {
      * can be made from this call.
      *
      * @param entry Entry.
-     * @param prev Previous candidate.
      * @param owner Current owner.
      */
-    public void onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate prev,
-        GridCacheMvccCandidate owner);
+    public void onOwnerChanged(GridCacheEntryEx entry, GridCacheMvccCandidate owner);
 
     /**
      * Called when entry has no more candidates. This call happens

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
index f1c1b83..e9dd455 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccCandidate.java
@@ -43,6 +43,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.LOCAL;
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.NEAR_LOCAL;
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.OWNER;
+import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.READ;
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.READY;
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.REENTRY;
 import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate.Mask.REMOVED;
@@ -54,7 +55,7 @@ import static org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate
  * Lock candidate.
  */
 public class GridCacheMvccCandidate implements Externalizable,
-    Comparable<GridCacheMvccCandidate> {
+    Comparable<GridCacheMvccCandidate>, CacheLockCandidates {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -69,14 +70,6 @@ public class GridCacheMvccCandidate implements Externalizable,
     @GridToStringInclude
     private GridCacheVersion ver;
 
-    /** Maximum wait time. */
-    @GridToStringInclude
-    private long timeout;
-
-    /** Candidate timestamp. */
-    @GridToStringInclude
-    private long ts;
-
     /** Thread ID. */
     @GridToStringInclude
     private long threadId;
@@ -143,7 +136,6 @@ public class GridCacheMvccCandidate implements Externalizable,
      * @param otherVer Other version.
      * @param threadId Requesting thread ID.
      * @param ver Cache version.
-     * @param timeout Maximum wait time.
      * @param loc {@code True} if the lock is local.
      * @param reentry {@code True} if candidate is for reentry.
      * @param tx Transaction flag.
@@ -151,6 +143,7 @@ public class GridCacheMvccCandidate implements Externalizable,
      * @param nearLoc Near-local flag.
      * @param dhtLoc DHT local flag.
      * @param serOrder Version for serializable transactions ordering.
+     * @param read Read lock flag.
      */
     public GridCacheMvccCandidate(
         GridCacheEntryEx parent,
@@ -159,14 +152,14 @@ public class GridCacheMvccCandidate implements Externalizable,
         @Nullable GridCacheVersion otherVer,
         long threadId,
         GridCacheVersion ver,
-        long timeout,
         boolean loc,
         boolean reentry,
         boolean tx,
         boolean singleImplicit,
         boolean nearLoc,
         boolean dhtLoc,
-        @Nullable GridCacheVersion serOrder
+        @Nullable GridCacheVersion serOrder,
+        boolean read
     ) {
         assert nodeId != null;
         assert ver != null;
@@ -178,7 +171,6 @@ public class GridCacheMvccCandidate implements Externalizable,
         this.otherVer = otherVer;
         this.threadId = threadId;
         this.ver = ver;
-        this.timeout = timeout;
         this.serOrder = serOrder;
 
         mask(LOCAL, loc);
@@ -187,8 +179,7 @@ public class GridCacheMvccCandidate implements Externalizable,
         mask(SINGLE_IMPLICIT, singleImplicit);
         mask(NEAR_LOCAL, nearLoc);
         mask(DHT_LOCAL, dhtLoc);
-
-        ts = U.currentTimeMillis();
+        mask(READ, read);
 
         id = IDGEN.incrementAndGet();
     }
@@ -245,14 +236,14 @@ public class GridCacheMvccCandidate implements Externalizable,
             otherVer,
             threadId,
             ver,
-            timeout,
             local(),
             /*reentry*/true,
             tx(),
             singleImplicit(),
             nearLocal(),
             dhtLocal(),
-            serializableOrder());
+            serializableOrder(),
+            read());
 
         reentry.topVer = topVer;
 
@@ -411,20 +402,6 @@ public class GridCacheMvccCandidate implements Externalizable,
     }
 
     /**
-     * @return Maximum wait time.
-     */
-    public long timeout() {
-        return timeout;
-    }
-
-    /**
-     * @return Timestamp at the time of entering pending set.
-     */
-    public long timestamp() {
-        return ts;
-    }
-
-    /**
      * @return {@code True} if lock is local.
      */
     public boolean local() {
@@ -474,6 +451,13 @@ public class GridCacheMvccCandidate implements Externalizable,
     }
 
     /**
+     * @return Read lock flag.
+     */
+    public boolean read() {
+        return READ.get(flags());
+    }
+
+    /**
      * @return {@code True} if this candidate is a reentry.
      */
     public boolean reentry() {
@@ -586,16 +570,21 @@ public class GridCacheMvccCandidate implements Externalizable,
         return parent0.txKey();
     }
 
-    /**
-     * Checks if this candidate matches version or thread-nodeId combination.
-     *
-     * @param nodeId Node ID to check.
-     * @param ver Version to check.
-     * @param threadId Thread ID to check.
-     * @return {@code True} if matched.
-     */
-    public boolean matches(GridCacheVersion ver, UUID nodeId, long threadId) {
-        return ver.equals(this.ver) || (nodeId.equals(this.nodeId) && threadId == this.threadId);
+    /** {@inheritDoc} */
+    @Override public GridCacheMvccCandidate candidate(int idx) {
+        assert idx == 0 : idx;
+
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int size() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean hasCandidate(GridCacheVersion ver) {
+        return this.ver.equals(ver);
     }
 
     /** {@inheritDoc} */
@@ -610,7 +599,6 @@ public class GridCacheMvccCandidate implements Externalizable,
             ver.writeExternal(out);
         }
 
-        out.writeLong(timeout);
         out.writeLong(threadId);
         out.writeLong(id);
         out.writeShort(flags());
@@ -626,7 +614,6 @@ public class GridCacheMvccCandidate implements Externalizable,
             ver.readExternal(in);
         }
 
-        timeout = in.readLong();
         threadId = in.readLong();
         id = in.readLong();
 
@@ -635,8 +622,6 @@ public class GridCacheMvccCandidate implements Externalizable,
         mask(OWNER, OWNER.get(flags));
         mask(USED, USED.get(flags));
         mask(TX, TX.get(flags));
-
-        ts = U.currentTimeMillis();
     }
 
     /** {@inheritDoc} */
@@ -719,7 +704,10 @@ public class GridCacheMvccCandidate implements Externalizable,
         NEAR_LOCAL(0x200),
 
         /** */
-        REMOVED(0x400);
+        REMOVED(0x400),
+
+        /** */
+        READ(0x800);
 
         /** All mask values. */
         private static final Mask[] MASKS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 c57e17c..0d0e9ee 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
@@ -140,15 +140,14 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     private final GridCacheMvccCallback cb = new GridCacheMvccCallback() {
         /** {@inheritDoc} */
         @SuppressWarnings({"unchecked"})
-        @Override public void onOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate prev,
-            final GridCacheMvccCandidate owner) {
+        @Override public void onOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate owner) {
             int nested = nestedLsnrCalls.get();
 
             if (nested < MAX_NESTED_LSNR_CALLS) {
                 nestedLsnrCalls.set(nested + 1);
 
                 try {
-                    notifyOwnerChanged(entry, prev, owner);
+                    notifyOwnerChanged(entry, owner);
                 }
                 finally {
                     nestedLsnrCalls.set(nested);
@@ -157,7 +156,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
             else {
                 cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
                     @Override public void run() {
-                        notifyOwnerChanged(entry, prev, owner);
+                        notifyOwnerChanged(entry, owner);
                     }
                 }, true);
             }
@@ -182,19 +181,13 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
 
     /**
      * @param entry Entry to notify callback for.
-     * @param prev Previous lock owner.
      * @param owner Current lock owner.
      */
-    private void notifyOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate prev,
-        final GridCacheMvccCandidate owner) {
+    private void notifyOwnerChanged(final GridCacheEntryEx entry, final GridCacheMvccCandidate owner) {
         assert entry != null;
-        assert owner != prev : "New and previous owner are identical instances: " + owner;
-        assert owner == null || prev == null || !owner.version().equals(prev.version()) :
-            "New and previous owners have identical versions [owner=" + owner + ", prev=" + prev + ']';
 
         if (log.isDebugEnabled())
-            log.debug("Received owner changed callback [" + entry.key() + ", owner=" + owner + ", prev=" +
-                prev + ']');
+            log.debug("Received owner changed callback [" + entry.key() + ", owner=" + owner + ']');
 
         if (owner != null && (owner.local() || owner.nearLocal())) {
             Collection<GridCacheMvccFuture<?>> futCol = mvccFuts.get(owner.version());
@@ -226,7 +219,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
 
         if (log.isDebugEnabled())
             log.debug("Lock future not found for owner change callback (will try transaction futures) [owner=" +
-                owner + ", prev=" + prev + ", entry=" + entry + ']');
+                owner + ", entry=" + entry + ']');
 
         // If no future was found, delegate to transaction manager.
         if (cctx.tm().onOwnerChanged(entry, owner)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 2d1b02e..3d55f31 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
@@ -23,6 +23,7 @@ import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -66,7 +67,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     /**
      *
      */
-    protected void refreshRemotes() {
+    private void refreshRemotes() {
         GridCacheMvcc mvcc = mvccExtras();
 
         rmts = mvcc == null ? Collections.<GridCacheMvccCandidate>emptyList() : mvcc.remoteCandidates();
@@ -82,6 +83,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
+     * @param read Read lock flag.
      * @return New candidate.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
@@ -92,10 +94,11 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
         long timeout,
         boolean reenter,
         boolean tx,
-        boolean implicitSingle) throws GridCacheEntryRemovedException {
+        boolean implicitSingle,
+        boolean read) throws GridCacheEntryRemovedException {
         GridCacheMvccCandidate cand;
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate owner;
+        CacheLockCandidates prev;
+        CacheLockCandidates owner;
 
         CacheObject val;
 
@@ -110,16 +113,23 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 mvccExtras(mvcc);
             }
 
-            prev = mvcc.anyOwner();
+            prev = mvcc.allOwners();
 
             boolean emptyBefore = mvcc.isEmpty();
 
-            cand = mvcc.addLocal(this, threadId, ver, timeout, reenter, tx, implicitSingle);
+            cand = mvcc.addLocal(this,
+                threadId,
+                ver,
+                timeout,
+                reenter,
+                tx,
+                implicitSingle,
+                read);
 
             if (cand != null)
                 cand.topologyVersion(topVer);
 
-            owner = mvcc.anyOwner();
+            owner = mvcc.allOwners();
 
             boolean emptyAfter = mvcc.isEmpty();
 
@@ -168,7 +178,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @param otherNodeId Other node ID.
      * @param threadId Thread ID.
      * @param ver Lock version.
-     * @param timeout Lock acquire timeout.
      * @param tx Transaction flag.
      * @param implicitSingle Implicit flag.
      * @param owned Owned candidate version.
@@ -180,13 +189,12 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
         @Nullable UUID otherNodeId,
         long threadId,
         GridCacheVersion ver,
-        long timeout,
         boolean tx,
         boolean implicitSingle,
         @Nullable GridCacheVersion owned
     ) throws GridDistributedLockCancelledException, GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate owner;
+        CacheLockCandidates prev;
+        CacheLockCandidates owner;
 
         CacheObject val;
 
@@ -204,7 +212,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 mvccExtras(mvcc);
             }
 
-            prev = mvcc.anyOwner();
+            prev = mvcc.allOwners();
 
             boolean emptyBefore = mvcc.isEmpty();
 
@@ -214,7 +222,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 otherNodeId,
                 threadId,
                 ver,
-                timeout,
                 tx,
                 implicitSingle,
                 /*near-local*/false
@@ -223,62 +230,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             if (owned != null)
                 mvcc.markOwned(ver, owned);
 
-            owner = mvcc.anyOwner();
-
-            boolean emptyAfter = mvcc.isEmpty();
-
-            checkCallbacks(emptyBefore, emptyAfter);
-
-            val = this.val;
-
-            refreshRemotes();
-
-            if (emptyAfter)
-                mvccExtras(null);
-        }
-
-        // This call must be outside of synchronization.
-        checkOwnerChanged(prev, owner, val);
-    }
-
-    /**
-     * Adds new lock candidate.
-     *
-     * @param cand Remote lock candidate.
-     * @throws GridDistributedLockCancelledException If lock has been canceled.
-     * @throws GridCacheEntryRemovedException If this entry is obsolete.
-     */
-    public void addRemote(GridCacheMvccCandidate cand) throws GridDistributedLockCancelledException,
-        GridCacheEntryRemovedException {
-
-        CacheObject val;
-
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate owner;
-
-        synchronized (this) {
-            cand.parent(this);
-
-            // Check removed locks prior to obsolete flag.
-            checkRemoved(cand.version());
-
-            checkObsolete();
-
-            GridCacheMvcc mvcc = mvccExtras();
-
-            if (mvcc == null) {
-                mvcc = new GridCacheMvcc(cctx);
-
-                mvccExtras(mvcc);
-            }
-
-            boolean emptyBefore = mvcc.isEmpty();
-
-            prev = mvcc.anyOwner();
-
-            mvcc.addRemote(cand);
-
-            owner = mvcc.anyOwner();
+            owner = mvcc.allOwners();
 
             boolean emptyAfter = mvcc.isEmpty();
 
@@ -303,8 +255,8 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public void removeExplicitNodeLocks(UUID nodeId) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val = null;
 
@@ -314,7 +266,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
@@ -346,8 +298,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @return Removed candidate, or <tt>null</tt> if thread still holds the lock.
      */
     @Nullable public GridCacheMvccCandidate removeLock() {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        GridCacheMvccCandidate rmvd = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -355,11 +308,11 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
-                owner = mvcc.releaseLocal();
+                rmvd = mvcc.releaseLocal();
 
                 boolean emptyAfter = mvcc.isEmpty();
 
@@ -367,28 +320,38 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
                 if (emptyAfter)
                     mvccExtras(null);
+                else
+                    owner = mvcc.allOwners();
             }
 
             val = this.val;
         }
 
-        if (log.isDebugEnabled())
-            log.debug("Released local candidate from entry [owner=" + owner + ", prev=" + prev +
+        if (log.isDebugEnabled()) {
+            log.debug("Released local candidate from entry [owner=" + owner +
+                ", prev=" + prev +
+                ", rmvd=" + rmvd +
                 ", entry=" + this + ']');
+        }
+
+        if (prev != null) {
+            for (int i = 0; i < prev.size(); i++) {
+                GridCacheMvccCandidate cand = prev.candidate(i);
 
-        if (prev != null && owner != prev)
-            checkThreadChain(prev);
+                checkThreadChain(cand);
+            }
+        }
 
         // This call must be outside of synchronization.
         checkOwnerChanged(prev, owner, val);
 
-        return owner != prev ? prev : null;
+        return rmvd;
     }
 
     /** {@inheritDoc} */
     @Override public boolean removeLock(GridCacheVersion ver) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         GridCacheMvccCandidate doomed;
 
@@ -408,13 +371,11 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 checkObsolete();
 
             if (doomed != null) {
-                assert mvcc != null;
-
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
-                owner = mvcc.remove(doomed.version());
+                mvcc.remove(doomed.version());
 
                 boolean emptyAfter = mvcc.isEmpty();
 
@@ -425,6 +386,8 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
                 if (emptyAfter)
                     mvccExtras(null);
+                else
+                    owner = mvcc.allOwners();
             }
 
             val = this.val;
@@ -477,10 +440,10 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @return Owner.
      * @throws GridCacheEntryRemovedException If entry is removed.
      */
-    @Nullable public GridCacheMvccCandidate readyLock(GridCacheVersion ver)
+    @Nullable public CacheLockCandidates readyLock(GridCacheVersion ver)
         throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -490,13 +453,13 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
                 owner = mvcc.readyLocal(ver);
 
-                assert owner == null || owner.owner() : "Owner flag not set for owner: " + owner;
+                assert owner == null || owner.candidate(0).owner() : "Owner flag not set for owner: " + owner;
 
                 boolean emptyAfter = mvcc.isEmpty();
 
@@ -523,16 +486,16 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      * @param committed Committed versions.
      * @param rolledBack Rolled back versions.
      * @param pending Pending locks on dht node with version less then mapped.
-     * @return Current lock owner.
      *
      * @throws GridCacheEntryRemovedException If entry is removed.
      */
-    @Nullable public GridCacheMvccCandidate readyNearLock(GridCacheVersion ver, GridCacheVersion mapped,
+    public void readyNearLock(GridCacheVersion ver, GridCacheVersion mapped,
         Collection<GridCacheVersion> committed,
         Collection<GridCacheVersion> rolledBack,
-        Collection<GridCacheVersion> pending) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        Collection<GridCacheVersion> pending) throws GridCacheEntryRemovedException
+    {
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -542,13 +505,13 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
                 owner = mvcc.readyNearLocal(ver, mapped, committed, rolledBack, pending);
 
-                assert owner == null || owner.owner() : "Owner flag is not set for owner: " + owner;
+                assert owner == null || owner.candidate(0).owner() : "Owner flag is not set for owner: " + owner;
 
                 boolean emptyAfter = mvcc.isEmpty();
 
@@ -563,75 +526,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
         // This call must be made outside of synchronization.
         checkOwnerChanged(prev, owner, val);
-
-        return owner;
-    }
-
-    /**
-     * Reorders completed versions.
-     *
-     * @param baseVer Base version for reordering.
-     * @param committedVers Completed versions.
-     * @param rolledbackVers Rolled back versions.
-     * @throws GridCacheEntryRemovedException If entry has been removed.
-     */
-    public void orderCompleted(GridCacheVersion baseVer, Collection<GridCacheVersion> committedVers,
-        Collection<GridCacheVersion> rolledbackVers)
-        throws GridCacheEntryRemovedException {
-        if (!F.isEmpty(committedVers) || !F.isEmpty(rolledbackVers)) {
-            GridCacheMvccCandidate prev = null;
-            GridCacheMvccCandidate owner = null;
-
-            CacheObject val;
-
-            synchronized (this) {
-                checkObsolete();
-
-                GridCacheMvcc mvcc = mvccExtras();
-
-                if (mvcc != null) {
-                    prev = mvcc.anyOwner();
-
-                    boolean emptyBefore = mvcc.isEmpty();
-
-                    owner = mvcc.orderCompleted(baseVer, committedVers, rolledbackVers);
-
-                    boolean emptyAfter = mvcc.isEmpty();
-
-                    checkCallbacks(emptyBefore, emptyAfter);
-
-                    if (emptyAfter)
-                        mvccExtras(null);
-                }
-
-                val = this.val;
-            }
-
-            // This call must be made outside of synchronization.
-            checkOwnerChanged(prev, owner, val);
-        }
-    }
-
-    /**
-     *
-     * @param lockVer Done version.
-     * @param baseVer Base version.
-     * @param committedVers Completed versions for reordering.
-     * @param rolledbackVers Rolled back versions for reordering.
-     * @param sysInvalidate Flag indicating if this entry is done from invalidated transaction (in case of tx
-     *      salvage). In this case all locks before salvaged lock will marked as used and corresponding
-     *      transactions will be invalidated.
-     * @throws GridCacheEntryRemovedException If entry has been removed.
-     * @return Owner.
-     */
-    @Nullable public GridCacheMvccCandidate doneRemote(
-        GridCacheVersion lockVer,
-        GridCacheVersion baseVer,
-        Collection<GridCacheVersion> committedVers,
-        Collection<GridCacheVersion> rolledbackVers,
-        boolean sysInvalidate) throws GridCacheEntryRemovedException {
-        return doneRemote(lockVer, baseVer, Collections.<GridCacheVersion>emptySet(), committedVers,
-            rolledbackVers, sysInvalidate);
     }
 
     /**
@@ -645,17 +539,16 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      *      salvage). In this case all locks before salvaged lock will marked as used and corresponding
      *      transactions will be invalidated.
      * @throws GridCacheEntryRemovedException If entry has been removed.
-     * @return Owner.
      */
-    @Nullable public GridCacheMvccCandidate doneRemote(
+    public void doneRemote(
         GridCacheVersion lockVer,
         GridCacheVersion baseVer,
         @Nullable Collection<GridCacheVersion> pendingVers,
         Collection<GridCacheVersion> committedVers,
         Collection<GridCacheVersion> rolledbackVers,
         boolean sysInvalidate) throws GridCacheEntryRemovedException {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -665,7 +558,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
@@ -680,7 +573,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 if (sysInvalidate && baseVer != null)
                     mvcc.salvageRemote(baseVer);
 
-                owner = mvcc.doneRemote(lockVer, maskNull(pendingVers), maskNull(committedVers),
+                owner = mvcc.doneRemote(lockVer,
+                    maskNull(pendingVers),
+                    maskNull(committedVers),
                     maskNull(rolledbackVers));
 
                 boolean emptyAfter = mvcc.isEmpty();
@@ -696,18 +591,14 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
         // This call must be made outside of synchronization.
         checkOwnerChanged(prev, owner, val);
-
-        return owner;
     }
 
     /**
      * Rechecks if lock should be reassigned.
-     *
-     * @return Current owner.
      */
-    @Nullable public GridCacheMvccCandidate recheck() {
-        GridCacheMvccCandidate prev = null;
-        GridCacheMvccCandidate owner = null;
+    public void recheck() {
+        CacheLockCandidates prev = null;
+        CacheLockCandidates owner = null;
 
         CacheObject val;
 
@@ -715,7 +606,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             GridCacheMvcc mvcc = mvccExtras();
 
             if (mvcc != null) {
-                prev = mvcc.anyOwner();
+                prev = mvcc.allOwners();
 
                 boolean emptyBefore = mvcc.isEmpty();
 
@@ -734,8 +625,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
         // This call must be made outside of synchronization.
         checkOwnerChanged(prev, owner, val);
-
-        return owner;
     }
 
     /** {@inheritDoc} */
@@ -743,7 +632,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
         long timeout,
         @Nullable GridCacheVersion serOrder,
         GridCacheVersion serReadVer,
-        boolean keepBinary
+        boolean read
     ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local())
             // Null is returned if timeout is negative and there is other lock owner.
@@ -754,7 +643,8 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 timeout,
                 /*reenter*/false,
                 /*tx*/true,
-                tx.implicitSingle()) != null;
+                tx.implicitSingle(),
+                read) != null;
 
         try {
             addRemote(
@@ -762,7 +652,6 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                 tx.otherNodeId(),
                 tx.threadId(),
                 tx.xidVersion(),
-                tx.timeout(),
                 /*tx*/true,
                 tx.implicitSingle(),
                 tx.ownedVersion(txKey())
@@ -779,7 +668,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     }
 
     /** {@inheritDoc} */
-    @Override public void txUnlock(IgniteInternalTx tx) throws GridCacheEntryRemovedException {
+    @Override public final void txUnlock(IgniteInternalTx tx) throws GridCacheEntryRemovedException {
         removeLock(tx.xidVersion());
     }
 
@@ -799,42 +688,8 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
         }
     }
 
-    /**
-     * @param prev Previous owner.
-     * @param owner Current owner.
-     * @param val Entry value.
-     */
-    protected void checkOwnerChanged(GridCacheMvccCandidate prev, GridCacheMvccCandidate owner, CacheObject val) {
-        assert !Thread.holdsLock(this);
-
-        if (owner != prev) {
-            cctx.mvcc().callback().onOwnerChanged(this, prev, owner);
-
-            if (owner != null && owner.local())
-                checkThreadChain(owner);
-
-            if (prev != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_UNLOCKED)) {
-                boolean hasVal = hasValue();
-
-                // Event notification.
-                cctx.events().addEvent(partition(), key, prev.nodeId(), prev, EVT_CACHE_OBJECT_UNLOCKED, val, hasVal,
-                    val, hasVal, null, null, null, true);
-            }
-
-            if (owner != null && cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED)) {
-                boolean hasVal = hasValue();
-
-                // Event notification.
-                cctx.events().addEvent(partition(), key, owner.nodeId(), owner, EVT_CACHE_OBJECT_LOCKED, val, hasVal,
-                    val, hasVal, null, null, null, true);
-            }
-        }
-    }
-
-    /**
-     * @param owner Starting candidate in the chain.
-     */
-    protected void checkThreadChain(GridCacheMvccCandidate owner) {
+    /** {@inheritDoc} */
+    @Override final protected void checkThreadChain(GridCacheMvccCandidate owner) {
         assert !Thread.holdsLock(this);
 
         assert owner != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/33dda460/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 b0b0a7e..cf4085b 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
@@ -27,6 +27,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.CacheLockCandidates;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryRemovedException;
@@ -135,7 +136,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
      * @return Local candidate by near version.
      * @throws GridCacheEntryRemovedException If removed.
      */
-    @Nullable public synchronized GridCacheMvccCandidate localCandidateByNearVersion(GridCacheVersion nearVer,
+    @Nullable synchronized GridCacheMvccCandidate localCandidateByNearVersion(GridCacheVersion nearVer,
         boolean rmv) throws GridCacheEntryRemovedException {
         checkObsolete();
 
@@ -165,34 +166,33 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
      * @param threadId Owning thread ID.
      * @param ver Lock version.
      * @param serOrder Version for serializable transactions ordering.
-     * @param serReadVer Optional read entry version for optimistic serializable transaction.
      * @param timeout Timeout to acquire lock.
      * @param reenter Reentry flag.
      * @param tx Tx flag.
      * @param implicitSingle Implicit flag.
+     * @param read Read lock flag.
      * @return New candidate.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      * @throws GridDistributedLockCancelledException If lock was cancelled.
      */
-    @Nullable public GridCacheMvccCandidate addDhtLocal(
+    @Nullable GridCacheMvccCandidate addDhtLocal(
         UUID nearNodeId,
         GridCacheVersion nearVer,
         AffinityTopologyVersion topVer,
         long threadId,
         GridCacheVersion ver,
         @Nullable GridCacheVersion serOrder,
-        @Nullable GridCacheVersion serReadVer,
         long timeout,
         boolean reenter,
         boolean tx,
-        boolean implicitSingle)
+        boolean implicitSingle,
+        boolean read)
         throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
-        assert serReadVer == null || serOrder != null;
         assert !reenter || serOrder == null;
 
         GridCacheMvccCandidate cand;
-        GridCacheMvccCandidate prev;
-        GridCacheMvccCandidate owner;
+        CacheLockCandidates prev;
+        CacheLockCandidates owner;
 
         CacheObject val;
 
@@ -211,7 +211,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 mvccExtras(mvcc);
             }
 
-            prev = mvcc.anyOwner();
+            prev = mvcc.allOwners();
 
             boolean emptyBefore = mvcc.isEmpty();
 
@@ -226,7 +226,8 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 reenter,
                 tx,
                 implicitSingle,
-                /*dht-local*/true
+                /*dht-local*/true,
+                read
             );
 
             if (cand == null)
@@ -234,10 +235,10 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
 
             cand.topologyVersion(topVer);
 
-            owner = mvcc.anyOwner();
+            owner = mvcc.allOwners();
 
             if (owner != null)
-                cand.ownerVersion(owner.version());
+                cand.ownerVersion(owner.candidate(0).version());
 
             boolean emptyAfter = mvcc.isEmpty();
 
@@ -264,7 +265,7 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
         long timeout,
         @Nullable GridCacheVersion serOrder,
         GridCacheVersion serReadVer,
-        boolean keepBinary
+        boolean read
     ) throws GridCacheEntryRemovedException, GridDistributedLockCancelledException {
         if (tx.local()) {
             GridDhtTxLocalAdapter dhtTx = (GridDhtTxLocalAdapter)tx;
@@ -277,11 +278,11 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 tx.threadId(),
                 tx.xidVersion(),
                 serOrder,
-                serReadVer,
                 timeout,
                 /*reenter*/false,
                 /*tx*/true,
-                tx.implicitSingle()) != null;
+                tx.implicitSingle(),
+                read) != null;
         }
 
         try {
@@ -290,7 +291,6 @@ public class GridDhtCacheEntry extends GridDistributedCacheEntry {
                 tx.otherNodeId(),
                 tx.threadId(),
                 tx.xidVersion(),
-                tx.timeout(),
                 /*tx*/true,
                 tx.implicit(),
                 null);


[10/19] ignite git commit: IGNITE-4305 marshalling fix in GridNearAtomicSingleUpdateInvokeRequest

Posted by sb...@apache.org.
IGNITE-4305 marshalling fix in GridNearAtomicSingleUpdateInvokeRequest


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

Branch: refs/heads/master
Commit: 6fbaef45af8f40062a95058df7ec0984c99035b9
Parents: 3e2ccfd
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Fri Nov 25 13:58:58 2016 +0300
Committer: Konstantin Dudkov <kd...@ya.ru>
Committed: Fri Nov 25 16:32:52 2016 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java      | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6fbaef45/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index 42b51d6..238db8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -202,8 +202,10 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
         if (!addDepInfo && ctx.deploymentEnabled())
             addDepInfo = true;
 
-        if (entryProcessor != null && entryProcessorBytes == null)
+        if (entryProcessor != null && entryProcessorBytes == null) {
+            prepareObject(entryProcessor, cctx);
             entryProcessorBytes = CU.marshal(cctx, entryProcessor);
+        }
 
         if (invokeArgsBytes == null)
             invokeArgsBytes = marshalInvokeArguments(invokeArgs, cctx);


[11/19] ignite git commit: IGNITE-4305 marshalling fix

Posted by sb...@apache.org.
IGNITE-4305 marshalling fix


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

Branch: refs/heads/master
Commit: 1a2de51f5807a91ce0d5dff28f24ed5bf7abebbc
Parents: 6fbaef4
Author: Konstantin Dudkov <kd...@ya.ru>
Authored: Mon Nov 28 12:59:02 2016 +0300
Committer: Konstantin Dudkov <kd...@ya.ru>
Committed: Tue Nov 29 11:53:07 2016 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java       | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1a2de51f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
index 238db8b..df9e384 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
@@ -203,7 +203,8 @@ public class GridNearAtomicSingleUpdateInvokeRequest extends GridNearAtomicSingl
             addDepInfo = true;
 
         if (entryProcessor != null && entryProcessorBytes == null) {
-            prepareObject(entryProcessor, cctx);
+            if (addDepInfo)
+                prepareObject(entryProcessor, cctx);
             entryProcessorBytes = CU.marshal(cctx, entryProcessor);
         }