You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/12/26 11:16:17 UTC

[01/50] [abbrv] ignite git commit: IGNITE-4386: Hadoop: implemented client cleanup on protocol close. This closes #1327. This closes #1339.

Repository: ignite
Updated Branches:
  refs/heads/ignite-2.0 c893da70a -> c5882a85f


IGNITE-4386: Hadoop: implemented client cleanup on protocol close. This closes #1327. This closes #1339.


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

Branch: refs/heads/ignite-2.0
Commit: ffe53eb5a59908db3684ce11474cb875c4bf392d
Parents: a976c42
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 12 11:29:23 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:24 2016 +0300

----------------------------------------------------------------------
 .../IgniteHadoopClientProtocolProvider.java     |  70 ++----
 .../hadoop/impl/proto/HadoopClientProtocol.java |  55 +++--
 .../hadoop/mapreduce/MapReduceClient.java       | 147 ++++++++++++
 ...opClientProtocolMultipleServersSelfTest.java |  93 +++-----
 .../client/HadoopClientProtocolSelfTest.java    | 228 ++++++++++---------
 5 files changed, 367 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ffe53eb5/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
index 1efe625..920e8b7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/mapreduce/IgniteHadoopClientProtocolProvider.java
@@ -23,24 +23,16 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.concurrent.ConcurrentHashMap;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.MRConfig;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocol;
 import org.apache.hadoop.mapreduce.protocol.ClientProtocolProvider;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.ConnectorConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.client.GridClient;
-import org.apache.ignite.internal.client.GridClientConfiguration;
-import org.apache.ignite.internal.client.GridClientException;
-import org.apache.ignite.internal.client.GridClientFactory;
-import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
 import org.apache.ignite.internal.processors.hadoop.impl.proto.HadoopClientProtocol;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.processors.hadoop.mapreduce.MapReduceClient;
 import org.apache.ignite.internal.util.typedef.F;
 
-import static org.apache.ignite.internal.client.GridClientProtocol.TCP;
-
 
 /**
  * Ignite Hadoop client protocol provider.
@@ -50,7 +42,7 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider {
     public static final String FRAMEWORK_NAME = "ignite";
 
     /** Clients. */
-    private static final ConcurrentHashMap<String, IgniteInternalFuture<GridClient>> cliMap = new ConcurrentHashMap<>();
+    private final ConcurrentHashMap<String, MapReduceClient> cliMap = new ConcurrentHashMap<>();
 
     /** {@inheritDoc} */
     @Override public ClientProtocol create(Configuration conf) throws IOException {
@@ -91,7 +83,12 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider {
 
     /** {@inheritDoc} */
     @Override public void close(ClientProtocol cliProto) throws IOException {
-        // No-op.
+        if (cliProto instanceof HadoopClientProtocol) {
+            MapReduceClient cli = ((HadoopClientProtocol)cliProto).client();
+
+            if (cli.release())
+                cliMap.remove(cli.cluster(), cli);
+        }
     }
 
     /**
@@ -102,7 +99,7 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider {
      * @return Client protocol.
      * @throws IOException If failed.
      */
-    private static ClientProtocol createProtocol(String addr, Configuration conf) throws IOException {
+    private ClientProtocol createProtocol(String addr, Configuration conf) throws IOException {
         return new HadoopClientProtocol(conf, client(addr, Collections.singletonList(addr)));
     }
 
@@ -114,45 +111,24 @@ public class IgniteHadoopClientProtocolProvider extends ClientProtocolProvider {
      * @return Client.
      * @throws IOException If failed.
      */
-    private static GridClient client(String clusterName, Collection<String> addrs) throws IOException {
-        try {
-            IgniteInternalFuture<GridClient> fut = cliMap.get(clusterName);
-
-            if (fut == null) {
-                GridFutureAdapter<GridClient> fut0 = new GridFutureAdapter<>();
-
-                IgniteInternalFuture<GridClient> oldFut = cliMap.putIfAbsent(clusterName, fut0);
+    @SuppressWarnings("unchecked")
+    private MapReduceClient client(String clusterName, Collection<String> addrs) throws IOException {
+        while (true) {
+            MapReduceClient cli = cliMap.get(clusterName);
 
-                if (oldFut != null)
-                    return oldFut.get();
-                else {
-                    GridClientConfiguration cliCfg = new GridClientConfiguration();
+            if (cli == null) {
+                cli = new MapReduceClient(clusterName, addrs);
 
-                    cliCfg.setProtocol(TCP);
-                    cliCfg.setServers(addrs);
-                    cliCfg.setMarshaller(new GridClientJdkMarshaller());
-                    cliCfg.setMaxConnectionIdleTime(24 * 60 * 60 * 1000L); // 1 day.
-                    cliCfg.setDaemon(true);
+                MapReduceClient oldCli = cliMap.putIfAbsent(clusterName, cli);
 
-                    try {
-                        GridClient cli = GridClientFactory.start(cliCfg);
-
-                        fut0.onDone(cli);
-
-                        return cli;
-                    }
-                    catch (GridClientException e) {
-                        fut0.onDone(e);
-
-                        throw new IOException("Failed to establish connection with Ignite: " + addrs, e);
-                    }
-                }
+                if (oldCli != null)
+                    cli = oldCli;
             }
+
+            if (cli.acquire())
+                return cli;
             else
-                return fut.get();
-        }
-        catch (IgniteCheckedException e) {
-            throw new IOException("Failed to establish connection with Ignite \u0441\u0434\u0433\u044b\u0435: " + addrs, e);
+                cliMap.remove(clusterName, cli);
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ffe53eb5/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java
index be2aa09..7fc0e77 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/proto/HadoopClientProtocol.java
@@ -43,7 +43,7 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.security.token.Token;
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.processors.hadoop.mapreduce.MapReduceClient;
 import org.apache.ignite.internal.client.GridClientException;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
@@ -78,7 +78,7 @@ public class HadoopClientProtocol implements ClientProtocol {
     private final Configuration conf;
 
     /** Ignite client. */
-    private volatile GridClient cli;
+    private final MapReduceClient cli;
 
     /** Last received version. */
     private long lastVer = -1;
@@ -90,9 +90,10 @@ public class HadoopClientProtocol implements ClientProtocol {
      * Constructor.
      *
      * @param conf Configuration.
-     * @param cli Ignite client.
+     * @param cli Client.
      */
-    public HadoopClientProtocol(Configuration conf, GridClient cli) {
+    public HadoopClientProtocol(Configuration conf, MapReduceClient cli) {
+        assert conf != null;
         assert cli != null;
 
         this.conf = conf;
@@ -104,7 +105,7 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             conf.setLong(HadoopCommonUtils.REQ_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
 
-            HadoopJobId jobID = cli.compute().execute(HadoopProtocolNextTaskIdTask.class.getName(), null);
+            HadoopJobId jobID = execute(HadoopProtocolNextTaskIdTask.class);
 
             conf.setLong(HadoopCommonUtils.RESPONSE_NEW_JOBID_TS_PROPERTY, U.currentTimeMillis());
 
@@ -121,8 +122,8 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             conf.setLong(HadoopCommonUtils.JOB_SUBMISSION_START_TS_PROPERTY, U.currentTimeMillis());
 
-            HadoopJobStatus status = cli.compute().execute(HadoopProtocolSubmitJobTask.class.getName(),
-                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), createJobInfo(conf)));
+            HadoopJobStatus status = execute(HadoopProtocolSubmitJobTask.class,
+                jobId.getJtIdentifier(), jobId.getId(), createJobInfo(conf));
 
             if (status == null)
                 throw new IOException("Failed to submit job (null status obtained): " + jobId);
@@ -157,8 +158,7 @@ public class HadoopClientProtocol implements ClientProtocol {
     /** {@inheritDoc} */
     @Override public void killJob(JobID jobId) throws IOException, InterruptedException {
         try {
-            cli.compute().execute(HadoopProtocolKillJobTask.class.getName(),
-                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
+            execute(HadoopProtocolKillJobTask.class, jobId.getJtIdentifier(), jobId.getId());
         }
         catch (GridClientException e) {
             throw new IOException("Failed to kill job: " + jobId, e);
@@ -181,11 +181,12 @@ public class HadoopClientProtocol implements ClientProtocol {
         try {
             Long delay = conf.getLong(HadoopJobProperty.JOB_STATUS_POLL_DELAY.propertyName(), -1);
 
-            HadoopProtocolTaskArguments args = delay >= 0 ?
-                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId(), delay) :
-                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId());
+            HadoopJobStatus status;
 
-            HadoopJobStatus status = cli.compute().execute(HadoopProtocolJobStatusTask.class.getName(), args);
+            if (delay >= 0)
+                status = execute(HadoopProtocolJobStatusTask.class, jobId.getJtIdentifier(), jobId.getId(), delay);
+            else
+                status = execute(HadoopProtocolJobStatusTask.class, jobId.getJtIdentifier(), jobId.getId());
 
             if (status == null)
                 throw new IOException("Job tracker doesn't have any information about the job: " + jobId);
@@ -200,8 +201,8 @@ public class HadoopClientProtocol implements ClientProtocol {
     /** {@inheritDoc} */
     @Override public Counters getJobCounters(JobID jobId) throws IOException, InterruptedException {
         try {
-            final HadoopCounters counters = cli.compute().execute(HadoopProtocolJobCountersTask.class.getName(),
-                new HadoopProtocolTaskArguments(jobId.getJtIdentifier(), jobId.getId()));
+            final HadoopCounters counters = execute(HadoopProtocolJobCountersTask.class,
+                jobId.getJtIdentifier(), jobId.getId());
 
             if (counters == null)
                 throw new IOException("Job tracker doesn't have any information about the job: " + jobId);
@@ -329,6 +330,21 @@ public class HadoopClientProtocol implements ClientProtocol {
     }
 
     /**
+     * Execute task.
+     *
+     * @param taskCls Task class.
+     * @param args Arguments.
+     * @return Result.
+     * @throws IOException If failed.
+     * @throws GridClientException If failed.
+     */
+    private <T> T execute(Class taskCls, Object... args) throws IOException, GridClientException {
+        HadoopProtocolTaskArguments args0 = args != null ? new HadoopProtocolTaskArguments(args) : null;
+
+        return cli.client().compute().execute(taskCls.getName(), args0);
+    }
+
+    /**
      * Process received status update.
      *
      * @param status Ignite status.
@@ -351,4 +367,13 @@ public class HadoopClientProtocol implements ClientProtocol {
 
         return HadoopUtils.status(lastStatus, conf);
     }
+
+    /**
+     * Gets the GridClient data.
+     *
+     * @return The client data.
+     */
+    public MapReduceClient client() {
+        return cli;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/ffe53eb5/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java
new file mode 100644
index 0000000..3d52176
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/mapreduce/MapReduceClient.java
@@ -0,0 +1,147 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.mapreduce;
+
+import org.apache.ignite.internal.client.GridClient;
+import org.apache.ignite.internal.client.GridClientConfiguration;
+import org.apache.ignite.internal.client.GridClientException;
+import org.apache.ignite.internal.client.GridClientFactory;
+import org.apache.ignite.internal.client.marshaller.jdk.GridClientJdkMarshaller;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.ignite.internal.client.GridClientProtocol.TCP;
+
+/**
+ * Client.
+ */
+public class MapReduceClient {
+    /** Cluster name. */
+    private final String cluster;
+
+    /** Addresses. */
+    private final Collection<String> addrs;
+
+    /** Mutex. */
+    private final Object mux = new Object();
+
+    /** Usage counter. */
+    private final AtomicInteger cnt = new AtomicInteger();
+
+    /** Client. */
+    private volatile GridClient cli;
+
+    /**
+     * Constructor.
+     *
+     * @param cluster Cluster name.
+     * @param addrs Addresses.
+     */
+    public MapReduceClient(String cluster, Collection<String> addrs) {
+        this.cluster = cluster;
+        this.addrs = addrs;
+    }
+
+    /**
+     * @return Cluster name..
+     */
+    public String cluster() {
+        return cluster;
+    }
+
+    /**
+     * Gets the client.
+     *
+     * @return The client.
+     */
+    public GridClient client() throws IOException {
+        GridClient cli0 = cli;
+
+        if (cli0 == null) {
+            synchronized (mux) {
+                cli0 = cli;
+
+                if (cli0 == null) {
+                    GridClientConfiguration cliCfg = new GridClientConfiguration();
+
+                    cliCfg.setProtocol(TCP);
+                    cliCfg.setServers(addrs);
+                    cliCfg.setMarshaller(new GridClientJdkMarshaller());
+                    cliCfg.setMaxConnectionIdleTime(24 * 60 * 60 * 1000L); // 1 day.
+                    cliCfg.setDaemon(true);
+
+                    try {
+                        cli0 = GridClientFactory.start(cliCfg);
+
+                        cli = cli0;
+                    }
+                    catch (GridClientException e) {
+                        throw new IOException("Failed to establish connection with Ignite: " + addrs, e);
+                    }
+                }
+            }
+        }
+
+        return cli0;
+    }
+
+    /**
+     * Increments usage count.
+     *
+     * @return {@code True} if succeeded and client can be used.
+     */
+    public boolean acquire() {
+        while (true) {
+            int cur = cnt.get();
+
+            if (cur < 0)
+                return false;
+
+            int next = cur + 1;
+
+            if (cnt.compareAndSet(cur, next))
+                return true;
+        }
+    }
+
+    /**
+     * Decrements the usages of the client and closes it if this is the last usage.
+     *
+     * @return {@code True} if client can be closed safely by the called.
+     */
+    public boolean release() {
+        int cnt0 = cnt.decrementAndGet();
+
+        assert cnt0 >= 0;
+
+        if (cnt0 == 0) {
+            if (cnt.compareAndSet(0, -1)) {
+                GridClient cli0 = cli;
+
+                if (cli0 != null)
+                    cli0.close();
+
+                return true;
+            }
+        }
+
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ffe53eb5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java
index 0805be1..a4b5e6a 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolMultipleServersSelfTest.java
@@ -23,8 +23,8 @@ import java.io.OutputStreamWriter;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.concurrent.Callable;
-import java.util.concurrent.ConcurrentHashMap;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
@@ -40,13 +40,10 @@ import org.apache.hadoop.mapreduce.Reducer;
 import org.apache.hadoop.mapreduce.TaskAttemptContext;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
-import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider;
 import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.client.GridClient;
 import org.apache.ignite.internal.client.GridServerUnreachableException;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
@@ -79,34 +76,12 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTest() throws Exception {
-        super.beforeTest();
-
-        clearClients();
-    }
-
-    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
-        clearClients();
-
         super.afterTest();
     }
 
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    private void clearConnectionMap() throws IgniteCheckedException {
-        ConcurrentHashMap<String, IgniteInternalFuture<GridClient>> cliMap =
-            GridTestUtils.getFieldValue(IgniteHadoopClientProtocolProvider.class, "cliMap");
-
-        for(IgniteInternalFuture<GridClient> fut : cliMap.values())
-            fut.get().close();
-
-        cliMap.clear();
-    }
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -117,18 +92,6 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS
     }
 
     /**
-     *
-     */
-    private void clearClients() {
-        ConcurrentHashMap<String, IgniteInternalFuture<GridClient>> cliMap = GridTestUtils.getFieldValue(
-            IgniteHadoopClientProtocolProvider.class,
-            IgniteHadoopClientProtocolProvider.class,
-            "cliMap");
-
-        cliMap.clear();
-    }
-
-    /**
      * @throws Exception If failed.
      */
     private void beforeJob() throws Exception {
@@ -154,26 +117,31 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS
     private void checkJobSubmit(Configuration conf) throws Exception {
         final Job job = Job.getInstance(conf);
 
-        job.setJobName(JOB_NAME);
+        try {
+            job.setJobName(JOB_NAME);
 
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(IntWritable.class);
 
-        job.setInputFormatClass(TextInputFormat.class);
-        job.setOutputFormatClass(OutFormat.class);
+            job.setInputFormatClass(TextInputFormat.class);
+            job.setOutputFormatClass(OutFormat.class);
 
-        job.setMapperClass(TestMapper.class);
-        job.setReducerClass(TestReducer.class);
+            job.setMapperClass(TestMapper.class);
+            job.setReducerClass(TestReducer.class);
 
-        job.setNumReduceTasks(0);
+            job.setNumReduceTasks(0);
 
-        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
+            FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
 
-        job.submit();
+            job.submit();
 
-        job.waitForCompletion(false);
+            job.waitForCompletion(false);
 
-        assert job.getStatus().getState() == JobStatus.State.SUCCEEDED : job.getStatus().getState();
+            assert job.getStatus().getState() == JobStatus.State.SUCCEEDED : job.getStatus().getState();
+        }
+        finally {
+            job.getCluster().close();
+        }
     }
 
     /**
@@ -197,18 +165,25 @@ public class HadoopClientProtocolMultipleServersSelfTest extends HadoopAbstractS
      */
     @SuppressWarnings({"ConstantConditions", "ThrowableResultOfMethodCallIgnored"})
     public void testSingleAddress() throws Exception {
-        // Don't use REST_PORT to test connection fails if the only this port is configured
-        restPort = REST_PORT + 1;
+        try {
+            // Don't use REST_PORT to test connection fails if the only this port is configured
+            restPort = REST_PORT + 1;
 
-        startGrids(gridCount());
+            startGrids(gridCount());
 
-        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    checkJobSubmit(configSingleAddress());
-                    return null;
-                }
-            },
-            GridServerUnreachableException.class, "Failed to connect to any of the servers in list");
+                        checkJobSubmit(configSingleAddress());
+                        return null;
+                    }
+                },
+                GridServerUnreachableException.class, "Failed to connect to any of the servers in list");
+        }
+        finally {
+            FileSystem fs = FileSystem.get(configSingleAddress());
+
+            fs.close();
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ffe53eb5/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
index 1ef7dd0..7156a3d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
@@ -50,7 +50,6 @@ import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsPath;
-import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopAbstractSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils;
 import org.apache.ignite.internal.util.lang.GridAbsPredicate;
@@ -115,7 +114,6 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
         stopAllGrids();
 
         super.afterTestsStopped();
-//        IgniteHadoopClientProtocolProvider.cliMap.clear();
     }
 
     /** {@inheritDoc} */
@@ -196,43 +194,48 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
 
         final Job job = Job.getInstance(conf);
 
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
+        try {
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(IntWritable.class);
 
-        job.setMapperClass(TestCountingMapper.class);
-        job.setReducerClass(TestCountingReducer.class);
-        job.setCombinerClass(TestCountingCombiner.class);
+            job.setMapperClass(TestCountingMapper.class);
+            job.setReducerClass(TestCountingReducer.class);
+            job.setCombinerClass(TestCountingCombiner.class);
 
-        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
-        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
+            FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
+            FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
 
-        job.submit();
+            job.submit();
 
-        final Counter cntr = job.getCounters().findCounter(TestCounter.COUNTER1);
+            final Counter cntr = job.getCounters().findCounter(TestCounter.COUNTER1);
 
-        assertEquals(0, cntr.getValue());
+            assertEquals(0, cntr.getValue());
 
-        cntr.increment(10);
+            cntr.increment(10);
 
-        assertEquals(10, cntr.getValue());
+            assertEquals(10, cntr.getValue());
 
-        // Transferring to map phase.
-        setupLockFile.delete();
+            // Transferring to map phase.
+            setupLockFile.delete();
 
-        // Transferring to reduce phase.
-        mapLockFile.delete();
+            // Transferring to reduce phase.
+            mapLockFile.delete();
 
-        job.waitForCompletion(false);
+            job.waitForCompletion(false);
 
-        assertEquals("job must end successfully", JobStatus.State.SUCCEEDED, job.getStatus().getState());
+            assertEquals("job must end successfully", JobStatus.State.SUCCEEDED, job.getStatus().getState());
 
-        final Counters counters = job.getCounters();
+            final Counters counters = job.getCounters();
 
-        assertNotNull("counters cannot be null", counters);
-        assertEquals("wrong counters count", 3, counters.countCounters());
-        assertEquals("wrong counter value", 15, counters.findCounter(TestCounter.COUNTER1).getValue());
-        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER2).getValue());
-        assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER3).getValue());
+            assertNotNull("counters cannot be null", counters);
+            assertEquals("wrong counters count", 3, counters.countCounters());
+            assertEquals("wrong counter value", 15, counters.findCounter(TestCounter.COUNTER1).getValue());
+            assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER2).getValue());
+            assertEquals("wrong counter value", 3, counters.findCounter(TestCounter.COUNTER3).getValue());
+        }
+        finally {
+            job.getCluster().close();
+        }
     }
 
     /**
@@ -304,114 +307,119 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
 
         final Job job = Job.getInstance(conf);
 
-        job.setJobName(JOB_NAME);
+        try {
+            job.setJobName(JOB_NAME);
 
-        job.setOutputKeyClass(Text.class);
-        job.setOutputValueClass(IntWritable.class);
+            job.setOutputKeyClass(Text.class);
+            job.setOutputValueClass(IntWritable.class);
 
-        job.setMapperClass(TestMapper.class);
-        job.setReducerClass(TestReducer.class);
+            job.setMapperClass(TestMapper.class);
+            job.setReducerClass(TestReducer.class);
 
-        if (!noCombiners)
-            job.setCombinerClass(TestCombiner.class);
+            if (!noCombiners)
+                job.setCombinerClass(TestCombiner.class);
 
-        if (noReducers)
-            job.setNumReduceTasks(0);
+            if (noReducers)
+                job.setNumReduceTasks(0);
 
-        job.setInputFormatClass(TextInputFormat.class);
-        job.setOutputFormatClass(TestOutputFormat.class);
+            job.setInputFormatClass(TextInputFormat.class);
+            job.setOutputFormatClass(TestOutputFormat.class);
 
-        FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
-        FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
+            FileInputFormat.setInputPaths(job, new Path(PATH_INPUT));
+            FileOutputFormat.setOutputPath(job, new Path(PATH_OUTPUT));
 
-        job.submit();
+            job.submit();
 
-        JobID jobId = job.getJobID();
+            JobID jobId = job.getJobID();
 
-        // Setup phase.
-        JobStatus jobStatus = job.getStatus();
-        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-        assert jobStatus.getSetupProgress() >= 0.0f && jobStatus.getSetupProgress() < 1.0f;
-        assert jobStatus.getMapProgress() == 0.0f;
-        assert jobStatus.getReduceProgress() == 0.0f;
+            // Setup phase.
+            JobStatus jobStatus = job.getStatus();
+            checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+            assert jobStatus.getSetupProgress() >= 0.0f && jobStatus.getSetupProgress() < 1.0f;
+            assert jobStatus.getMapProgress() == 0.0f;
+            assert jobStatus.getReduceProgress() == 0.0f;
 
-        U.sleep(2100);
+            U.sleep(2100);
 
-        JobStatus recentJobStatus = job.getStatus();
+            JobStatus recentJobStatus = job.getStatus();
 
-        assert recentJobStatus.getSetupProgress() > jobStatus.getSetupProgress() :
-            "Old=" + jobStatus.getSetupProgress() + ", new=" + recentJobStatus.getSetupProgress();
+            assert recentJobStatus.getSetupProgress() > jobStatus.getSetupProgress() : "Old="
+                + jobStatus.getSetupProgress() + ", new=" + recentJobStatus.getSetupProgress();
 
-        // Transferring to map phase.
-        setupLockFile.delete();
+            // Transferring to map phase.
+            setupLockFile.delete();
 
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                try {
-                    return F.eq(1.0f, job.getStatus().getSetupProgress());
+            assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    try {
+                        return F.eq(1.0f, job.getStatus().getSetupProgress());
+                    }
+                    catch (Exception e) {
+                        throw new RuntimeException("Unexpected exception.", e);
+                    }
                 }
-                catch (Exception e) {
-                    throw new RuntimeException("Unexpected exception.", e);
-                }
-            }
-        }, 5000L);
+            }, 5000L);
 
-        // Map phase.
-        jobStatus = job.getStatus();
-        checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-        assert jobStatus.getSetupProgress() == 1.0f;
-        assert jobStatus.getMapProgress() >= 0.0f && jobStatus.getMapProgress() < 1.0f;
-        assert jobStatus.getReduceProgress() == 0.0f;
+            // Map phase.
+            jobStatus = job.getStatus();
+            checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+            assert jobStatus.getSetupProgress() == 1.0f;
+            assert jobStatus.getMapProgress() >= 0.0f && jobStatus.getMapProgress() < 1.0f;
+            assert jobStatus.getReduceProgress() == 0.0f;
 
-        U.sleep(2100);
+            U.sleep(2100);
 
-        recentJobStatus = job.getStatus();
+            recentJobStatus = job.getStatus();
 
-        assert recentJobStatus.getMapProgress() > jobStatus.getMapProgress() :
-            "Old=" + jobStatus.getMapProgress() + ", new=" + recentJobStatus.getMapProgress();
+            assert recentJobStatus.getMapProgress() > jobStatus.getMapProgress() : "Old=" + jobStatus.getMapProgress()
+                + ", new=" + recentJobStatus.getMapProgress();
 
-        // Transferring to reduce phase.
-        mapLockFile.delete();
+            // Transferring to reduce phase.
+            mapLockFile.delete();
 
-        assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-            @Override public boolean apply() {
-                try {
-                    return F.eq(1.0f, job.getStatus().getMapProgress());
-                }
-                catch (Exception e) {
-                    throw new RuntimeException("Unexpected exception.", e);
+            assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    try {
+                        return F.eq(1.0f, job.getStatus().getMapProgress());
+                    }
+                    catch (Exception e) {
+                        throw new RuntimeException("Unexpected exception.", e);
+                    }
                 }
-            }
-        }, 5000L);
+            }, 5000L);
 
-        if (!noReducers) {
-            // Reduce phase.
-            jobStatus = job.getStatus();
-            checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
-            assert jobStatus.getSetupProgress() == 1.0f;
-            assert jobStatus.getMapProgress() == 1.0f;
-            assert jobStatus.getReduceProgress() >= 0.0f && jobStatus.getReduceProgress() < 1.0f;
+            if (!noReducers) {
+                // Reduce phase.
+                jobStatus = job.getStatus();
+                checkJobStatus(jobStatus, jobId, JOB_NAME, JobStatus.State.RUNNING, 0.0f);
+                assert jobStatus.getSetupProgress() == 1.0f;
+                assert jobStatus.getMapProgress() == 1.0f;
+                assert jobStatus.getReduceProgress() >= 0.0f && jobStatus.getReduceProgress() < 1.0f;
 
-            // Ensure that reduces progress increases.
-            U.sleep(2100);
+                // Ensure that reduces progress increases.
+                U.sleep(2100);
 
-            recentJobStatus = job.getStatus();
+                recentJobStatus = job.getStatus();
 
-            assert recentJobStatus.getReduceProgress() > jobStatus.getReduceProgress() :
-                "Old=" + jobStatus.getReduceProgress() + ", new=" + recentJobStatus.getReduceProgress();
+                assert recentJobStatus.getReduceProgress() > jobStatus.getReduceProgress() : "Old="
+                    + jobStatus.getReduceProgress() + ", new=" + recentJobStatus.getReduceProgress();
 
-            reduceLockFile.delete();
-        }
+                reduceLockFile.delete();
+            }
 
-        job.waitForCompletion(false);
+            job.waitForCompletion(false);
 
-        jobStatus = job.getStatus();
-        checkJobStatus(job.getStatus(), jobId, JOB_NAME, JobStatus.State.SUCCEEDED, 1.0f);
-        assert jobStatus.getSetupProgress() == 1.0f;
-        assert jobStatus.getMapProgress() == 1.0f;
-        assert jobStatus.getReduceProgress() == 1.0f;
+            jobStatus = job.getStatus();
+            checkJobStatus(job.getStatus(), jobId, JOB_NAME, JobStatus.State.SUCCEEDED, 1.0f);
+            assert jobStatus.getSetupProgress() == 1.0f;
+            assert jobStatus.getMapProgress() == 1.0f;
+            assert jobStatus.getReduceProgress() == 1.0f;
 
-        dumpIgfs(igfs, new IgfsPath(PATH_OUTPUT));
+            dumpIgfs(igfs, new IgfsPath(PATH_OUTPUT));
+        }
+        finally {
+            job.getCluster().close();
+        }
     }
 
     /**
@@ -517,7 +525,12 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
      * Test Hadoop counters.
      */
     public enum TestCounter {
-        COUNTER1, COUNTER2, COUNTER3
+        /** */
+        COUNTER1,
+        /** */
+        COUNTER2,
+        /** */
+        COUNTER3
     }
 
     /**
@@ -535,6 +548,7 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
      * Test combiner that counts invocations.
      */
     public static class TestCountingCombiner extends TestReducer {
+        /** {@inheritDoc} */
         @Override public void reduce(Text key, Iterable<IntWritable> values,
             Context ctx) throws IOException, InterruptedException {
             ctx.getCounter(TestCounter.COUNTER1).increment(1);
@@ -552,6 +566,7 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
      * Test reducer that counts invocations.
      */
     public static class TestCountingReducer extends TestReducer {
+        /** {@inheritDoc} */
         @Override public void reduce(Text key, Iterable<IntWritable> values,
             Context ctx) throws IOException, InterruptedException {
             ctx.getCounter(TestCounter.COUNTER1).increment(1);
@@ -566,6 +581,9 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
         // No-op.
     }
 
+    /**
+     * Test output format.
+     */
     public static class TestOutputFormat<K, V> extends TextOutputFormat<K, V> {
         /** {@inheritDoc} */
         @Override public synchronized OutputCommitter getOutputCommitter(TaskAttemptContext ctx)


[41/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
new file mode 100644
index 0000000..a185485
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Custom.service.js
@@ -0,0 +1,23 @@
+/*
+ * 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.
+ */
+
+// Optional content generation entry point.
+export default class IgniteCustomGenerator {
+    optionalContent(zip, cluster) { // eslint-disable-line no-unused-vars
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
index f9776a2..bcfa2e2 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Docker.service.js
@@ -18,7 +18,7 @@
 /**
  * Docker file generation entry point.
  */
-class GeneratorDocker {
+export default class IgniteDockerGenerator {
     /**
      * Generate from section.
      *
@@ -74,5 +74,3 @@ class GeneratorDocker {
         ].join('\n');
     }
 }
-
-export default ['GeneratorDocker', GeneratorDocker];


[39/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
new file mode 100644
index 0000000..2e01761
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Maven.service.js
@@ -0,0 +1,234 @@
+/*
+ * 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 StringBuilder from './StringBuilder';
+
+// Java built-in class names.
+import POM_DEPENDENCIES from 'app/data/pom-dependencies.json';
+
+/**
+ * Pom file generation entry point.
+ */
+export default class IgniteMavenGenerator {
+    escapeId(s) {
+        if (typeof (s) !== 'string')
+            return s;
+
+        return s.replace(/[^A-Za-z0-9_\-.]+/g, '_');
+    }
+
+    addProperty(sb, tag, val) {
+        sb.append('<' + tag + '>' + val + '</' + tag + '>');
+    }
+
+    addDependency(deps, groupId, artifactId, version, jar) {
+        if (!_.find(deps, (dep) => dep.groupId === groupId && dep.artifactId === artifactId))
+            deps.push({groupId, artifactId, version, jar});
+    }
+
+    addResource(sb, dir, exclude) {
+        sb.startBlock('<resource>');
+        if (dir)
+            this.addProperty(sb, 'directory', dir);
+
+        if (exclude) {
+            sb.startBlock('<excludes>');
+            this.addProperty(sb, 'exclude', exclude);
+            sb.endBlock('</excludes>');
+        }
+
+        sb.endBlock('</resource>');
+    }
+
+    artifact(sb, cluster, version) {
+        this.addProperty(sb, 'groupId', 'org.apache.ignite');
+        this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project');
+        this.addProperty(sb, 'version', version);
+
+        sb.emptyLine();
+    }
+
+    dependencies(sb, cluster, deps) {
+        sb.startBlock('<dependencies>');
+
+        _.forEach(deps, (dep) => {
+            sb.startBlock('<dependency>');
+
+            this.addProperty(sb, 'groupId', dep.groupId);
+            this.addProperty(sb, 'artifactId', dep.artifactId);
+            this.addProperty(sb, 'version', dep.version);
+
+            if (dep.jar) {
+                this.addProperty(sb, 'scope', 'system');
+                this.addProperty(sb, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar);
+            }
+
+            sb.endBlock('</dependency>');
+        });
+
+        sb.endBlock('</dependencies>');
+
+        return sb;
+    }
+
+    build(sb = new StringBuilder(), cluster, excludeGroupIds) {
+        sb.startBlock('<build>');
+        sb.startBlock('<resources>');
+        this.addResource(sb, 'src/main/java', '**/*.java');
+        this.addResource(sb, 'src/main/resources');
+        sb.endBlock('</resources>');
+
+        sb.startBlock('<plugins>');
+        sb.startBlock('<plugin>');
+        this.addProperty(sb, 'artifactId', 'maven-dependency-plugin');
+        sb.startBlock('<executions>');
+        sb.startBlock('<execution>');
+        this.addProperty(sb, 'id', 'copy-libs');
+        this.addProperty(sb, 'phase', 'test-compile');
+        sb.startBlock('<goals>');
+        this.addProperty(sb, 'goal', 'copy-dependencies');
+        sb.endBlock('</goals>');
+        sb.startBlock('<configuration>');
+        this.addProperty(sb, 'excludeGroupIds', excludeGroupIds.join(','));
+        this.addProperty(sb, 'outputDirectory', 'target/libs');
+        this.addProperty(sb, 'includeScope', 'compile');
+        this.addProperty(sb, 'excludeTransitive', 'true');
+        sb.endBlock('</configuration>');
+        sb.endBlock('</execution>');
+        sb.endBlock('</executions>');
+        sb.endBlock('</plugin>');
+        sb.startBlock('<plugin>');
+        this.addProperty(sb, 'artifactId', 'maven-compiler-plugin');
+        this.addProperty(sb, 'version', '3.1');
+        sb.startBlock('<configuration>');
+        this.addProperty(sb, 'source', '1.7');
+        this.addProperty(sb, 'target', '1.7');
+        sb.endBlock('</configuration>');
+        sb.endBlock('</plugin>');
+        sb.endBlock('</plugins>');
+        sb.endBlock('</build>');
+
+        sb.endBlock('</project>');
+    }
+
+    /**
+     * Add dependency for specified store factory if not exist.
+     * @param storeDeps Already added dependencies.
+     * @param storeFactory Store factory to add dependency.
+     */
+    storeFactoryDependency(storeDeps, storeFactory) {
+        if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) {
+            const dep = POM_DEPENDENCIES[storeFactory.dialect];
+
+            this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar);
+        }
+    }
+
+    /**
+     * Generate pom.xml.
+     *
+     * @param cluster Cluster  to take info about dependencies.
+     * @param version Ignite version for Ignite dependencies.
+     * @param sb Resulting output with generated pom.
+     * @returns {string} Generated content.
+     */
+    generate(cluster, version, sb = new StringBuilder()) {
+        const caches = cluster.caches;
+        const deps = [];
+        const storeDeps = [];
+        const excludeGroupIds = ['org.apache.ignite'];
+
+        const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}};
+
+        _.forEach(caches, (cache) => {
+            if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind)
+                this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]);
+
+            if (_.get(cache, 'nodeFilter.kind') === 'Exclude')
+                this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version);
+        });
+
+        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
+
+        sb.emptyLine();
+
+        sb.append(`<!-- ${sb.generatedBy()} -->`);
+
+        sb.emptyLine();
+
+        sb.startBlock('<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">');
+
+        sb.append('<modelVersion>4.0.0</modelVersion>');
+
+        sb.emptyLine();
+
+        this.artifact(sb, cluster, version);
+
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version);
+
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', version);
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version);
+        this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version);
+
+        if (_.get(cluster, 'deploymentSpi.kind') === 'URI')
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-urideploy', version);
+
+        let dep = POM_DEPENDENCIES[cluster.discovery.kind];
+
+        if (dep)
+            this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+
+        if (cluster.discovery.kind === 'Jdbc') {
+            const store = cluster.discovery.Jdbc;
+
+            if (store.dataSourceBean && store.dialect)
+                this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc);
+        }
+
+        _.forEach(cluster.checkpointSpi, (spi) => {
+            if (spi.kind === 'S3') {
+                dep = POM_DEPENDENCIES.S3;
+
+                if (dep)
+                    this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+            }
+            else if (spi.kind === 'JDBC')
+                this.storeFactoryDependency(storeDeps, spi.JDBC);
+        });
+
+        if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled))
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version);
+
+        if (_.find(caches, blobStoreFactory))
+            this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', version);
+
+        if (cluster.logger && cluster.logger.kind) {
+            dep = POM_DEPENDENCIES[cluster.logger.kind];
+
+            if (dep)
+                this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
+        }
+
+        this.dependencies(sb, cluster, deps.concat(storeDeps));
+
+        sb.emptyLine();
+
+        this.build(sb, cluster, excludeGroupIds);
+
+        return sb;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
deleted file mode 100644
index db58532..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/Pom.service.js
+++ /dev/null
@@ -1,233 +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 StringBuilder from './StringBuilder';
-
-// Java built-in class names.
-import POM_DEPENDENCIES from 'app/data/pom-dependencies.json';
-
-/**
- * Pom file generation entry point.
- */
-class GeneratorPom {
-    escapeId(s) {
-        if (typeof (s) !== 'string')
-            return s;
-
-        return s.replace(/[^A-Za-z0-9_\-.]+/g, '_');
-    }
-
-    addProperty(sb, tag, val) {
-        sb.append('<' + tag + '>' + val + '</' + tag + '>');
-    }
-
-    addDependency(deps, groupId, artifactId, version, jar) {
-        if (!_.find(deps, (dep) => dep.groupId === groupId && dep.artifactId === artifactId))
-            deps.push({groupId, artifactId, version, jar});
-    }
-
-    addResource(sb, dir, exclude) {
-        sb.startBlock('<resource>');
-        if (dir)
-            this.addProperty(sb, 'directory', dir);
-
-        if (exclude) {
-            sb.startBlock('<excludes>');
-            this.addProperty(sb, 'exclude', exclude);
-            sb.endBlock('</excludes>');
-        }
-
-        sb.endBlock('</resource>');
-    }
-
-    artifact(sb, cluster, version) {
-        this.addProperty(sb, 'groupId', 'org.apache.ignite');
-        this.addProperty(sb, 'artifactId', this.escapeId(cluster.name) + '-project');
-        this.addProperty(sb, 'version', version);
-
-        sb.emptyLine();
-    }
-
-    dependencies(sb, cluster, deps) {
-        sb.startBlock('<dependencies>');
-
-        _.forEach(deps, (dep) => {
-            sb.startBlock('<dependency>');
-
-            this.addProperty(sb, 'groupId', dep.groupId);
-            this.addProperty(sb, 'artifactId', dep.artifactId);
-            this.addProperty(sb, 'version', dep.version);
-
-            if (dep.jar) {
-                this.addProperty(sb, 'scope', 'system');
-                this.addProperty(sb, 'systemPath', '${project.basedir}/jdbc-drivers/' + dep.jar);
-            }
-
-            sb.endBlock('</dependency>');
-        });
-
-        sb.endBlock('</dependencies>');
-
-        return sb;
-    }
-
-    build(sb = new StringBuilder(), cluster, excludeGroupIds) {
-        sb.startBlock('<build>');
-        sb.startBlock('<resources>');
-        this.addResource(sb, 'src/main/java', '**/*.java');
-        this.addResource(sb, 'src/main/resources');
-        sb.endBlock('</resources>');
-
-        sb.startBlock('<plugins>');
-        sb.startBlock('<plugin>');
-        this.addProperty(sb, 'artifactId', 'maven-dependency-plugin');
-        sb.startBlock('<executions>');
-        sb.startBlock('<execution>');
-        this.addProperty(sb, 'id', 'copy-libs');
-        this.addProperty(sb, 'phase', 'test-compile');
-        sb.startBlock('<goals>');
-        this.addProperty(sb, 'goal', 'copy-dependencies');
-        sb.endBlock('</goals>');
-        sb.startBlock('<configuration>');
-        this.addProperty(sb, 'excludeGroupIds', excludeGroupIds.join(','));
-        this.addProperty(sb, 'outputDirectory', 'target/libs');
-        this.addProperty(sb, 'includeScope', 'compile');
-        this.addProperty(sb, 'excludeTransitive', 'true');
-        sb.endBlock('</configuration>');
-        sb.endBlock('</execution>');
-        sb.endBlock('</executions>');
-        sb.endBlock('</plugin>');
-        sb.startBlock('<plugin>');
-        this.addProperty(sb, 'artifactId', 'maven-compiler-plugin');
-        this.addProperty(sb, 'version', '3.1');
-        sb.startBlock('<configuration>');
-        this.addProperty(sb, 'source', '1.7');
-        this.addProperty(sb, 'target', '1.7');
-        sb.endBlock('</configuration>');
-        sb.endBlock('</plugin>');
-        sb.endBlock('</plugins>');
-        sb.endBlock('</build>');
-
-        sb.endBlock('</project>');
-    }
-
-    /**
-     * Add dependency for specified store factory if not exist.
-     * @param storeDeps Already added dependencies.
-     * @param storeFactory Store factory to add dependency.
-     */
-    storeFactoryDependency(storeDeps, storeFactory) {
-        if (storeFactory.dialect && (!storeFactory.connectVia || storeFactory.connectVia === 'DataSource')) {
-            const dep = POM_DEPENDENCIES[storeFactory.dialect];
-
-            this.addDependency(storeDeps, dep.groupId, dep.artifactId, dep.version, dep.jar);
-        }
-    }
-
-    /**
-     * Generate pom.xml.
-     *
-     * @param cluster Cluster  to take info about dependencies.
-     * @param version Ignite version for Ignite dependencies.
-     * @param sb Resulting output with generated pom.
-     * @returns {string} Generated content.
-     */
-    generate(cluster, version, sb = new StringBuilder()) {
-        const caches = cluster.caches;
-        const deps = [];
-        const storeDeps = [];
-        const excludeGroupIds = ['org.apache.ignite'];
-
-        const blobStoreFactory = {cacheStoreFactory: {kind: 'CacheHibernateBlobStoreFactory'}};
-
-        _.forEach(caches, (cache) => {
-            if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind)
-                this.storeFactoryDependency(storeDeps, cache.cacheStoreFactory[cache.cacheStoreFactory.kind]);
-
-            if (_.get(cache, 'nodeFilter.kind') === 'Exclude')
-                this.addDependency(deps, 'org.apache.ignite', 'ignite-extdata-p2p', version);
-        });
-
-        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
-
-        sb.emptyLine();
-
-        sb.append(`<!-- ${sb.generatedBy()} -->`);
-
-        sb.emptyLine();
-
-        sb.startBlock('<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">');
-
-        sb.append('<modelVersion>4.0.0</modelVersion>');
-
-        sb.emptyLine();
-
-        this.artifact(sb, cluster, version);
-
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-core', version);
-
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-spring', version);
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-indexing', version);
-        this.addDependency(deps, 'org.apache.ignite', 'ignite-rest-http', version);
-
-        let dep = POM_DEPENDENCIES[cluster.discovery.kind];
-
-        if (dep)
-            this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-
-        if (cluster.discovery.kind === 'Jdbc') {
-            const store = cluster.discovery.Jdbc;
-
-            if (store.dataSourceBean && store.dialect)
-                this.storeFactoryDependency(storeDeps, cluster.discovery.Jdbc);
-        }
-
-        _.forEach(cluster.checkpointSpi, (spi) => {
-            if (spi.kind === 'S3') {
-                dep = POM_DEPENDENCIES.S3;
-
-                if (dep)
-                    this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-            }
-            else if (spi.kind === 'JDBC')
-                this.storeFactoryDependency(storeDeps, spi.JDBC);
-        });
-
-        if (_.find(cluster.igfss, (igfs) => igfs.secondaryFileSystemEnabled))
-            this.addDependency(deps, 'org.apache.ignite', 'ignite-hadoop', version);
-
-        if (_.find(caches, blobStoreFactory))
-            this.addDependency(deps, 'org.apache.ignite', 'ignite-hibernate', version);
-
-        if (cluster.logger && cluster.logger.kind) {
-            dep = POM_DEPENDENCIES[cluster.logger.kind];
-
-            if (dep)
-                this.addDependency(deps, 'org.apache.ignite', dep.artifactId, version);
-        }
-
-        this.dependencies(sb, cluster, deps.concat(storeDeps));
-
-        sb.emptyLine();
-
-        this.build(sb, cluster, excludeGroupIds);
-
-        return sb;
-    }
-}
-
-export default ['GeneratorPom', GeneratorPom];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 49b4aa6..8a6a471 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
@@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder';
 /**
  * Properties generation entry point.
  */
-export default class PropertiesGenerator {
+export default class IgnitePropertiesGenerator {
     _collectProperties(bean) {
         const props = [];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
index 7043807..0aa34ee 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Readme.service.js
@@ -20,7 +20,7 @@ import StringBuilder from './StringBuilder';
 /**
  * Properties generation entry point.
  */
-export default class ReadmeGenerator {
+export default class IgniteReadmeGenerator {
     header(sb) {
         sb.append('Content of this folder was generated by Apache Ignite Web Console');
         sb.append('=================================================================');

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
index 19043f6..6e6bffe 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/SharpTransformer.service.js
@@ -19,225 +19,238 @@ import _ from 'lodash';
 import AbstractTransformer from './AbstractTransformer';
 import StringBuilder from './StringBuilder';
 
-export default ['JavaTypes', 'IgnitePlatformGenerator', (JavaTypes, generator) => {
-    return class SharpTransformer extends AbstractTransformer {
-        static generator = generator;
-
-        static commentBlock(sb, ...lines) {
-            _.forEach(lines, (line) => sb.append(`// ${line}`));
-        }
-
-        static doc(sb, ...lines) {
-            sb.append('/// <summary>');
-            _.forEach(lines, (line) => sb.append(`/// ${line}`));
-            sb.append('/// </summary>');
-        }
-
-        static mainComment(sb) {
-            return this.doc(sb, sb.generatedBy());
-        }
-
-        /**
-         *
-         * @param {Array.<String>} sb
-         * @param {Bean} bean
-         */
-        static _defineBean(sb, bean) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            sb.append(`var ${bean.id} = new ${shortClsName}();`);
-        }
-
-        /**
-         * @param {StringBuilder} sb
-         * @param {Bean} parent
-         * @param {Bean} propertyName
-         * @param {String|Bean} value
-         * @private
-         */
-        static _setProperty(sb, parent, propertyName, value) {
-            sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`);
-        }
-
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} parent
-         * @param {String} propertyName
-         * @param {Bean} bean
-         * @private
-         */
-        static _setBeanProperty(sb, parent, propertyName, bean) {
-            sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`);
-        }
-
-        static _toObject(clsName, val) {
-            const items = _.isArray(val) ? val : [val];
-
-            return _.map(items, (item, idx) => {
-                if (_.isNil(item))
-                    return 'null';
-
-                const shortClsName = JavaTypes.shortClassName(clsName);
-
-                switch (shortClsName) {
-                    // case 'byte':
-                    //     return `(byte) ${item}`;
-                    // case 'Serializable':
-                    case 'String':
-                        if (items.length > 1)
-                            return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`;
-
-                        return `"${item}"`;
-                    // case 'Path':
-                    //     return `"${item.replace(/\\/g, '\\\\')}"`;
-                    // case 'Class':
-                    //     return `${this.shortClassName(item)}.class`;
-                    // case 'UUID':
-                    //     return `UUID.fromString("${item}")`;
-                    // case 'PropertyChar':
-                    //     return `props.getProperty("${item}").toCharArray()`;
-                    // case 'Property':
-                    //     return `props.getProperty("${item}")`;
-                    // case 'Bean':
-                    //     if (item.isComplex())
-                    //         return item.id;
+import ConfigurationGenerator from './ConfigurationGenerator';
+
+import ClusterDefaults from './defaults/Cluster.service';
+import CacheDefaults from './defaults/Cache.service';
+import IGFSDefaults from './defaults/IGFS.service';
+
+import JavaTypes from '../../../services/JavaTypes.service';
+
+const generator = new ConfigurationGenerator();
+
+const clusterDflts = new ClusterDefaults();
+const cacheDflts = new CacheDefaults();
+const igfsDflts = new IGFSDefaults();
+
+const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
+
+export default class SharpTransformer extends AbstractTransformer {
+    static generator = generator;
+
+    static commentBlock(sb, ...lines) {
+        _.forEach(lines, (line) => sb.append(`// ${line}`));
+    }
+
+    static doc(sb, ...lines) {
+        sb.append('/// <summary>');
+        _.forEach(lines, (line) => sb.append(`/// ${line}`));
+        sb.append('/// </summary>');
+    }
+
+    static mainComment(sb) {
+        return this.doc(sb, sb.generatedBy());
+    }
+
+    /**
+     *
+     * @param {Array.<String>} sb
+     * @param {Bean} bean
+     */
+    static _defineBean(sb, bean) {
+        const shortClsName = javaTypes.shortClassName(bean.clsName);
+
+        sb.append(`var ${bean.id} = new ${shortClsName}();`);
+    }
+
+    /**
+     * @param {StringBuilder} sb
+     * @param {Bean} parent
+     * @param {Bean} propertyName
+     * @param {String|Bean} value
+     * @private
+     */
+    static _setProperty(sb, parent, propertyName, value) {
+        sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${value};`);
+    }
+
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} parent
+     * @param {String} propertyName
+     * @param {Bean} bean
+     * @private
+     */
+    static _setBeanProperty(sb, parent, propertyName, bean) {
+        sb.append(`${parent.id}.${_.upperFirst(propertyName)} = ${bean.id};`);
+    }
+
+    static _toObject(clsName, val) {
+        const items = _.isArray(val) ? val : [val];
+
+        return _.map(items, (item, idx) => {
+            if (_.isNil(item))
+                return 'null';
+
+            const shortClsName = javaTypes.shortClassName(clsName);
+
+            switch (shortClsName) {
+                // case 'byte':
+                //     return `(byte) ${item}`;
+                // case 'Serializable':
+                case 'String':
+                    if (items.length > 1)
+                        return `"${item}"${idx !== items.length - 1 ? ' +' : ''}`;
+
+                    return `"${item}"`;
+                // case 'Path':
+                //     return `"${item.replace(/\\/g, '\\\\')}"`;
+                // case 'Class':
+                //     return `${this.shortClassName(item)}.class`;
+                // case 'UUID':
+                //     return `UUID.fromString("${item}")`;
+                // case 'PropertyChar':
+                //     return `props.getProperty("${item}").toCharArray()`;
+                // case 'Property':
+                //     return `props.getProperty("${item}")`;
+                // case 'Bean':
+                //     if (item.isComplex())
+                //         return item.id;
+                //
+                //     return this._newBean(item);
+                default:
+                    if (javaTypes.nonEnum(shortClsName))
+                        return item;
+
+                    return `${shortClsName}.${item}`;
+            }
+        });
+    }
+
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @returns {Array}
+     */
+    static _setProperties(sb = new StringBuilder(), bean) {
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'ICollection':
+                    // const implClsName = JavaTypes.shortClassName(prop.implClsName);
+
+                    const colTypeClsName = javaTypes.shortClassName(prop.typeClsName);
+
+                    if (colTypeClsName === 'String') {
+                        const items = this._toObject(colTypeClsName, prop.items);
+
+                        sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`);
+                    }
+                    // else {
+                    //     if (_.includes(vars, prop.id))
+                    //         sb.append(`${prop.id} = new ${implClsName}<>();`);
+                    //     else {
+                    //         vars.push(prop.id);
                     //
-                    //     return this._newBean(item);
-                    default:
-                        if (JavaTypes.nonEnum(shortClsName))
-                            return item;
-
-                        return `${shortClsName}.${item}`;
-                }
-            });
-        }
-
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @returns {Array}
-         */
-        static _setProperties(sb = new StringBuilder(), bean) {
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'ICollection':
-                        // const implClsName = JavaTypes.shortClassName(prop.implClsName);
-
-                        const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName);
-
-                        if (colTypeClsName === 'String') {
-                            const items = this._toObject(colTypeClsName, prop.items);
-
-                            sb.append(`${bean.id}.${_.upperFirst(prop.name)} = new {${items.join(', ')}};`);
-                        }
-                        // else {
-                        //     if (_.includes(vars, prop.id))
-                        //         sb.append(`${prop.id} = new ${implClsName}<>();`);
-                        //     else {
-                        //         vars.push(prop.id);
-                        //
-                        //         sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`);
-                        //     }
-                        //
-                        //     sb.emptyLine();
-                        //
-                        //     if (nonBean) {
-                        //         const items = this._toObject(colTypeClsName, prop.items);
-                        //
-                        //         _.forEach(items, (item) => {
-                        //             sb.append(`${prop.id}.add("${item}");`);
-                        //
-                        //             sb.emptyLine();
-                        //         });
-                        //     }
-                        //     else {
-                        //         _.forEach(prop.items, (item) => {
-                        //             this.constructBean(sb, item, vars, limitLines);
-                        //
-                        //             sb.append(`${prop.id}.add(${item.id});`);
-                        //
-                        //             sb.emptyLine();
-                        //         });
-                        //
-                        //         this._setProperty(sb, bean.id, prop.name, prop.id);
-                        //     }
-                        // }
-
-                        break;
-
-                    case 'Bean':
-                        const nestedBean = prop.value;
-
-                        this._defineBean(sb, nestedBean);
-
-                        sb.emptyLine();
-
-                        this._setProperties(sb, nestedBean);
-
-                        sb.emptyLine();
-
-                        this._setBeanProperty(sb, bean, prop.name, nestedBean);
-
-                        break;
-                    default:
-                        this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value));
-                }
-            });
-
-            return sb;
-        }
-
-        /**
-         * Build Java startup class with configuration.
-         *
-         * @param {Bean} cfg
-         * @param pkg Package name.
-         * @param clsName Class name for generate factory class otherwise generate code snippet.
-         * @param clientNearCfg Optional near cache configuration for client node.
-         * @returns {String}
-         */
-        static toClassFile(cfg, pkg, clsName) {
-            const sb = new StringBuilder();
-
-            sb.startBlock(`namespace ${pkg}`, '{');
-
-            _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`));
-            sb.emptyLine();
-
-
-            this.mainComment(sb);
-            sb.startBlock(`public class ${clsName}`, '{');
-
-            this.doc(sb, 'Configure grid.');
-            sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{');
-
-            this._defineBean(sb, cfg);
-
-            sb.emptyLine();
-
-            this._setProperties(sb, cfg);
+                    //         sb.append(`${clsName}<${colTypeClsName}> ${prop.id} = new ${implClsName}<>();`);
+                    //     }
+                    //
+                    //     sb.emptyLine();
+                    //
+                    //     if (nonBean) {
+                    //         const items = this._toObject(colTypeClsName, prop.items);
+                    //
+                    //         _.forEach(items, (item) => {
+                    //             sb.append(`${prop.id}.add("${item}");`);
+                    //
+                    //             sb.emptyLine();
+                    //         });
+                    //     }
+                    //     else {
+                    //         _.forEach(prop.items, (item) => {
+                    //             this.constructBean(sb, item, vars, limitLines);
+                    //
+                    //             sb.append(`${prop.id}.add(${item.id});`);
+                    //
+                    //             sb.emptyLine();
+                    //         });
+                    //
+                    //         this._setProperty(sb, bean.id, prop.name, prop.id);
+                    //     }
+                    // }
+
+                    break;
+
+                case 'Bean':
+                    const nestedBean = prop.value;
+
+                    this._defineBean(sb, nestedBean);
+
+                    sb.emptyLine();
+
+                    this._setProperties(sb, nestedBean);
+
+                    sb.emptyLine();
+
+                    this._setBeanProperty(sb, bean, prop.name, nestedBean);
+
+                    break;
+                default:
+                    this._setProperty(sb, bean, prop.name, this._toObject(prop.clsName, prop.value));
+            }
+        });
+
+        return sb;
+    }
+
+    /**
+     * Build Java startup class with configuration.
+     *
+     * @param {Bean} cfg
+     * @param pkg Package name.
+     * @param clsName Class name for generate factory class otherwise generate code snippet.
+     * @returns {String}
+     */
+    static toClassFile(cfg, pkg, clsName) {
+        const sb = new StringBuilder();
+
+        sb.startBlock(`namespace ${pkg}`, '{');
+
+        _.forEach(_.sortBy(cfg.collectClasses()), (cls) => sb.append(`using ${cls};`));
+        sb.emptyLine();
+
+
+        this.mainComment(sb);
+        sb.startBlock(`public class ${clsName}`, '{');
+
+        this.doc(sb, 'Configure grid.');
+        sb.startBlock('public static IgniteConfiguration CreateConfiguration()', '{');
+
+        this._defineBean(sb, cfg);
+
+        sb.emptyLine();
+
+        this._setProperties(sb, cfg);
 
-            sb.emptyLine();
+        sb.emptyLine();
 
-            sb.append(`return ${cfg.id};`);
+        sb.append(`return ${cfg.id};`);
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            sb.endBlock('}');
+        sb.endBlock('}');
 
-            return sb.asString();
-        }
+        return sb.asString();
+    }
 
-        static generateSection(bean) {
-            const sb = new StringBuilder();
+    static generateSection(bean) {
+        const sb = new StringBuilder();
 
-            this._setProperties(sb, bean);
+        this._setProperties(sb, bean);
 
-            return sb.asString();
-        }
-    };
-}];
+        return sb.asString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 73df25e..b234575 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,314 +20,311 @@ 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;
-
-        static commentBlock(sb, ...lines) {
-            if (lines.length > 1) {
-                sb.append('<!--');
-
-                _.forEach(lines, (line) => sb.append(`  ${line}`));
-
-                sb.append('-->');
-            }
-            else
-                sb.append(`<!-- ${_.head(lines)} -->`);
+export default class IgniteSpringTransformer extends AbstractTransformer {
+    static escapeXml(str) {
+        return str.replace(/&/g, '&amp;')
+            .replace(/"/g, '&quot;')
+            .replace(/'/g, '&apos;')
+            .replace(/>/g, '&gt;')
+            .replace(/</g, '&lt;');
+    }
+
+    static commentBlock(sb, ...lines) {
+        if (lines.length > 1) {
+            sb.append('<!--');
+
+            _.forEach(lines, (line) => sb.append(`  ${line}`));
+
+            sb.append('-->');
         }
+        else
+            sb.append(`<!-- ${_.head(lines)} -->`);
+    }
 
-        static appendBean(sb, bean, appendId) {
-            const beanTags = [];
-
-            if (appendId)
-                beanTags.push(`id="${bean.id}"`);
-
-            beanTags.push(`class="${bean.clsName}"`);
-
-            if (bean.factoryMtd)
-                beanTags.push(`factory-method="${bean.factoryMtd}"`);
-
-            sb.startBlock(`<bean ${beanTags.join(' ')}>`);
-
-            _.forEach(bean.arguments, (arg) => {
-                if (arg.clsName === 'MAP') {
-                    sb.startBlock('<constructor-arg>');
-                    this._constructMap(sb, arg);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (_.isNil(arg.value)) {
-                    sb.startBlock('<constructor-arg>');
-                    sb.append('<null/>');
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (arg.constant) {
-                    sb.startBlock('<constructor-arg>');
-                    sb.append(`<util:constant static-field="${arg.clsName}.${arg.value}"/>`);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else if (arg.clsName === 'BEAN') {
-                    sb.startBlock('<constructor-arg>');
-                    this.appendBean(sb, arg.value);
-                    sb.endBlock('</constructor-arg>');
-                }
-                else
-                    sb.append(`<constructor-arg value="${this._toObject(arg.clsName, arg.value)}"/>`);
-            });
+    static appendBean(sb, bean, appendId) {
+        const beanTags = [];
 
-            this._setProperties(sb, bean);
+        if (appendId)
+            beanTags.push(`id="${bean.id}"`);
 
-            sb.endBlock('</bean>');
-        }
+        beanTags.push(`class="${bean.clsName}"`);
 
-        static _toObject(clsName, items) {
-            return _.map(_.isArray(items) ? items : [items], (item) => {
-                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;
-                }
-            });
-        }
+        if (bean.factoryMtd)
+            beanTags.push(`factory-method="${bean.factoryMtd}"`);
 
-        static _isBean(clsName) {
-            return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.');
-        }
+        sb.startBlock(`<bean ${beanTags.join(' ')}>`);
 
-        static _setCollection(sb, prop) {
-            sb.startBlock(`<property name="${prop.name}">`);
-            sb.startBlock('<list>');
+        _.forEach(bean.arguments, (arg) => {
+            if (arg.clsName === 'MAP') {
+                sb.startBlock('<constructor-arg>');
+                this._constructMap(sb, arg);
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (_.isNil(arg.value)) {
+                sb.startBlock('<constructor-arg>');
+                sb.append('<null/>');
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (arg.constant) {
+                sb.startBlock('<constructor-arg>');
+                sb.append(`<util:constant static-field="${arg.clsName}.${arg.value}"/>`);
+                sb.endBlock('</constructor-arg>');
+            }
+            else if (arg.clsName === 'BEAN') {
+                sb.startBlock('<constructor-arg>');
+                this.appendBean(sb, arg.value);
+                sb.endBlock('</constructor-arg>');
+            }
+            else
+                sb.append(`<constructor-arg value="${this._toObject(arg.clsName, arg.value)}"/>`);
+        });
+
+        this._setProperties(sb, bean);
+
+        sb.endBlock('</bean>');
+    }
+
+    static _toObject(clsName, items) {
+        return _.map(_.isArray(items) ? items : [items], (item) => {
+            switch (clsName) {
+                case 'PROPERTY':
+                case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
+                    return `\${${item}}`;
+                case 'java.lang.Class':
+                    return this.javaTypes.fullClassName(item);
+                case 'long':
+                    return `${item}L`;
+                case 'java.lang.String':
+                case 'PATH':
+                    return this.escapeXml(item);
+                default:
+                    return item;
+            }
+        });
+    }
 
-            _.forEach(prop.items, (item, idx) => {
-                if (this._isBean(prop.typeClsName)) {
-                    if (idx !== 0)
-                        sb.emptyLine();
+    static _isBean(clsName) {
+        return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+    }
 
-                    this.appendBean(sb, item);
-                }
-                else
-                    sb.append(`<value>${item}</value>`);
-            });
+    static _setCollection(sb, prop) {
+        sb.startBlock(`<property name="${prop.name}">`);
+        sb.startBlock('<list>');
 
-            sb.endBlock('</list>');
-            sb.endBlock('</property>');
-        }
+        _.forEach(prop.items, (item, idx) => {
+            if (this._isBean(prop.typeClsName)) {
+                if (idx !== 0)
+                    sb.emptyLine();
 
-        static _constructMap(sb, map) {
-            sb.startBlock('<map>');
+                this.appendBean(sb, item);
+            }
+            else
+                sb.append(`<value>${item}</value>`);
+        });
 
-            _.forEach(map.entries, (entry) => {
-                const key = entry[map.keyField];
-                const val = entry[map.valField];
+        sb.endBlock('</list>');
+        sb.endBlock('</property>');
+    }
 
-                const isKeyBean = this._isBean(map.keyClsName);
-                const isValBean = this._isBean(map.valClsName);
+    static _constructMap(sb, map) {
+        sb.startBlock('<map>');
 
+        _.forEach(map.entries, (entry) => {
+            const key = entry[map.keyField];
+            const val = entry[map.valField];
 
-                if (isKeyBean || isValBean) {
-                    sb.startBlock('<entry>');
+            const isKeyBean = this._isBean(map.keyClsName);
+            const isValBean = this._isBean(map.valClsName);
 
-                    sb.startBlock('<key>');
-                    if (isKeyBean)
-                        this.appendBean(sb, key);
-                    else
-                        sb.append(this._toObject(map.keyClsName, key));
-                    sb.endBlock('</key>');
 
-                    sb.startBlock('<value>');
-                    if (isValBean)
-                        this.appendBean(sb, val);
-                    else
-                        sb.append(this._toObject(map.valClsName, val));
-                    sb.endBlock('</value>');
+            if (isKeyBean || isValBean) {
+                sb.startBlock('<entry>');
 
-                    sb.endBlock('</entry>');
-                }
+                sb.startBlock('<key>');
+                if (isKeyBean)
+                    this.appendBean(sb, key);
                 else
-                    sb.append(`<entry key="${this._toObject(map.keyClsName, key)}" value="${this._toObject(map.valClsName, val)}"/>`);
-            });
+                    sb.append(this._toObject(map.keyClsName, key));
+                sb.endBlock('</key>');
 
-            sb.endBlock('</map>');
-        }
+                sb.startBlock('<value>');
+                if (isValBean)
+                    this.appendBean(sb, val);
+                else
+                    sb.append(this._toObject(map.valClsName, val));
+                sb.endBlock('</value>');
 
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @returns {StringBuilder}
-         */
-        static _setProperties(sb, bean) {
-            _.forEach(bean.properties, (prop, idx) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        const valAttr = prop.name === 'dataSource' ? 'ref' : 'value';
+                sb.endBlock('</entry>');
+            }
+            else
+                sb.append(`<entry key="${this._toObject(map.keyClsName, key)}" value="${this._toObject(map.valClsName, val)}"/>`);
+        });
 
-                        sb.append(`<property name="${prop.name}" ${valAttr}="${prop.id}"/>`);
+        sb.endBlock('</map>');
+    }
 
-                        break;
-                    case 'EVENT_TYPES':
-                        sb.startBlock(`<property name="${prop.name}">`);
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @returns {StringBuilder}
+     */
+    static _setProperties(sb, bean) {
+        _.forEach(bean.properties, (prop, idx) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    const valAttr = prop.name === 'dataSource' ? 'ref' : 'value';
 
-                        if (prop.eventTypes.length === 1) {
-                            const evtGrp = _.find(eventGroups, {value: _.head(prop.eventTypes)});
+                    sb.append(`<property name="${prop.name}" ${valAttr}="${prop.id}"/>`);
 
-                            evtGrp && sb.append(`<util:constant static-field="${evtGrp.class}.${evtGrp.value}"/>`);
-                        }
-                        else {
-                            sb.startBlock('<list>');
+                    break;
+                case 'EVENT_TYPES':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                            _.forEach(prop.eventTypes, (item, ix) => {
-                                ix > 0 && sb.emptyLine();
+                    if (prop.eventTypes.length === 1) {
+                        const evtGrp = _.find(this.eventGroups, {value: _.head(prop.eventTypes)});
 
-                                const evtGrp = _.find(eventGroups, {value: item});
+                        evtGrp && sb.append(`<util:constant static-field="${evtGrp.class}.${evtGrp.value}"/>`);
+                    }
+                    else {
+                        sb.startBlock('<list>');
 
-                                if (evtGrp) {
-                                    sb.append(`<!-- EventType.${item} -->`);
+                        _.forEach(prop.eventTypes, (item, ix) => {
+                            ix > 0 && sb.emptyLine();
 
-                                    _.forEach(evtGrp.events, (event) =>
-                                        sb.append(`<util:constant static-field="${evtGrp.class}.${event}"/>`));
-                                }
-                            });
+                            const evtGrp = _.find(this.eventGroups, {value: item});
 
-                            sb.endBlock('</list>');
-                        }
+                            if (evtGrp) {
+                                sb.append(`<!-- EventType.${item} -->`);
 
-                        sb.endBlock('</property>');
+                                _.forEach(evtGrp.events, (event) =>
+                                    sb.append(`<util:constant static-field="${evtGrp.class}.${event}"/>`));
+                            }
+                        });
 
-                        break;
-                    case 'ARRAY':
-                    case 'COLLECTION':
-                        this._setCollection(sb, prop);
+                        sb.endBlock('</list>');
+                    }
 
-                        break;
-                    case 'MAP':
-                        sb.startBlock(`<property name="${prop.name}">`);
+                    sb.endBlock('</property>');
 
-                        this._constructMap(sb, prop);
+                    break;
+                case 'ARRAY':
+                case 'COLLECTION':
+                    this._setCollection(sb, prop);
 
-                        sb.endBlock('</property>');
+                    break;
+                case 'MAP':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                        break;
-                    case 'java.util.Properties':
-                        sb.startBlock(`<property name="${prop.name}">`);
-                        sb.startBlock('<props>');
+                    this._constructMap(sb, prop);
 
-                        _.forEach(prop.entries, (entry) => {
-                            sb.append(`<prop key="${entry.name}">${entry.value}</prop>`);
-                        });
+                    sb.endBlock('</property>');
 
-                        sb.endBlock('</props>');
-                        sb.endBlock('</property>');
+                    break;
+                case 'java.util.Properties':
+                    sb.startBlock(`<property name="${prop.name}">`);
+                    sb.startBlock('<props>');
 
-                        break;
-                    case 'BEAN':
-                        sb.startBlock(`<property name="${prop.name}">`);
+                    _.forEach(prop.entries, (entry) => {
+                        sb.append(`<prop key="${entry.name}">${entry.value}</prop>`);
+                    });
 
-                        this.appendBean(sb, prop.value);
+                    sb.endBlock('</props>');
+                    sb.endBlock('</property>');
 
-                        sb.endBlock('</property>');
+                    break;
+                case 'BEAN':
+                    sb.startBlock(`<property name="${prop.name}">`);
 
-                        break;
-                    default:
-                        sb.append(`<property name="${prop.name}" value="${this._toObject(prop.clsName, prop.value)}"/>`);
-                }
+                    this.appendBean(sb, prop.value);
 
-                this._emptyLineIfNeeded(sb, bean.properties, idx);
-            });
+                    sb.endBlock('</property>');
 
-            return sb;
-        }
+                    break;
+                default:
+                    sb.append(`<property name="${prop.name}" value="${this._toObject(prop.clsName, prop.value)}"/>`);
+            }
 
-        /**
-         * Build final XML.
-         *
-         * @param {Bean} cfg Ignite configuration.
-         * @param {Boolean} clientNearCaches
-         * @returns {StringBuilder}
-         */
-        static igniteConfiguration(cfg, clientNearCaches) {
-            const sb = new StringBuilder();
-
-            // 0. Add header.
-            sb.append('<?xml version="1.0" encoding="UTF-8"?>');
-            sb.emptyLine();
+            this._emptyLineIfNeeded(sb, bean.properties, idx);
+        });
+
+        return sb;
+    }
+
+    /**
+     * Build final XML.
+     *
+     * @param {Bean} cfg Ignite configuration.
+     * @param {Boolean} clientNearCaches
+     * @returns {StringBuilder}
+     */
+    static igniteConfiguration(cfg, clientNearCaches) {
+        const sb = new StringBuilder();
+
+        // 0. Add header.
+        sb.append('<?xml version="1.0" encoding="UTF-8"?>');
+        sb.emptyLine();
+
+        this.mainComment(sb);
+        sb.emptyLine();
+
+        // 1. Start beans section.
+        sb.startBlock([
+            '<beans xmlns="http://www.springframework.org/schema/beans"',
+            '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"',
+            '       xmlns:util="http://www.springframework.org/schema/util"',
+            '       xsi:schemaLocation="http://www.springframework.org/schema/beans',
+            '                           http://www.springframework.org/schema/beans/spring-beans.xsd',
+            '                           http://www.springframework.org/schema/util',
+            '                           http://www.springframework.org/schema/util/spring-util.xsd">']);
+
+        // 2. Add external property file
+        if (this.hasProperties(cfg)) {
+            this.commentBlock(sb, 'Load external properties file.');
+
+            sb.startBlock('<bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">');
+            sb.append('<property name="location" value="classpath:secret.properties"/>');
+            sb.endBlock('</bean>');
 
-            this.mainComment(sb);
             sb.emptyLine();
+        }
 
-            // 1. Start beans section.
-            sb.startBlock([
-                '<beans xmlns="http://www.springframework.org/schema/beans"',
-                '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"',
-                '       xmlns:util="http://www.springframework.org/schema/util"',
-                '       xsi:schemaLocation="http://www.springframework.org/schema/beans',
-                '                           http://www.springframework.org/schema/beans/spring-beans.xsd',
-                '                           http://www.springframework.org/schema/util',
-                '                           http://www.springframework.org/schema/util/spring-util.xsd">']);
+        // 3. Add data sources.
+        const dataSources = this.collectDataSources(cfg);
 
-            // 2. Add external property file
-            if (this.hasProperties(cfg)) {
-                this.commentBlock(sb, 'Load external properties file.');
+        if (dataSources.length) {
+            this.commentBlock(sb, 'Data source beans will be initialized from external properties file.');
 
-                sb.startBlock('<bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">');
-                sb.append('<property name="location" value="classpath:secret.properties"/>');
-                sb.endBlock('</bean>');
+            _.forEach(dataSources, (ds) => {
+                this.appendBean(sb, ds, true);
 
                 sb.emptyLine();
-            }
-
-            // 3. Add data sources.
-            const dataSources = this.collectDataSources(cfg);
-
-            if (dataSources.length) {
-                this.commentBlock(sb, 'Data source beans will be initialized from external properties file.');
-
-                _.forEach(dataSources, (ds) => {
-                    this.appendBean(sb, ds, true);
-
-                    sb.emptyLine();
-                });
-            }
+            });
+        }
 
-            _.forEach(clientNearCaches, (cache) => {
-                this.commentBlock(sb, 'Configuration of near cache for cache "' + cache.name + '"');
+        _.forEach(clientNearCaches, (cache) => {
+            this.commentBlock(sb, `Configuration of near cache for cache "${cache.name}"`);
 
-                this.appendBean(sb, generator.cacheNearClient(cache), true);
+            this.appendBean(sb, this.generator.cacheNearClient(cache), true);
 
-                sb.emptyLine();
-            });
+            sb.emptyLine();
+        });
 
-            // 3. Add main content.
-            this.appendBean(sb, cfg);
+        // 3. Add main content.
+        this.appendBean(sb, cfg);
 
-            // 4. Close beans section.
-            sb.endBlock('</beans>');
+        // 4. Close beans section.
+        sb.endBlock('</beans>');
 
-            return sb;
-        }
+        return sb;
+    }
 
-        static cluster(cluster, client) {
-            const cfg = generator.igniteConfiguration(cluster, client);
+    static cluster(cluster, client) {
+        const cfg = this.generator.igniteConfiguration(cluster, client);
 
-            const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
+        const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
 
-            return this.igniteConfiguration(cfg, clientNearCaches);
-        }
-    };
-}];
+        return this.igniteConfiguration(cfg, clientNearCaches);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
new file mode 100644
index 0000000..eeac3a0
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.platform.service.js
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import _ from 'lodash';
+
+const enumValueMapper = (val) => _.capitalize(val);
+
+const DFLT_CACHE = {
+    cacheMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
+        mapper: enumValueMapper
+    },
+    atomicityMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheAtomicityMode',
+        mapper: enumValueMapper
+    },
+    memoryMode: {
+        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMemoryMode',
+        value: 'ONHEAP_TIERED',
+        mapper: enumValueMapper
+    },
+    atomicWriteOrderMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode',
+        mapper: enumValueMapper
+    },
+    writeSynchronizationMode: {
+        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
+        value: 'PRIMARY_SYNC',
+        mapper: enumValueMapper
+    },
+    rebalanceMode: {
+        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
+        value: 'ASYNC',
+        mapper: enumValueMapper
+    }
+};
+
+export default class IgniteCachePlatformDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CACHE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
new file mode 100644
index 0000000..14b315f
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cache.service.js
@@ -0,0 +1,131 @@
+/*
+ * 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.
+ */
+
+const DFLT_CACHE = {
+    cacheMode: {
+        clsName: 'org.apache.ignite.cache.CacheMode'
+    },
+    atomicityMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicityMode'
+    },
+    memoryMode: {
+        clsName: 'org.apache.ignite.cache.CacheMemoryMode',
+        value: 'ONHEAP_TIERED'
+    },
+    offHeapMaxMemory: -1,
+    startSize: 1500000,
+    swapEnabled: false,
+    sqlOnheapRowCacheSize: 10240,
+    longQueryWarningTimeout: 3000,
+    snapshotableIndex: false,
+    sqlEscapeAll: false,
+    storeKeepBinary: false,
+    loadPreviousValue: false,
+    cacheStoreFactory: {
+        CacheJdbcPojoStoreFactory: {
+            batchSize: 512,
+            maximumWriteAttempts: 2,
+            parallelLoadCacheMinimumThreshold: 512,
+            sqlEscapeAll: false
+        }
+    },
+    readThrough: false,
+    writeThrough: false,
+    writeBehindEnabled: false,
+    writeBehindBatchSize: 512,
+    writeBehindFlushSize: 10240,
+    writeBehindFlushFrequency: 5000,
+    writeBehindFlushThreadCount: 1,
+    maxConcurrentAsyncOperations: 500,
+    defaultLockTimeout: 0,
+    atomicWriteOrderMode: {
+        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'
+    },
+    writeSynchronizationMode: {
+        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
+        value: 'PRIMARY_SYNC'
+    },
+    rebalanceMode: {
+        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
+        value: 'ASYNC'
+    },
+    rebalanceThreadPoolSize: 1,
+    rebalanceBatchSize: 524288,
+    rebalanceBatchesPrefetchCount: 2,
+    rebalanceOrder: 0,
+    rebalanceDelay: 0,
+    rebalanceTimeout: 10000,
+    rebalanceThrottle: 0,
+    statisticsEnabled: false,
+    managementEnabled: false,
+    nearConfiguration: {
+        nearStartSize: 375000
+    },
+    clientNearConfiguration: {
+        nearStartSize: 375000
+    },
+    evictionPolicy: {
+        LRU: {
+            batchSize: 1,
+            maxSize: 100000
+        },
+        FIFO: {
+            batchSize: 1,
+            maxSize: 100000
+        },
+        SORTED: {
+            batchSize: 1,
+            maxSize: 100000
+        }
+    },
+    queryMetadata: 'Configuration',
+    fields: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        valField: 'className',
+        entries: []
+    },
+    aliases: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        keyField: 'field',
+        valField: 'alias',
+        entries: []
+    },
+    indexes: {
+        indexType: {
+            clsName: 'org.apache.ignite.cache.QueryIndexType'
+        },
+        fields: {
+            keyClsName: 'java.lang.String',
+            valClsName: 'java.lang.Boolean',
+            valField: 'direction',
+            entries: []
+        }
+    },
+    typeField: {
+        databaseFieldType: {
+            clsName: 'java.sql.Types'
+        }
+    }
+};
+
+export default class IgniteCacheDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CACHE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
new file mode 100644
index 0000000..b701951
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.platform.service.js
@@ -0,0 +1,43 @@
+/*
+ * 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.
+ */
+
+const enumValueMapper = (val) => _.capitalize(val);
+
+const DFLT_CLUSTER = {
+    atomics: {
+        cacheMode: {
+            clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
+            mapper: enumValueMapper
+        }
+    },
+    transactionConfiguration: {
+        defaultTxConcurrency: {
+            clsName: 'Apache.Ignite.Core.Transactions.TransactionConcurrency',
+            mapper: enumValueMapper
+        },
+        defaultTxIsolation: {
+            clsName: 'Apache.Ignite.Core.Transactions.TransactionIsolation',
+            mapper: enumValueMapper
+        }
+    }
+};
+
+export default class IgniteClusterPlatformDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CLUSTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
new file mode 100644
index 0000000..6333ef9
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Cluster.service.js
@@ -0,0 +1,289 @@
+/*
+ * 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.
+ */
+
+const DFLT_CLUSTER = {
+    localHost: '0.0.0.0',
+    discovery: {
+        localPort: 47500,
+        localPortRange: 100,
+        socketTimeout: 5000,
+        ackTimeout: 5000,
+        maxAckTimeout: 600000,
+        networkTimeout: 5000,
+        joinTimeout: 0,
+        threadPriority: 10,
+        heartbeatFrequency: 2000,
+        maxMissedHeartbeats: 1,
+        maxMissedClientHeartbeats: 5,
+        topHistorySize: 1000,
+        reconnectCount: 10,
+        statisticsPrintFrequency: 0,
+        ipFinderCleanFrequency: 60000,
+        forceServerMode: false,
+        clientReconnectDisabled: false,
+        Multicast: {
+            multicastGroup: '228.1.2.4',
+            multicastPort: 47400,
+            responseWaitTime: 500,
+            addressRequestAttempts: 2,
+            localAddress: '0.0.0.0'
+        },
+        Jdbc: {
+            initSchema: false
+        },
+        SharedFs: {
+            path: 'disco/tcp'
+        },
+        ZooKeeper: {
+            basePath: '/services',
+            serviceName: 'ignite',
+            allowDuplicateRegistrations: false,
+            ExponentialBackoff: {
+                baseSleepTimeMs: 1000,
+                maxRetries: 10
+            },
+            BoundedExponentialBackoffRetry: {
+                baseSleepTimeMs: 1000,
+                maxSleepTimeMs: 2147483647,
+                maxRetries: 10
+            },
+            UntilElapsed: {
+                maxElapsedTimeMs: 60000,
+                sleepMsBetweenRetries: 1000
+            },
+            RetryNTimes: {
+                n: 10,
+                sleepMsBetweenRetries: 1000
+            },
+            OneTime: {
+                sleepMsBetweenRetry: 1000
+            },
+            Forever: {
+                retryIntervalMs: 1000
+            }
+        }
+    },
+    atomics: {
+        atomicSequenceReserveSize: 1000,
+        backups: 0,
+        cacheMode: {
+            clsName: 'org.apache.ignite.cache.CacheMode',
+            value: 'PARTITIONED'
+        }
+    },
+    binary: {
+        compactFooter: true,
+        typeConfigurations: {
+            enum: false
+        }
+    },
+    collision: {
+        kind: null,
+        JobStealing: {
+            activeJobsThreshold: 95,
+            waitJobsThreshold: 0,
+            messageExpireTime: 1000,
+            maximumStealingAttempts: 5,
+            stealingEnabled: true,
+            stealingAttributes: {
+                keyClsName: 'java.lang.String',
+                valClsName: 'java.io.Serializable',
+                items: []
+            }
+        },
+        PriorityQueue: {
+            priorityAttributeKey: 'grid.task.priority',
+            jobPriorityAttributeKey: 'grid.job.priority',
+            defaultPriority: 0,
+            starvationIncrement: 1,
+            starvationPreventionEnabled: true
+        }
+    },
+    communication: {
+        localPort: 47100,
+        localPortRange: 100,
+        sharedMemoryPort: 48100,
+        directBuffer: false,
+        directSendBuffer: false,
+        idleConnectionTimeout: 30000,
+        connectTimeout: 5000,
+        maxConnectTimeout: 600000,
+        reconnectCount: 10,
+        socketSendBuffer: 32768,
+        socketReceiveBuffer: 32768,
+        messageQueueLimit: 1024,
+        tcpNoDelay: true,
+        ackSendThreshold: 16,
+        unacknowledgedMessagesBufferSize: 0,
+        socketWriteTimeout: 2000
+    },
+    networkTimeout: 5000,
+    networkSendRetryDelay: 1000,
+    networkSendRetryCount: 3,
+    discoveryStartupDelay: 60000,
+    connector: {
+        port: 11211,
+        portRange: 100,
+        idleTimeout: 7000,
+        idleQueryCursorTimeout: 600000,
+        idleQueryCursorCheckFrequency: 60000,
+        receiveBufferSize: 32768,
+        sendBufferSize: 32768,
+        sendQueueLimit: 0,
+        directBuffer: false,
+        noDelay: true,
+        sslEnabled: false,
+        sslClientAuth: false
+    },
+    deploymentMode: {
+        clsName: 'org.apache.ignite.configuration.DeploymentMode',
+        value: 'SHARED'
+    },
+    peerClassLoadingEnabled: false,
+    peerClassLoadingMissedResourcesCacheSize: 100,
+    peerClassLoadingThreadPoolSize: 2,
+    failoverSpi: {
+        JobStealing: {
+            maximumFailoverAttempts: 5
+        },
+        Always: {
+            maximumFailoverAttempts: 5
+        }
+    },
+    logger: {
+        Log4j: {
+            level: {
+                clsName: 'org.apache.log4j.Level'
+            }
+        },
+        Log4j2: {
+            level: {
+                clsName: 'org.apache.logging.log4j.Level'
+            }
+        }
+    },
+    marshalLocalJobs: false,
+    marshallerCacheKeepAliveTime: 10000,
+    metricsHistorySize: 10000,
+    metricsLogFrequency: 60000,
+    metricsUpdateFrequency: 2000,
+    clockSyncSamples: 8,
+    clockSyncFrequency: 120000,
+    timeServerPortBase: 31100,
+    timeServerPortRange: 100,
+    transactionConfiguration: {
+        defaultTxConcurrency: {
+            clsName: 'org.apache.ignite.transactions.TransactionConcurrency',
+            value: 'PESSIMISTIC'
+        },
+        defaultTxIsolation: {
+            clsName: 'org.apache.ignite.transactions.TransactionIsolation',
+            value: 'REPEATABLE_READ'
+        },
+        defaultTxTimeout: 0,
+        pessimisticTxLogLinger: 10000
+    },
+    attributes: {
+        keyClsName: 'java.lang.String',
+        valClsName: 'java.lang.String',
+        items: []
+    },
+    odbcConfiguration: {
+        endpointAddress: '0.0.0.0:10800..10810',
+        maxOpenCursors: 128
+    },
+    eventStorage: {
+        Memory: {
+            expireCount: 10000
+        }
+    },
+    checkpointSpi: {
+        S3: {
+            bucketNameSuffix: 'default-bucket',
+            clientConfiguration: {
+                protocol: {
+                    clsName: 'com.amazonaws.Protocol',
+                    value: 'HTTPS'
+                },
+                maxConnections: 50,
+                retryPolicy: {
+                    retryCondition: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    },
+                    backoffStrategy: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    },
+                    maxErrorRetry: {
+                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
+                    }
+                },
+                maxErrorRetry: -1,
+                socketTimeout: 50000,
+                connectionTimeout: 50000,
+                requestTimeout: 0,
+                socketSendBufferSizeHints: 0,
+                connectionTTL: -1,
+                connectionMaxIdleMillis: 60000,
+                responseMetadataCacheSize: 50,
+                useReaper: true,
+                useGzip: false,
+                preemptiveBasicProxyAuth: false,
+                useTcpKeepAlive: false
+            }
+        },
+        JDBC: {
+            checkpointTableName: 'CHECKPOINTS',
+            keyFieldName: 'NAME',
+            keyFieldType: 'VARCHAR',
+            valueFieldName: 'VALUE',
+            valueFieldType: 'BLOB',
+            expireDateFieldName: 'EXPIRE_DATE',
+            expireDateFieldType: 'DATETIME',
+            numberOfRetries: 2
+        }
+    },
+    loadBalancingSpi: {
+        RoundRobin: {
+            perTask: false
+        },
+        Adaptive: {
+            loadProbe: {
+                Job: {
+                    useAverage: true
+                },
+                CPU: {
+                    useAverage: true,
+                    useProcessors: true,
+                    processorCoefficient: 1
+                },
+                ProcessingTime: {
+                    useAverage: true
+                }
+            }
+        },
+        WeightedRandom: {
+            nodeWeight: 10,
+            useWeights: false
+        }
+    }
+};
+
+export default class IgniteClusterDefaults {
+    constructor() {
+        Object.assign(this, DFLT_CLUSTER);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
new file mode 100644
index 0000000..315da1f
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/Event-groups.service.js
@@ -0,0 +1,27 @@
+/*
+ * 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 _ from 'lodash';
+
+// Events groups.
+import EVENT_GROUPS from 'app/data/event-groups.json';
+
+export default class IgniteEventGroups {
+    constructor() {
+        return _.clone(EVENT_GROUPS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
new file mode 100644
index 0000000..985a56e
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/configuration/generator/defaults/IGFS.service.js
@@ -0,0 +1,64 @@
+/*
+ * 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.
+ */
+
+const DFLT_IGFS = {
+    defaultMode: {
+        clsName: 'org.apache.ignite.igfs.IgfsMode',
+        value: 'DUAL_ASYNC'
+    },
+    secondaryFileSystem: {
+
+    },
+    ipcEndpointConfiguration: {
+        type: {
+            clsName: 'org.apache.ignite.igfs.IgfsIpcEndpointType'
+        },
+        host: '127.0.0.1',
+        port: 10500,
+        memorySize: 262144,
+        tokenDirectoryPath: 'ipc/shmem'
+    },
+    fragmentizerConcurrentFiles: 0,
+    fragmentizerThrottlingBlockLength: 16777216,
+    fragmentizerThrottlingDelay: 200,
+    dualModeMaxPendingPutsSize: 0,
+    dualModePutExecutorServiceShutdown: false,
+    blockSize: 65536,
+    streamBufferSize: 65536,
+    maxSpaceSize: 0,
+    maximumTaskRangeLength: 0,
+    managementPort: 11400,
+    perNodeBatchSize: 100,
+    perNodeParallelBatchCount: 8,
+    prefetchBlocks: 0,
+    sequentialReadsBeforePrefetch: 0,
+    trashPurgeTimeout: 1000,
+    colocateMetadata: true,
+    relaxedConsistency: true,
+    pathModes: {
+        keyClsName: 'java.lang.String',
+        keyField: 'path',
+        valClsName: 'org.apache.ignite.igfs.IgfsMode',
+        valField: 'mode'
+    }
+};
+
+export default class IgniteIGFSDefaults {
+    constructor() {
+        Object.assign(this, DFLT_IGFS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
deleted file mode 100644
index f06e11b..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.platform.provider.js
+++ /dev/null
@@ -1,60 +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 _ from 'lodash';
-
-const enumValueMapper = (val) => _.capitalize(val);
-
-const DFLT_CACHE = {
-    cacheMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
-        mapper: enumValueMapper
-    },
-    atomicityMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheAtomicityMode',
-        mapper: enumValueMapper
-    },
-    memoryMode: {
-        clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMemoryMode',
-        value: 'ONHEAP_TIERED',
-        mapper: enumValueMapper
-    },
-    atomicWriteOrderMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode',
-        mapper: enumValueMapper
-    },
-    writeSynchronizationMode: {
-        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
-        value: 'PRIMARY_SYNC',
-        mapper: enumValueMapper
-    },
-    rebalanceMode: {
-        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
-        value: 'ASYNC',
-        mapper: enumValueMapper
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CACHE, dflts);
-    };
-
-    this.$get = ['igniteCacheDefaults', (cacheDefaults) => {
-        return _.merge({}, cacheDefaults, DFLT_CACHE);
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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
deleted file mode 100644
index f50e493..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cache.provider.js
+++ /dev/null
@@ -1,137 +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 _ from 'lodash';
-
-const DFLT_CACHE = {
-    cacheMode: {
-        clsName: 'org.apache.ignite.cache.CacheMode'
-    },
-    atomicityMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicityMode'
-    },
-    memoryMode: {
-        clsName: 'org.apache.ignite.cache.CacheMemoryMode',
-        value: 'ONHEAP_TIERED'
-    },
-    offHeapMaxMemory: -1,
-    startSize: 1500000,
-    swapEnabled: false,
-    sqlOnheapRowCacheSize: 10240,
-    longQueryWarningTimeout: 3000,
-    snapshotableIndex: false,
-    sqlEscapeAll: false,
-    storeKeepBinary: false,
-    loadPreviousValue: false,
-    cacheStoreFactory: {
-        CacheJdbcPojoStoreFactory: {
-            batchSize: 512,
-            maximumWriteAttempts: 2,
-            parallelLoadCacheMinimumThreshold: 512,
-            sqlEscapeAll: false
-        }
-    },
-    readThrough: false,
-    writeThrough: false,
-    writeBehindEnabled: false,
-    writeBehindBatchSize: 512,
-    writeBehindFlushSize: 10240,
-    writeBehindFlushFrequency: 5000,
-    writeBehindFlushThreadCount: 1,
-    maxConcurrentAsyncOperations: 500,
-    defaultLockTimeout: 0,
-    atomicWriteOrderMode: {
-        clsName: 'org.apache.ignite.cache.CacheAtomicWriteOrderMode'
-    },
-    writeSynchronizationMode: {
-        clsName: 'org.apache.ignite.cache.CacheWriteSynchronizationMode',
-        value: 'PRIMARY_SYNC'
-    },
-    rebalanceMode: {
-        clsName: 'org.apache.ignite.cache.CacheRebalanceMode',
-        value: 'ASYNC'
-    },
-    rebalanceThreadPoolSize: 1,
-    rebalanceBatchSize: 524288,
-    rebalanceBatchesPrefetchCount: 2,
-    rebalanceOrder: 0,
-    rebalanceDelay: 0,
-    rebalanceTimeout: 10000,
-    rebalanceThrottle: 0,
-    statisticsEnabled: false,
-    managementEnabled: false,
-    nearConfiguration: {
-        nearStartSize: 375000
-    },
-    clientNearConfiguration: {
-        nearStartSize: 375000
-    },
-    evictionPolicy: {
-        LRU: {
-            batchSize: 1,
-            maxSize: 100000
-        },
-        FIFO: {
-            batchSize: 1,
-            maxSize: 100000
-        },
-        SORTED: {
-            batchSize: 1,
-            maxSize: 100000
-        }
-    },
-    queryMetadata: 'Configuration',
-    fields: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        valField: 'className',
-        entries: []
-    },
-    aliases: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        keyField: 'field',
-        valField: 'alias',
-        entries: []
-    },
-    indexes: {
-        indexType: {
-            clsName: 'org.apache.ignite.cache.QueryIndexType'
-        },
-        fields: {
-            keyClsName: 'java.lang.String',
-            valClsName: 'java.lang.Boolean',
-            valField: 'direction',
-            entries: []
-        }
-    },
-    typeField: {
-        databaseFieldType: {
-            clsName: 'java.sql.Types'
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CACHE, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_CACHE;
-    }];
-}


[31/50] [abbrv] ignite git commit: Merge remote-tracking branch 'origin/master'

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/master'


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

Branch: refs/heads/ignite-2.0
Commit: e2ac66678fa28c6744c4806bda87ffc6800018ec
Parents: 2591c16 848bb66
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 22 13:06:01 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 22 13:06:01 2016 -0800

----------------------------------------------------------------------
 .gitignore                                      |   1 +
 bin/ignite.sh                                   |   9 +
 examples/pom-standalone.xml                     |   6 +
 modules/cassandra/pom.xml                       |  24 +-
 .../store/cassandra/CassandraCacheStore.java    |   6 +
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 +--
 .../org/apache/ignite/internal/GridTopic.java   |   8 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      | 734 +++++++------------
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../GridCachePartitionExchangeManager.java      |  60 +-
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../processors/cache/GridCacheUtils.java        |  10 +-
 .../processors/cache/IgniteCacheProxy.java      |   8 -
 .../CacheDataStructuresManager.java             |   6 +-
 .../distributed/GridCacheTxRecoveryFuture.java  |   4 +-
 .../GridDistributedTxFinishResponse.java        |   6 -
 .../GridDistributedTxRemoteAdapter.java         |   2 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |   2 +-
 .../distributed/dht/GridDhtLockFuture.java      |   4 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  74 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   | 125 ++--
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   7 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   4 +-
 .../dht/atomic/GridDhtAtomicCache.java          | 472 +++++++-----
 .../dht/colocated/GridDhtColocatedCache.java    |  13 -
 .../colocated/GridDhtColocatedLockFuture.java   |   4 +-
 .../GridDhtPartitionsExchangeFuture.java        |   6 +-
 .../distributed/near/GridNearAtomicCache.java   |  65 +-
 .../distributed/near/GridNearLockFuture.java    |   4 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   4 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |  15 +-
 .../GridNearPessimisticTxPrepareFuture.java     |   4 +-
 .../near/GridNearTxFinishFuture.java            |  46 +-
 .../cache/distributed/near/GridNearTxLocal.java |  48 +-
 .../local/atomic/GridLocalAtomicCache.java      | 177 +----
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../cache/store/CacheStoreManager.java          |   4 +-
 .../store/GridCacheStoreManagerAdapter.java     |  34 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   6 +-
 .../cache/transactions/IgniteTxHandler.java     |  13 +-
 .../transactions/IgniteTxLocalAdapter.java      |  30 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../GridCacheAtomicSequenceImpl.java            |  12 +-
 .../processors/hadoop/HadoopClassLoader.java    |   1 +
 .../processors/hadoop/HadoopJobProperty.java    |  10 +-
 .../processors/hadoop/HadoopTaskContext.java    |   8 +
 .../io/PartiallyOffheapRawComparatorEx.java     |  33 +
 .../internal/processors/job/GridJobWorker.java  |  76 +-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../service/GridServiceProcessor.java           |  11 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../util/future/GridCompoundFuture.java         |  56 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   4 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  14 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 .../GridCacheMissingCommitVersionSelfTest.java  |   6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   1 +
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 ++++
 ...achePartitionedAtomicSequenceTxSelfTest.java | 169 +++++
 .../CacheLockReleaseNodeLeaveTest.java          | 135 ++++
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |  37 +-
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 ----------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 ++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 +++++
 ...lockMessageSystemPoolStarvationSelfTest.java |  14 +-
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheDataStructuresSelfTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../hadoop/io/PartiallyRawComparator.java       |  33 +
 .../org/apache/ignite/hadoop/io/RawMemory.java  |  86 +++
 .../hadoop/io/TextPartiallyRawComparator.java   | 115 +++
 .../apache/ignite/hadoop/io/package-info.java   |  22 +
 ...DelegatingPartiallyOffheapRawComparator.java |  54 ++
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  21 +
 .../processors/hadoop/io/OffheapRawMemory.java  | 131 ++++
 .../hadoop/shuffle/HadoopShuffleJob.java        |   2 +-
 .../shuffle/collections/HadoopMultimapBase.java |   5 +-
 .../shuffle/collections/HadoopSkipList.java     |  14 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |   7 +
 .../collections/HadoopAbstractMapTest.java      |   6 +
 .../processors/query/h2/IgniteH2Indexing.java   |  75 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |  15 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 .../IgniteCacheQuerySelfTestSuite.java          | 130 +++-
 .../IgniteCacheQuerySelfTestSuite2.java         | 111 +--
 .../IgniteH2IndexingSpiTestSuite.java           |  16 +-
 .../include/ignite/binary/binary_raw_reader.h   |  21 +
 .../ignite/impl/binary/binary_reader_impl.h     |   9 +-
 .../src/impl/binary/binary_reader_impl.cpp      |  17 +
 .../platforms/cpp/common/include/Makefile.am    |   5 +-
 .../common/include/ignite/common/concurrent.h   |  90 ++-
 .../include/ignite/common/reference_impl.h      | 286 ++++++++
 .../cpp/common/include/ignite/reference.h       | 564 ++++++++++++++
 .../cpp/common/project/vs/common.vcxproj        |   2 +
 .../common/project/vs/common.vcxproj.filters    |   6 +
 modules/platforms/cpp/core-test/Makefile.am     |   7 +-
 .../core-test/config/cache-query-continuous.xml |  87 +++
 .../cpp/core-test/config/cache-test.xml         |   1 +
 .../platforms/cpp/core-test/config/invalid.xml  |  39 +
 .../cpp/core-test/project/vs/core-test.vcxproj  |  13 +-
 .../project/vs/core-test.vcxproj.filters        |  11 +-
 .../cpp/core-test/src/cache_query_test.cpp      |  76 +-
 .../cpp/core-test/src/continuous_query_test.cpp | 611 +++++++++++++++
 .../cpp/core-test/src/handle_registry_test.cpp  |  18 +-
 .../cpp/core-test/src/interop_test.cpp          |  17 +-
 .../cpp/core-test/src/reference_test.cpp        | 412 +++++++++++
 modules/platforms/cpp/core/Makefile.am          |   1 +
 .../cpp/core/include/ignite/cache/cache.h       | 104 ++-
 .../cpp/core/include/ignite/cache/cache_entry.h |  40 +-
 .../ignite/cache/event/cache_entry_event.h      | 139 ++++
 .../cache/event/cache_entry_event_listener.h    |  71 ++
 .../cache/query/continuous/continuous_query.h   | 239 ++++++
 .../query/continuous/continuous_query_handle.h  | 133 ++++
 .../core/include/ignite/impl/cache/cache_impl.h | 116 ++-
 .../continuous/continuous_query_handle_impl.h   | 101 +++
 .../query/continuous/continuous_query_impl.h    | 351 +++++++++
 .../core/include/ignite/impl/handle_registry.h  |  62 +-
 .../include/ignite/impl/ignite_environment.h    |  34 +-
 modules/platforms/cpp/core/namespaces.dox       |  74 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   7 +
 .../cpp/core/project/vs/core.vcxproj.filters    |  30 +
 modules/platforms/cpp/core/src/ignition.cpp     |  15 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |  31 +
 .../continuous/continuous_query_handle_impl.cpp |  96 +++
 .../cpp/core/src/impl/handle_registry.cpp       | 102 ++-
 .../cpp/core/src/impl/ignite_environment.cpp    |  98 ++-
 modules/platforms/cpp/examples/Makefile.am      |   1 +
 modules/platforms/cpp/examples/configure.ac     |   1 +
 .../continuous-query-example/Makefile.am        |  58 ++
 .../config/continuous-query-example.xml         |  52 ++
 .../project/vs/continuous-query-example.vcxproj | 110 +++
 .../vs/continuous-query-example.vcxproj.filters |  35 +
 .../src/continuous_query_example.cpp            | 142 ++++
 .../examples/include/ignite/examples/person.h   |   2 +-
 .../cpp/examples/project/vs/ignite-examples.sln |   6 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |   2 +-
 modules/platforms/cpp/jni/src/java.cpp          |   4 +-
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |   4 +-
 modules/platforms/cpp/project/vs/ignite.slnrel  |   3 +
 .../platforms/cpp/project/vs/ignite_x86.slnrel  |   3 +
 .../Cache/Query/CacheLinqTest.cs                |  21 +-
 .../Apache.Ignite.Core.Tests/TestUtils.cs       |   3 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |  12 +-
 .../Apache.Ignite.Linq/Impl/MethodVisitor.cs    |  34 +-
 modules/platforms/dotnet/Apache.Ignite.sln      |   2 +
 modules/platforms/dotnet/DEVNOTES.txt           |  12 +-
 modules/platforms/dotnet/build.bat              |  14 +
 modules/platforms/dotnet/build.ps1              | 211 ++++++
 171 files changed, 8519 insertions(+), 2409 deletions(-)
----------------------------------------------------------------------


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


[14/50] [abbrv] ignite git commit: ignite-4411 Changed test to avoid deadlocks when striped pool is used.

Posted by vo...@apache.org.
ignite-4411 Changed test to avoid deadlocks when striped pool is used.

(cherry picked from commit 9a62d53)


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

Branch: refs/heads/ignite-2.0
Commit: 28dab6ea7573affe95512a403821cee5d82fcaf5
Parents: 05dd08b
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 15 12:15:22 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 16 14:11:04 2016 +0300

----------------------------------------------------------------------
 ...achePartitionedMultiNodeFullApiSelfTest.java | 37 ++++++++++++++++----
 1 file changed, 30 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/28dab6ea/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 71b14eb..34b67bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -24,6 +24,7 @@ import java.util.HashSet;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteCache;
@@ -37,10 +38,12 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.resources.LoggerResource;
+import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.cache.CacheMode.REPLICATED;
@@ -194,7 +197,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
         for (int i = 0; i < gridCount(); i++)
             grid(i).events().localListen(
-                    new SwapUnswapLocalListener(), EVT_CACHE_OBJECT_SWAPPED, EVT_CACHE_OBJECT_UNSWAPPED);
+                new SwapUnswapLocalListener(), EVT_CACHE_OBJECT_SWAPPED, EVT_CACHE_OBJECT_UNSWAPPED);
 
         jcache().put("key", 1);
 
@@ -202,13 +205,19 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
             if (grid(i).affinity(null).isBackup(grid(i).localNode(), "key")) {
                 jcache(i).localEvict(Collections.singleton("key"));
 
-                assert jcache(i).localPeek("key", ONHEAP) == null;
+                assertNull(jcache(i).localPeek("key", ONHEAP));
 
-                assert jcache(i).get("key") == 1;
+                assertEquals((Integer)1, jcache(i).get("key"));
+
+                GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                    @Override public boolean apply() {
+                        return swapEvts.get() == 1 && unswapEvts.get() == 1;
+                    }
+                }, 5000);
 
-                assert swapEvts.get() == 1 : "Swap events: " + swapEvts.get();
+                assertEquals(1, swapEvts.get());
 
-                assert unswapEvts.get() == 1 : "Unswap events: " + unswapEvts.get();
+                assertEquals(1, unswapEvts.get());
 
                 break;
             }
@@ -464,11 +473,25 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
 
             switch (evt.type()) {
                 case EVT_CACHE_OBJECT_SWAPPED:
-                    ignite.atomicLong("swapEvts", 0, false).incrementAndGet();
+                    // Run from another thread to avoid deadlock with striped pool.
+                    GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            ignite.atomicLong("swapEvts", 0, false).incrementAndGet();
+
+                            return null;
+                        }
+                    });
 
                     break;
                 case EVT_CACHE_OBJECT_UNSWAPPED:
-                    ignite.atomicLong("unswapEvts", 0, false).incrementAndGet();
+                    // Run from another thread to avoid deadlock with striped pool.
+                    GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            ignite.atomicLong("unswapEvts", 0, false).incrementAndGet();
+
+                            return null;
+                        }
+                    });
 
                     break;
             }


[10/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 63c9845..66f9176 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.util.nio;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.Semaphore;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -37,17 +39,14 @@ import org.jsr166.ConcurrentLinkedDeque8;
  */
 class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     /** Pending write requests. */
-    private final ConcurrentLinkedDeque8<GridNioFuture<?>> queue = new ConcurrentLinkedDeque8<>();
+    private final ConcurrentLinkedDeque8<SessionWriteRequest> queue = new ConcurrentLinkedDeque8<>();
 
     /** Selection key associated with this session. */
     @GridToStringExclude
     private SelectionKey key;
 
-    /** Worker index for server */
-    private final int selectorIdx;
-
-    /** Size counter. */
-    private final AtomicInteger queueSize = new AtomicInteger();
+    /** Current worker thread. */
+    private volatile GridNioWorker worker;
 
     /** Semaphore. */
     @GridToStringExclude
@@ -59,17 +58,29 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     /** Read buffer. */
     private ByteBuffer readBuf;
 
-    /** Recovery data. */
-    private GridNioRecoveryDescriptor recovery;
+    /** Incoming recovery data. */
+    private GridNioRecoveryDescriptor inRecovery;
+
+    /** Outgoing recovery data. */
+    private GridNioRecoveryDescriptor outRecovery;
 
     /** Logger. */
     private final IgniteLogger log;
 
+    /** */
+    private List<GridNioServer.SessionChangeRequest> pendingStateChanges;
+
+    /** */
+    final AtomicBoolean procWrite = new AtomicBoolean();
+
+    /** */
+    private Object sysMsg;
+
     /**
      * Creates session instance.
      *
      * @param log Logger.
-     * @param selectorIdx Selector index for this session.
+     * @param worker NIO worker thread.
      * @param filterChain Filter chain that will handle requests.
      * @param locAddr Local address.
      * @param rmtAddr Remote address.
@@ -80,7 +91,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      */
     GridSelectorNioSessionImpl(
         IgniteLogger log,
-        int selectorIdx,
+        GridNioWorker worker,
         GridNioFilterChain filterChain,
         InetSocketAddress locAddr,
         InetSocketAddress rmtAddr,
@@ -91,7 +102,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     ) {
         super(filterChain, locAddr, rmtAddr, accepted);
 
-        assert selectorIdx >= 0;
+        assert worker != null;
         assert sndQueueLimit >= 0;
 
         assert locAddr != null : "GridSelectorNioSessionImpl should have local socket address.";
@@ -101,7 +112,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
         this.log = log;
 
-        this.selectorIdx = selectorIdx;
+        this.worker = worker;
 
         sem = sndQueueLimit > 0 ? new Semaphore(sndQueueLimit) : null;
 
@@ -119,12 +130,19 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
+     * @return Worker.
+     */
+    GridNioWorker worker() {
+        return worker;
+    }
+
+    /**
      * Sets selection key for this session.
      *
      * @param key Selection key.
      */
     void key(SelectionKey key) {
-        assert this.key == null;
+        assert key != null;
 
         this.key = key;
     }
@@ -151,10 +169,88 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
-     * @return Selector index.
+     * @param from Current session worker.
+     * @param fut Move future.
+     * @return {@code True} if session move was scheduled.
+     */
+    boolean offerMove(GridNioWorker from, GridNioServer.SessionChangeRequest fut) {
+        synchronized (this) {
+            if (log.isDebugEnabled())
+                log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+            GridNioWorker worker0 = worker;
+
+            if (worker0 != from)
+                return false;
+
+            worker.offer(fut);
+        }
+
+        return true;
+    }
+
+    /**
+     * @param fut Future.
+     */
+    void offerStateChange(GridNioServer.SessionChangeRequest fut) {
+        synchronized (this) {
+            if (log.isDebugEnabled())
+                log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+            GridNioWorker worker0 = worker;
+
+            if (worker0 == null) {
+                if (pendingStateChanges == null)
+                    pendingStateChanges = new ArrayList<>();
+
+                pendingStateChanges.add(fut);
+            }
+            else
+                worker0.offer(fut);
+        }
+    }
+
+    /**
+     * @param moveFrom Current session worker.
      */
-    int selectorIndex() {
-        return selectorIdx;
+    void startMoveSession(GridNioWorker moveFrom) {
+        synchronized (this) {
+            assert this.worker == moveFrom;
+
+            if (log.isDebugEnabled())
+                log.debug("Started moving [ses=" + this + ", from=" + moveFrom + ']');
+
+            List<GridNioServer.SessionChangeRequest> sesReqs = moveFrom.clearSessionRequests(this);
+
+            worker = null;
+
+            if (sesReqs != null) {
+                if (pendingStateChanges == null)
+                    pendingStateChanges = new ArrayList<>();
+
+                pendingStateChanges.addAll(sesReqs);
+            }
+        }
+    }
+
+    /**
+     * @param moveTo New session worker.
+     */
+    void finishMoveSession(GridNioWorker moveTo) {
+        synchronized (this) {
+            assert worker == null;
+
+            if (log.isDebugEnabled())
+                log.debug("Finishing moving [ses=" + this + ", to=" + moveTo + ']');
+
+            worker = moveTo;
+
+            if (pendingStateChanges != null) {
+                moveTo.offer(pendingStateChanges);
+
+                pendingStateChanges = null;
+            }
+        }
     }
 
     /**
@@ -163,14 +259,14 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param writeFut Write request.
      * @return Updated size of the queue.
      */
-    int offerSystemFuture(GridNioFuture<?> writeFut) {
+    int offerSystemFuture(SessionWriteRequest writeFut) {
         writeFut.messageThread(true);
 
         boolean res = queue.offerFirst(writeFut);
 
         assert res : "Future was not added to queue";
 
-        return queueSize.incrementAndGet();
+        return queue.sizex();
     }
 
     /**
@@ -183,7 +279,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param writeFut Write request to add.
      * @return Updated size of the queue.
      */
-    int offerFuture(GridNioFuture<?> writeFut) {
+    int offerFuture(SessionWriteRequest writeFut) {
         boolean msgThread = GridNioBackPressureControl.threadProcessingMessage();
 
         if (sem != null && !msgThread)
@@ -195,47 +291,41 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
         assert res : "Future was not added to queue";
 
-        return queueSize.incrementAndGet();
+        return queue.sizex();
     }
 
     /**
      * @param futs Futures to resend.
      */
-    void resend(Collection<GridNioFuture<?>> futs) {
+    void resend(Collection<SessionWriteRequest> futs) {
         assert queue.isEmpty() : queue.size();
 
         boolean add = queue.addAll(futs);
 
         assert add;
-
-        boolean set = queueSize.compareAndSet(0, futs.size());
-
-        assert set;
     }
 
     /**
      * @return Message that is in the head of the queue, {@code null} if queue is empty.
      */
-    @Nullable GridNioFuture<?> pollFuture() {
-        GridNioFuture<?> last = queue.poll();
+    @Nullable SessionWriteRequest pollFuture() {
+        SessionWriteRequest last = queue.poll();
 
         if (last != null) {
-            queueSize.decrementAndGet();
-
             if (sem != null && !last.messageThread())
                 sem.release();
 
-            if (recovery != null) {
-                if (!recovery.add(last)) {
+            if (outRecovery != null) {
+                if (!outRecovery.add(last)) {
                     LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
                         "[remoteAddr=" + remoteAddress() +
-                        ", queueLimit=" + recovery.queueLimit() + ']');
+                        ", queueLimit=" + outRecovery.queueLimit() + ']');
 
                     if (log.isDebugEnabled())
                         log.debug("Unacknowledged messages queue size overflow, will attempt to reconnect " +
                             "[remoteAddr=" + remoteAddress() +
-                            ", queueSize=" + recovery.messagesFutures().size() +
-                            ", queueLimit=" + recovery.queueLimit() + ']');
+                            ", queueSize=" + outRecovery.messagesRequests().size() +
+                            ", queueLimit=" + outRecovery.queueLimit() + ']');
 
                     close();
                 }
@@ -249,7 +339,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param fut Future.
      * @return {@code True} if future was removed from queue.
      */
-    boolean removeFuture(GridNioFuture<?> fut) {
+    boolean removeFuture(SessionWriteRequest fut) {
         assert closed();
 
         return queue.removeLastOccurrence(fut);
@@ -261,35 +351,49 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @return Number of write requests.
      */
     int writeQueueSize() {
-        return queueSize.get();
+        return queue.sizex();
     }
 
     /**
      * @return Write requests.
      */
-    Collection<GridNioFuture<?>> writeQueue() {
+    Collection<SessionWriteRequest> writeQueue() {
         return queue;
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         assert recoveryDesc != null;
 
-        recovery = recoveryDesc;
+        outRecovery = recoveryDesc;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
-        return recovery;
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+        return outRecovery;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        assert recoveryDesc != null;
+
+        inRecovery = recoveryDesc;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+        return inRecovery;
     }
 
     /** {@inheritDoc} */
     @Override public <T> T addMeta(int key, @Nullable T val) {
-        if (val instanceof GridNioRecoveryDescriptor) {
-            recovery = (GridNioRecoveryDescriptor)val;
+        if (!accepted() && val instanceof GridNioRecoveryDescriptor) {
+            outRecovery = (GridNioRecoveryDescriptor)val;
+
+            if (!outRecovery.pairedConnections())
+                inRecovery = outRecovery;
 
-            if (!accepted())
-                recovery.connected();
+            outRecovery.onConnected();
 
             return null;
         }
@@ -313,6 +417,31 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public void systemMessage(Object sysMsg) {
+        this.sysMsg = sysMsg;
+    }
+
+    /**
+     * @return {@code True} if have pending system message to send.
+     */
+    boolean hasSystemMessage() {
+        return sysMsg != null;
+    }
+
+    /**
+     * Gets and clears pending system message.
+     *
+     * @return Pending system message.
+     */
+    Object systemMessage() {
+        Object ret = sysMsg;
+
+        sysMsg = null;
+
+        return ret;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
index ebe86fb..d941bae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -48,6 +48,7 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
     private final MessageFormatter formatter;
 
     /**
+     * @param connIdx Connection index.
      * @param metricsLsnr Metrics listener.
      * @param port Shared memory IPC server port.
      * @param connTimeout Connection timeout.
@@ -55,14 +56,16 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
      * @param formatter Message formatter.
      * @throws IgniteCheckedException If failed.
      */
-    public GridShmemCommunicationClient(GridNioMetricsListener metricsLsnr,
+    public GridShmemCommunicationClient(
+        int connIdx,
+        GridNioMetricsListener metricsLsnr,
         int port,
         long connTimeout,
         IgniteLogger log,
         MessageFormatter formatter)
         throws IgniteCheckedException
     {
-        super(metricsLsnr);
+        super(connIdx, metricsLsnr);
 
         assert metricsLsnr != null;
         assert port > 0 && port < 0xffff;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 5fe521d..3397772 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -45,11 +45,16 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     private final IgniteLogger log;
 
     /**
+     * @param connIdx Connection index.
      * @param ses Session.
      * @param log Logger.
      */
-    public GridTcpNioCommunicationClient(GridNioSession ses, IgniteLogger log) {
-        super(null);
+    public GridTcpNioCommunicationClient(
+        int connIdx,
+        GridNioSession ses,
+        IgniteLogger log
+    ) {
+        super(connIdx, null);
 
         assert ses != null;
         assert log != null;
@@ -104,40 +109,36 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     }
 
     /** {@inheritDoc} */
-    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> closure)
+    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> c)
         throws IgniteCheckedException {
-        // Node ID is never provided in asynchronous send mode.
-        assert nodeId == null;
+        try {
+            // Node ID is never provided in asynchronous send mode.
+            assert nodeId == null;
 
-        if (closure != null)
-            ses.addMeta(ACK_CLOSURE.ordinal(), closure);
+            if (c != null)
+                ses.addMeta(ACK_CLOSURE.ordinal(), c);
 
-        GridNioFuture<?> fut = ses.send(msg);
+            ses.sendNoFuture(msg);
 
-        if (fut.isDone()) {
-            try {
-                fut.get();
-            }
-            catch (IgniteCheckedException e) {
-                if (closure != null)
-                    ses.removeMeta(ACK_CLOSURE.ordinal());
+            if (c != null)
+                ses.removeMeta(ACK_CLOSURE.ordinal());
+        }
+        catch (IgniteCheckedException e) {
+            if (c != null)
+                ses.removeMeta(ACK_CLOSURE.ordinal());
 
-                if (log.isDebugEnabled())
-                    log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
+            if (log.isDebugEnabled())
+                log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
 
-                if (e.getCause() instanceof IOException) {
-                    ses.close();
+            if (e.getCause() instanceof IOException) {
+                ses.close();
 
-                    return true;
-                }
-                else
-                    throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
+                return true;
             }
+            else
+                throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
         }
 
-        if (closure != null)
-            ses.removeMeta(ACK_CLOSURE.ordinal());
-
         return false;
     }
 
@@ -159,4 +160,4 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     @Override public String toString() {
         return S.toString(GridTcpNioCommunicationClient.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
new file mode 100644
index 0000000..508c791
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
+
+/**
+ *
+ */
+public interface SessionWriteRequest {
+    /**
+     * Sets flag indicating that message send future was created in thread that was processing a message.
+     *
+     * @param msgThread {@code True} if future was created in thread that is processing message.
+     */
+    public void messageThread(boolean msgThread);
+
+    /**
+     * @return {@code True} if future was created in thread that was processing message.
+     */
+    public boolean messageThread();
+
+    /**
+     * @return {@code True} if skip recovery for this operation.
+     */
+    public boolean skipRecovery();
+
+    /**
+     * Sets ack closure which will be applied when ack received.
+     *
+     * @param c Ack closure.
+     */
+    public void ackClosure(IgniteInClosure<IgniteException> c);
+
+    /**
+     * The method will be called when ack received.
+     */
+    public void onAckReceived();
+
+    /**
+     * @return Ack closure.
+     */
+    public IgniteInClosure<IgniteException> ackClosure();
+
+    /**
+     * @return Session.
+     */
+    public GridNioSession session();
+
+    /**
+     * @param ses Session.
+     */
+    public void resetSession(GridNioSession ses);
+
+    /**
+     *
+     */
+    public void onError(Exception e);
+
+    /**
+     * @return Message.
+     */
+    public Object message();
+
+    /**
+     *
+     */
+    public void onMessageWritten();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
index d6f9d10..8ed7db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
@@ -282,9 +282,13 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         if (directMode)
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
 
         ByteBuffer input = checkMessage(ses, msg);
 
@@ -441,4 +445,4 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
 
         return (ByteBuffer)msg;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
index eb8dad4..269e8b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
@@ -437,7 +437,7 @@ class GridNioSslHandler extends ReentrantLock {
         while (!deferredWriteQueue.isEmpty()) {
             WriteRequest req = deferredWriteQueue.poll();
 
-            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer()));
+            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer(), true));
         }
     }
 
@@ -482,7 +482,7 @@ class GridNioSslHandler extends ReentrantLock {
 
         ByteBuffer cp = copy(outNetBuf);
 
-        return parent.proceedSessionWrite(ses, cp);
+        return parent.proceedSessionWrite(ses, cp, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index b29d7cd..86aa7a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -598,4 +598,4 @@ public class GridToStringBuilder {
 
         return cd;
     }
-}
\ No newline at end of file
+}


[35/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 4e6e372..0d0b171 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.controller.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.controller.js
@@ -50,6 +50,9 @@ class Paragraph {
         const self = this;
 
         self.id = 'paragraph-' + paragraphId++;
+        self.qryType = paragraph.qryType || 'query';
+        self.maxPages = 0;
+        self.filter = '';
 
         _.assign(this, paragraph);
 
@@ -77,27 +80,28 @@ class Paragraph {
             enableColumnMenus: false,
             flatEntityAccess: true,
             fastWatch: true,
+            categories: [],
             rebuildColumns() {
                 if (_.isNil(this.api))
                     return;
 
-                this.categories = [];
+                this.categories.length = 0;
+
                 this.columnDefs = _.reduce(self.meta, (cols, col, idx) => {
-                    if (self.columnFilter(col)) {
-                        cols.push({
-                            displayName: col.fieldName,
-                            headerTooltip: _fullColName(col),
-                            field: idx.toString(),
-                            minWidth: 50,
-                            cellClass: 'cell-left'
-                        });
+                    cols.push({
+                        displayName: col.fieldName,
+                        headerTooltip: _fullColName(col),
+                        field: idx.toString(),
+                        minWidth: 50,
+                        cellClass: 'cell-left',
+                        visible: self.columnFilter(col)
+                    });
 
-                        this.categories.push({
-                            name: col.fieldName,
-                            visible: true,
-                            selectable: true
-                        });
-                    }
+                    this.categories.push({
+                        name: col.fieldName,
+                        visible: self.columnFilter(col),
+                        selectable: true
+                    });
 
                     return cols;
                 }, []);
@@ -182,8 +186,8 @@ class Paragraph {
 }
 
 // Controller for SQL notebook screen.
-export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteScanFilterInput', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion',
-    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, ScanFilterInput, Nodes, uiGridExporterConstants, Version) {
+export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval', '$animate', '$location', '$anchorScroll', '$state', '$filter', '$modal', '$popover', 'IgniteLoading', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteAgentMonitor', 'IgniteChartColors', 'IgniteNotebook', 'IgniteNodes', 'uiGridExporterConstants', 'IgniteVersion',
+    function($root, $scope, $http, $q, $timeout, $interval, $animate, $location, $anchorScroll, $state, $filter, $modal, $popover, Loading, LegacyUtils, Messages, Confirm, agentMonitor, IgniteChartColors, Notebook, Nodes, uiGridExporterConstants, Version) {
         let stopTopology = null;
 
         const _tryStopRefresh = function(paragraph) {
@@ -206,6 +210,15 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         $scope.caches = [];
 
         $scope.pageSizes = [50, 100, 200, 400, 800, 1000];
+        $scope.maxPages = [
+            {label: 'Unlimited', value: 0},
+            {label: '1', value: 1},
+            {label: '5', value: 5},
+            {label: '10', value: 10},
+            {label: '20', value: 20},
+            {label: '50', value: 50},
+            {label: '100', value: 100}
+        ];
 
         $scope.timeLineSpans = ['1', '5', '10', '15', '30'];
 
@@ -213,7 +226,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
         $scope.modes = LegacyUtils.mkOptions(['PARTITIONED', 'REPLICATED', 'LOCAL']);
 
-        $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading notebook screen...';
+        $scope.loadingText = $root.IgniteDemoMode ? 'Demo grid is starting. Please wait...' : 'Loading query notebook screen...';
 
         $scope.timeUnit = [
             {value: 1000, label: 'seconds', short: 's'},
@@ -768,11 +781,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             if (idx >= 0) {
                 if (!_.includes($scope.notebook.expandedParagraphs, idx))
-                    $scope.notebook.expandedParagraphs.push(idx);
+                    $scope.notebook.expandedParagraphs = $scope.notebook.expandedParagraphs.concat([idx]);
 
-                setTimeout(function() {
-                    $scope.notebook.paragraphs[idx].ace.focus();
-                });
+                if ($scope.notebook.paragraphs[idx].ace)
+                    setTimeout(() => $scope.notebook.paragraphs[idx].ace.focus());
             }
 
             $location.hash(id);
@@ -816,7 +828,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                             let item = _.find(cachesAcc, {name: cache.name});
 
                             if (_.isNil(item)) {
-                                cache.label = maskCacheName(cache.name);
+                                cache.label = maskCacheName(cache.name, true);
+                                cache.value = cache.name;
 
                                 cache.nodes = [];
 
@@ -839,7 +852,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                         return;
 
                     // Reset to first cache in case of stopped selected.
-                    const cacheNames = _.map($scope.caches, (cache) => cache.name);
+                    const cacheNames = _.map($scope.caches, (cache) => cache.value);
 
                     _.forEach($scope.notebook.paragraphs, (paragraph) => {
                         if (!_.includes(cacheNames, paragraph.cacheName))
@@ -885,7 +898,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     (paragraph) => new Paragraph($animate, $timeout, paragraph));
 
                 if (_.isEmpty($scope.notebook.paragraphs))
-                    $scope.addParagraph();
+                    $scope.addQuery();
                 else
                     $scope.rebuildScrollParagraphs();
             })
@@ -936,32 +949,37 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 paragraph.edit = false;
         };
 
-        $scope.addParagraph = function() {
+        $scope.addParagraph = (paragraph, sz) => {
+            if ($scope.caches && $scope.caches.length > 0)
+                paragraph.cacheName = _.head($scope.caches).value;
+
+            $scope.notebook.paragraphs.push(paragraph);
+
+            $scope.notebook.expandedParagraphs.push(sz);
+
+            $scope.rebuildScrollParagraphs();
+
+            $location.hash(paragraph.id);
+        };
+
+        $scope.addQuery = function() {
             const sz = $scope.notebook.paragraphs.length;
 
             const paragraph = new Paragraph($animate, $timeout, {
                 name: 'Query' + (sz === 0 ? '' : sz),
                 query: '',
-                pageSize: $scope.pageSizes[0],
+                pageSize: $scope.pageSizes[1],
                 timeLineSpan: $scope.timeLineSpans[0],
                 result: 'none',
                 rate: {
                     value: 1,
                     unit: 60000,
                     installed: false
-                }
+                },
+                qryType: 'query'
             });
 
-            if ($scope.caches && $scope.caches.length > 0)
-                paragraph.cacheName = $scope.caches[0].name;
-
-            $scope.notebook.paragraphs.push(paragraph);
-
-            $scope.notebook.expandedParagraphs.push(sz);
-
-            $scope.rebuildScrollParagraphs();
-
-            $location.hash(paragraph.id);
+            $scope.addParagraph(paragraph, sz);
 
             $timeout(() => {
                 $anchorScroll();
@@ -970,6 +988,26 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             });
         };
 
+        $scope.addScan = function() {
+            const sz = $scope.notebook.paragraphs.length;
+
+            const paragraph = new Paragraph($animate, $timeout, {
+                name: 'Scan' + (sz === 0 ? '' : sz),
+                query: '',
+                pageSize: $scope.pageSizes[1],
+                timeLineSpan: $scope.timeLineSpans[0],
+                result: 'none',
+                rate: {
+                    value: 1,
+                    unit: 60000,
+                    installed: false
+                },
+                qryType: 'scan'
+            });
+
+            $scope.addParagraph(paragraph, sz);
+        };
+
         function _saveChartSettings(paragraph) {
             if (!_.isEmpty(paragraph.charts)) {
                 const chart = paragraph.charts[0].api.getScope().chart;
@@ -1010,7 +1048,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         $scope.removeParagraph = function(paragraph) {
-            Confirm.confirm('Are you sure you want to remove: "' + paragraph.name + '"?')
+            Confirm.confirm('Are you sure you want to remove query: "' + paragraph.name + '"?')
                 .then(function() {
                     $scope.stopRefresh(paragraph);
 
@@ -1315,8 +1353,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
             return false;
         };
 
-        $scope.execute = (paragraph, nonCollocatedJoins = false) => {
-            const local = !!paragraph.localQry;
+        $scope.execute = (paragraph, local = false) => {
+            const nonCollocatedJoins = !!paragraph.nonCollocatedJoins;
 
             $scope.actionAvailable(paragraph, true) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
@@ -1330,16 +1368,16 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     return _closeOldQuery(paragraph)
                         .then(() => {
                             const args = paragraph.queryArgs = {
+                                type: 'QUERY',
                                 cacheName: paragraph.cacheName,
-                                pageSize: paragraph.pageSize,
                                 query: paragraph.query,
-                                firstPageOnly: paragraph.firstPageOnly,
+                                pageSize: paragraph.pageSize,
+                                maxPages: paragraph.maxPages,
                                 nonCollocatedJoins,
-                                type: 'QUERY',
                                 localNid: local ? nid : null
                             };
 
-                            const qry = args.firstPageOnly ? addLimit(args.query, args.pageSize) : paragraph.query;
+                            const qry = args.maxPages ? addLimit(args.query, args.pageSize * args.maxPages) : paragraph.query;
 
                             return agentMonitor.query(nid, args.cacheName, qry, nonCollocatedJoins, local, args.pageSize);
                         })
@@ -1386,10 +1424,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 .then(() => _chooseNode(paragraph.cacheName, false))
                 .then((nid) => {
                     const args = paragraph.queryArgs = {
+                        type: 'EXPLAIN',
                         cacheName: paragraph.cacheName,
-                        pageSize: paragraph.pageSize,
                         query: 'EXPLAIN ' + paragraph.query,
-                        type: 'EXPLAIN'
+                        pageSize: paragraph.pageSize
                     };
 
                     return agentMonitor.query(nid, args.cacheName, args.query, false, false, args.pageSize);
@@ -1403,8 +1441,10 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                 .then(() => paragraph.ace.focus());
         };
 
-        $scope.scan = (paragraph, query = null) => {
-            const local = !!paragraph.localQry;
+        $scope.scan = (paragraph, local = false) => {
+            const {filter, caseSensitive} = paragraph;
+            const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER;
+            const query = `${prefix}${filter}`;
 
             $scope.actionAvailable(paragraph, false) && _chooseNode(paragraph.cacheName, local)
                 .then((nid) => {
@@ -1418,45 +1458,22 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     _closeOldQuery(paragraph)
                         .then(() => {
                             const args = paragraph.queryArgs = {
+                                type: 'SCAN',
                                 cacheName: paragraph.cacheName,
-                                pageSize: paragraph.pageSize,
-                                firstPageOnly: paragraph.firstPageOnly,
                                 query,
-                                type: 'SCAN',
+                                filter,
+                                pageSize: paragraph.pageSize,
                                 localNid: local ? nid : null
                             };
 
                             return agentMonitor.query(nid, args.cacheName, query, false, local, args.pageSize);
                         })
-                        .then((res) => {
-                            if (paragraph.firstPageOnly) {
-                                res.hasMore = false;
-
-                                _processQueryResult(paragraph, true, res);
-
-                                _closeOldQuery(paragraph);
-                            }
-                            else
-                                _processQueryResult(paragraph, true, res);
-                        })
+                        .then((res) => _processQueryResult(paragraph, true, res))
                         .catch((err) => {
                             paragraph.errMsg = err.message;
 
                             _showLoading(paragraph, false);
-                        })
-                        .then(() => paragraph.ace.focus());
-                });
-        };
-
-        $scope.scanWithFilter = (paragraph) => {
-            if (!$scope.actionAvailable(paragraph, false))
-                return;
-
-            ScanFilterInput.open()
-                .then(({filter, caseSensitive}) => {
-                    const prefix = caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE : SCAN_CACHE_WITH_FILTER;
-
-                    $scope.scan(paragraph, `${prefix}${filter}`);
+                        });
                 });
         };
 
@@ -1511,25 +1528,23 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
                     _showLoading(paragraph, false);
                 })
-                .then(() => paragraph.ace.focus());
+                .then(() => paragraph.ace && paragraph.ace.focus());
         };
 
-        const _export = (fileName, columnFilter, meta, rows) => {
+        const _export = (fileName, columnDefs, meta, rows) => {
             let csvContent = '';
 
             const cols = [];
             const excludedCols = [];
 
-            if (meta) {
-                _.forEach(meta, (col, idx) => {
-                    if (columnFilter(col))
-                        cols.push(_fullColName(col));
-                    else
-                        excludedCols.push(idx);
-                });
+            _.forEach(meta, (col, idx) => {
+                if (columnDefs[idx].visible)
+                    cols.push(_fullColName(col));
+                else
+                    excludedCols.push(idx);
+            });
 
-                csvContent += cols.join(';') + '\n';
-            }
+            csvContent += cols.join(';') + '\n';
 
             _.forEach(rows, (row) => {
                 cols.length = 0;
@@ -1543,8 +1558,8 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
                     });
                 }
                 else {
-                    _.forEach(meta, (col) => {
-                        if (columnFilter(col)) {
+                    _.forEach(columnDefs, (col) => {
+                        if (col.visible) {
                             const elem = row[col.fieldName];
 
                             cols.push(_.isUndefined(elem) ? '' : JSON.stringify(elem));
@@ -1559,7 +1574,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         };
 
         $scope.exportCsv = function(paragraph) {
-            _export(paragraph.name + '.csv', paragraph.columnFilter, paragraph.meta, paragraph.rows);
+            _export(paragraph.name + '.csv', paragraph.gridOptions.columnDefs, paragraph.meta, paragraph.rows);
 
             // paragraph.gridOptions.api.exporter.csvExport(uiGridExporterConstants.ALL, uiGridExporterConstants.VISIBLE);
         };
@@ -1573,17 +1588,17 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
 
             return Promise.resolve(args.localNid || _chooseNode(args.cacheName, false))
                 .then((nid) => agentMonitor.queryGetAll(nid, args.cacheName, args.query, !!args.nonCollocatedJoins, !!args.localNid))
-                .then((res) => _export(paragraph.name + '-all.csv', paragraph.columnFilter, res.columns, res.rows))
+                .then((res) => _export(paragraph.name + '-all.csv', paragraph.gridOptions.columnDefs, res.columns, res.rows))
                 .catch(Messages.showError)
-                .then(() => paragraph.ace.focus());
+                .then(() => paragraph.ace && paragraph.ace.focus());
         };
 
         // $scope.exportPdfAll = function(paragraph) {
         //    $http.post('/api/v1/agent/query/getAll', {query: paragraph.query, cacheName: paragraph.cacheName})
-        //        .success(function(item) {
-        //            _export(paragraph.name + '-all.csv', item.meta, item.rows);
+        //    .then(({data}) {
+        //        _export(paragraph.name + '-all.csv', data.meta, data.rows);
         //    })
-        //    .error(Messages.showError);
+        //    .catch(Messages.showError);
         // };
 
         $scope.rateAsString = function(paragraph) {
@@ -1652,9 +1667,7 @@ export default ['$rootScope', '$scope', '$http', '$q', '$timeout', '$interval',
         $scope.dblclickMetadata = function(paragraph, node) {
             paragraph.ace.insert(node.name);
 
-            setTimeout(function() {
-                paragraph.ace.focus();
-            }, 1);
+            setTimeout(() => paragraph.ace.focus(), 1);
         };
 
         $scope.importMetadata = function() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/sql.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/sql.module.js b/modules/web-console/frontend/app/modules/sql/sql.module.js
index d615d28..a1ffde9 100644
--- a/modules/web-console/frontend/app/modules/sql/sql.module.js
+++ b/modules/web-console/frontend/app/modules/sql/sql.module.js
@@ -19,7 +19,6 @@ import angular from 'angular';
 
 import NotebookData from './Notebook.data';
 import Notebook from './Notebook.service';
-import ScanFilterInput from './scan-filter-input.service';
 import notebook from './notebook.controller';
 import sql from './sql.controller';
 
@@ -55,6 +54,5 @@ angular.module('ignite-console.sql', [
     )
     .service('IgniteNotebookData', NotebookData)
     .service('IgniteNotebook', Notebook)
-    .service('IgniteScanFilterInput', ScanFilterInput)
     .controller('notebookController', notebook)
     .controller('sqlController', sql);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration.state.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration.state.js b/modules/web-console/frontend/app/modules/states/configuration.state.js
index 888c804..61dca13 100644
--- a/modules/web-console/frontend/app/modules/states/configuration.state.js
+++ b/modules/web-console/frontend/app/modules/states/configuration.state.js
@@ -24,12 +24,14 @@ import previewPanel from './configuration/preview-panel.directive.js';
 import ConfigurationSummaryCtrl from './configuration/summary/summary.controller';
 import ConfigurationResource from './configuration/Configuration.resource';
 import summaryTabs from './configuration/summary/summary-tabs.directive';
+import IgniteSummaryZipper from './configuration/summary/summary-zipper.service';
 
 angular.module('ignite-console.states.configuration', ['ui.router'])
     .directive(...previewPanel)
     // Summary screen
     .directive(...summaryTabs)
     // Services.
+    .service('IgniteSummaryZipper', IgniteSummaryZipper)
     .service('IgniteConfigurationResource', ConfigurationResource)
     // Configure state provider.
     .config(['$stateProvider', 'AclRouteProvider', ($stateProvider, AclRoute) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 b34aba0..bcac5ad 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
@@ -54,6 +54,6 @@ include /app/helpers/jade/mixins.jade
                         -var required = nodeFilterKind + ' === "Custom"'
 
                         +java-class('Class name:', customNodeFilter + '.className', '"customNodeFilter"',
-                            'true', required, 'Class name of custom node filter implementation')
+                            'true', required, 'Class name of custom node filter implementation', required)
             .col-sm-6
                 +preview-xml-java(model, 'cacheNodeFilter', 'igfss')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 5062ce1..cfbaf12 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
@@ -53,6 +53,9 @@ include /app/helpers/jade/mixins.jade
                     +number('Long query timeout:', model + '.longQueryWarningTimeout', '"longQueryWarningTimeout"', 'true', '3000', '0',
                         'Timeout in milliseconds after which long query warning will be printed')
                 .settings-row
+                    +number('History size:', model + '.queryDetailMetricsSize', '"queryDetailMetricsSize"', 'true', '0', '0',
+                        'Size of queries detail metrics that will be stored in memory for monitoring purposes')
+                .settings-row
                     -var form = 'querySqlFunctionClasses';
                     -var sqlFunctionClasses = model + '.sqlFunctionClasses';
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 1cf80b8..ea350f2 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
@@ -102,9 +102,9 @@ mixin hibernateField(name, model, items, valid, save, newItem)
                                         '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')
+                                    +java-class('Hasher', pojoStoreFactory + '.hasher', '"pojoHasher"', 'true', 'false', 'Hash calculator', required)
                                 .details-row
-                                    +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer')
+                                    +java-class('Transformer', pojoStoreFactory + '.transformer', '"pojoTransformer"', 'true', 'false', 'Types transformer', required)
                                 .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/>\

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 5cc996d..259909e 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
@@ -19,6 +19,7 @@ include /app/helpers/jade/mixins.jade
 -var form = 'checkpoint'
 -var model = 'backupItem.checkpointSpi'
 -var CustomCheckpoint = 'model.kind === "Custom"'
+-var CacheCheckpoint = 'model.kind === "Cache"'
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")')
@@ -44,7 +45,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{model} && #{model}.length > 0' ng-repeat='model in #{model} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\
+                                +dropdown-required-autofocus('Checkpoint SPI:', 'model.kind', '"checkpointKind" + $index', 'true', 'true', 'Choose checkpoint configuration variant', '[\
                                         {value: "FS", label: "File System"},\
                                         {value: "Cache", label: "Cache"},\
                                         {value: "S3", label: "Amazon S3"},\
@@ -64,13 +65,13 @@ include /app/helpers/jade/mixins.jade
                             div(ng-show='model.kind === "FS"')
                                 include ./checkpoint/fs.jade
 
-                            div(ng-show='model.kind === "Cache"')
+                            div(ng-show=CacheCheckpoint)
                                 .settings-row
-                                    +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', 'true',
+                                    +dropdown-required-empty('Cache:', 'model.Cache.cache', '"checkpointCacheCache"+ $index', 'true', CacheCheckpoint,
                                         'Choose cache', 'No caches configured for current cluster', 'clusterCaches', 'Cache to use for storing checkpoints')
                                 .settings-row
                                     +java-class('Listener:', 'model.Cache.checkpointListener', '"checkpointCacheListener" + $index', 'true', 'false',
-                                        'Checkpoint listener implementation class name')
+                                        'Checkpoint listener implementation class name', CacheCheckpoint)
 
                             div(ng-show='model.kind === "S3"')
                                 include ./checkpoint/s3.jade
@@ -80,6 +81,6 @@ include /app/helpers/jade/mixins.jade
 
                             .settings-row(ng-show=CustomCheckpoint)
                                 +java-class('Class name:', 'model.Custom.className', '"checkpointCustomClassName" + $index', 'true', CustomCheckpoint,
-                                'Custom CheckpointSpi implementation class')
+                                'Custom CheckpointSpi implementation class', CustomCheckpoint)
             .col-sm-6
                 +preview-xml-java('backupItem', 'clusterCheckpoint', 'caches')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 efb6ad0..6ec4535 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
@@ -36,13 +36,13 @@ include /app/helpers/jade/mixins.jade
             -var valid = form + '[' + name + '].$valid'
             -var save = dirPaths + '[$index] = ' + model
 
-            div(ng-repeat='model in #{dirPaths} track by $index' ng-init='obj = {}')
+            div(ng-repeat='item in #{dirPaths} track by $index' ng-init='obj = {}')
                 label.col-xs-12.col-sm-12.col-md-12
                     .indexField
                         | {{ $index+1 }})
-                    +table-remove-conditional-button(dirPaths, 'true', 'Remove path')
+                    +table-remove-conditional-button(dirPaths, 'true', 'Remove path', 'item')
                     span(ng-hide='field.edit')
-                        a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                        a.labelFormField(ng-click='(field.edit = true) && (#{model} = item)') {{ item }}
                     span(ng-if='field.edit')
                         +table-text-field(name, model, dirPaths, valid, save, 'Input directory path', false)
                             +table-save-button(valid, save, false)
@@ -63,4 +63,4 @@ include /app/helpers/jade/mixins.jade
 
 .settings-row
     +java-class('Listener:', 'model.FS.checkpointListener', '"checkpointFsListener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', 'model.kind === "FS"')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 874799c..5a13337 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
@@ -16,15 +16,17 @@
 
 include /app/helpers/jade/mixins.jade
 
+-var jdbcCheckpoint = 'model.kind === "JDBC"'
+
 .settings-row
-    +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', 'model.kind === "JDBC"', 'Input bean name',
+    +text('Data source bean name:', 'model.JDBC.dataSourceBean', '"checkpointJdbcDataSourceBean" + $index', jdbcCheckpoint, 'Input bean name',
     'Name of the data source bean in Spring context')
 .settings-row
-    +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', 'model.kind === "JDBC"',
+    +dialect('Dialect:', 'model.JDBC.dialect', '"checkpointJdbcDialect" + $index', jdbcCheckpoint,
     'Dialect of SQL implemented by a particular RDBMS:', 'Generic JDBC dialect', 'Choose JDBC dialect')
 .settings-row
     +java-class('Listener:', 'model.JDBC.checkpointListener', '"checkpointJdbcListener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', jdbcCheckpoint)
 +showHideLink('jdbcExpanded', 'settings')
     .details-row
         +text('User:', 'model.JDBC.user', '"checkpointJdbcUser" + $index', 'false', 'Input user name', 'Checkpoint jdbc user name')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 da28da7..6531897 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
@@ -18,16 +18,17 @@ include /app/helpers/jade/mixins.jade
 
 -var credentialsModel = 'model.S3.awsCredentials'
 -var clientCfgModel = 'model.S3.clientConfiguration'
--var checkpointS3Path = 'model.S3.awsCredentials.kind === "Properties"'
--var checkpointS3Custom = 'model.S3.awsCredentials.kind === "Custom"'
+-var checkpointS3 = 'model.kind === "S3"'
+-var checkpointS3Path = checkpointS3 + ' && model.S3.awsCredentials.kind === "Properties"'
+-var checkpointS3Custom = checkpointS3 + ' && model.S3.awsCredentials.kind === "Custom"'
 
 -var clientRetryModel = clientCfgModel + '.retryPolicy'
--var checkpointS3DefaultMaxRetry = clientRetryModel + '.kind === "DefaultMaxRetries"'
--var checkpointS3DynamoDbMaxRetry = clientRetryModel + '.kind === "DynamoDBMaxRetries"'
--var checkpointS3CustomRetry = clientRetryModel + '.kind === "Custom"'
+-var checkpointS3DefaultMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DefaultMaxRetries"'
+-var checkpointS3DynamoDbMaxRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "DynamoDBMaxRetries"'
+-var checkpointS3CustomRetry = checkpointS3 + ' && ' + clientRetryModel + '.kind === "Custom"'
 
 .settings-row
-    +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', 'model.kind === "S3"', 'Custom', '[\
+    +dropdown-required('AWS credentials:', 'model.S3.awsCredentials.kind', '"checkpointS3AwsCredentials"', 'true', checkpointS3, 'Custom', '[\
         {value: "Basic", label: "Basic"},\
         {value: "Properties", label: "Properties"},\
         {value: "Anonymous", label: "Anonymous"},\
@@ -51,12 +52,12 @@ include /app/helpers/jade/mixins.jade
 .panel-details(ng-show=checkpointS3Custom)
     .details-row
         +java-class('Class name:', credentialsModel + '.Custom.className', '"checkpointS3CustomClassName" + $index', 'true', checkpointS3Custom,
-        'Custom AWS credentials provider implementation class')
+        'Custom AWS credentials provider implementation class', checkpointS3Custom)
 .settings-row
     +text('Bucket name suffix:', 'model.S3.bucketNameSuffix', '"checkpointS3BucketNameSuffix"', 'false', 'default-bucket', 'Bucket name suffix')
 .settings-row
     +java-class('Listener:', 'model.S3.checkpointListener', '"checkpointS3Listener" + $index', 'true', 'false',
-        'Checkpoint listener implementation class name')
+        'Checkpoint listener implementation class name', checkpointS3)
 +showHideLink('s3Expanded', 'client configuration')
     .details-row
         +dropdown('Protocol:', clientCfgModel + '.protocol', '"checkpointS3Protocol"', 'true', 'HTTPS', '[\
@@ -121,10 +122,10 @@ include /app/helpers/jade/mixins.jade
     .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')
+            'Retry condition on whether a specific request and exception should be retried', checkpointS3CustomRetry)
         .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')
+            'Back-off strategy for controlling how long the next retry should wait', checkpointS3CustomRetry)
         .details-row
             +number-required('Maximum retry attempts:', clientRetryModel + '.Custom.maxErrorRetry', '"checkpointS3CustomMaxErrorRetry"', 'true', checkpointS3CustomRetry, '-1', '1',
             'Maximum number of retry attempts for failed requests')
@@ -159,13 +160,13 @@ include /app/helpers/jade/mixins.jade
         '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')
+        'DNS Resolver that should be used to for resolving AWS IP addresses', checkpointS3)
     .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')
+        'SecureRandom to be used by the SDK class name', checkpointS3)
     .details-row
         +checkbox('Use reaper', clientCfgModel + '.useReaper', '"checkpointS3UseReaper"', 'Checks if the IdleConnectionReaper is to be started')
     .details-row

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 31a6be7..8e77ac4 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
@@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade
 
 div
     .details-row
-        +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class')
+        +java-class('Class:', model + '.class', '"collisionCustom"', 'true', required, 'CollisionSpi implementation class', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 d4e537a..dbe0478 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
@@ -37,7 +37,7 @@ div
             'Node should attempt to steal jobs from other nodes')
     .details-row
         +java-class('External listener:', model + '.externalCollisionListener', '"jsExternalCollisionListener"', 'true', 'false',
-            'Listener to be set for notification of external collision events')
+            'Listener to be set for notification of external collision events', 'backupItem.collision.kind === "JobStealing"')
     .details-row
         +ignite-form-group
             ignite-form-field-label

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 4cfd9f5..aa99b49 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
@@ -18,8 +18,14 @@ include /app/helpers/jade/mixins.jade
 
 -var form = 'deployment'
 -var model = 'backupItem'
+-var modelDeployment = 'backupItem.deploymentSpi'
 -var exclude = model + '.peerClassLoadingLocalClassPathExclude'
 -var enabled = 'backupItem.peerClassLoadingEnabled'
+-var uriListModel = modelDeployment + '.URI.uriList'
+-var scannerModel = modelDeployment + '.URI.scanners'
+-var uriDeployment = modelDeployment + '.kind === "URI"'
+-var localDeployment = modelDeployment + '.kind === "Local"'
+-var customDeployment = modelDeployment + '.kind === "Custom"'
 
 .panel.panel-default(ng-form=form novalidate)
     .panel-heading(bs-collapse-toggle ng-click='ui.loadPanel("#{form}")')
@@ -57,7 +63,7 @@ include /app/helpers/jade/mixins.jade
                 .settings-row
                     +number('Pool size:', model + '.peerClassLoadingThreadPoolSize', '"peerClassLoadingThreadPoolSize"', enabled, '2', '1', 'Thread pool size to use for peer class loading')
                 .settings-row
-                    +ignite-form-group(ng-model=exclude ng-form=form)
+                    +ignite-form-group
                         -var uniqueTip = 'Such package already exists'
 
                         ignite-form-field-label
@@ -81,7 +87,7 @@ include /app/helpers/jade/mixins.jade
                                             | {{ $index+1 }})
                                         +table-remove-button(exclude, 'Remove package name')
                                         span(ng-hide='field.edit')
-                                            a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }}
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
                                         span(ng-if='field.edit')
                                             +table-java-package-field(name, model, exclude, valid, save, false)
                                                 +table-save-button(valid, save, false)
@@ -107,8 +113,125 @@ include /app/helpers/jade/mixins.jade
                                         +table-save-button(valid, save, true)
                                         +unique-feedback(name, uniqueTip)
 
-
                         .group-content-empty(ng-if='!(#{exclude}.length) && !group.add.length')
                             | Not defined
+                .settings-row
+                    +dropdown('Deployment variant:', modelDeployment + '.kind', '"deploymentKind"', 'true', 'Default',
+                        '[\
+                            {value: "URI", label: "URI"},\
+                            {value: "Local", label: "Local"}, \
+                            {value: "Custom", label: "Custom"},\
+                            {value: undefined, label: "Default"}\
+                        ]',
+                        'Grid deployment SPI is in charge of deploying tasks and classes from different sources:\
+                        <ul>\
+                            <li>URI - Deploy tasks from different sources like file system folders, email and HTTP</li>\
+                            <li>Local - Only within VM deployment on local node</li>\
+                            <li>Custom - Custom implementation of DeploymentSpi</li>\
+                            <li>Default - Default configuration of LocalDeploymentSpi will be used</li>\
+                        </ul>')
+                .panel-details(ng-show=uriDeployment)
+                    .details-row
+                        +ignite-form-group()
+                            -var uniqueTip = 'Such URI already configured'
+
+                            ignite-form-field-label
+                                | URI list
+                            ignite-form-group-tooltip
+                                | List of URI which point to GAR file and which should be scanned by SPI for the new tasks
+                            ignite-form-group-add(ng-click='(group.add = [{}])')
+                                | Add URI.
+
+                            .group-content(ng-if=uriListModel + '.length')
+                                -var model = 'obj.model';
+                                -var name = '"edit" + $index'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = uriListModel + '[$index] = ' + model
+
+                                div(ng-repeat='model in #{uriListModel} track by $index' ng-init='obj = {}')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        .indexField
+                                            | {{ $index+1 }})
+                                        +table-remove-button(uriListModel, 'Remove URI')
+                                        span(ng-hide='field.edit')
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                                        span(ng-if='field.edit')
+                                            +table-url-field(name, model, uriListModel, valid, save, false)
+                                                +table-save-button(valid, save, false)
+                                                +unique-feedback(name, uniqueTip)
+
+                            .group-content(ng-repeat='field in group.add')
+                                -var model = 'new';
+                                -var name = '"new"'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = uriListModel + '.push(' + model + ')'
+
+                                div(type='internal' name='URI')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        +table-url-field(name, model, uriListModel, valid, save, true)
+                                            +table-save-button(valid, save, true)
+                                            +unique-feedback(name, uniqueTip)
+
+                            .group-content-empty(ng-if='!(#{uriListModel}.length) && !group.add.length')
+                                | Not defined
+                    .details-row
+                        +text('Temporary directory path:', modelDeployment + '.URI.temporaryDirectoryPath', '"DeploymentURITemporaryDirectoryPath"', 'false', 'Temporary directory path',
+                        'Absolute path to temporary directory which will be used by deployment SPI to keep all deployed classes in')
+                    .details-row
+                        +ignite-form-group()
+                            -var uniqueTip = 'Such scanner already configured'
+
+                            ignite-form-field-label
+                                | Scanner list
+                            ignite-form-group-tooltip
+                                | List of URI deployment scanners
+                            ignite-form-group-add(ng-click='(group.add = [{}])')
+                                | Add scanner
+
+                            .group-content(ng-if=scannerModel + '.length')
+                                -var model = 'obj.model';
+                                -var name = '"edit" + $index'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = scannerModel + '[$index] = ' + model
+
+                                div(ng-repeat='model in #{scannerModel} track by $index' ng-init='obj = {}')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        .indexField
+                                            | {{ $index+1 }})
+                                        +table-remove-button(scannerModel, 'Remove scanner')
+                                        span(ng-hide='field.edit')
+                                            a.labelFormField(ng-click='(field.edit = true) && (#{model} = model)') {{ model }}
+                                        span(ng-if='field.edit')
+                                            +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, false)
+                                                +table-save-button(valid, save, false)
+                                                +unique-feedback(name, uniqueTip)
+
+                            .group-content(ng-repeat='field in group.add')
+                                -var model = 'new';
+                                -var name = '"new"'
+                                -var valid = form + '[' + name + '].$valid'
+                                -var save = scannerModel + '.push(' + model + ')'
+
+                                div(type='internal' name='Scanner')
+                                    label.col-xs-12.col-sm-12.col-md-12
+                                        // (lbl, name, model, items, valid, save, newItem)
+                                        +table-java-class-field('Scanner:', name, model, scannerModel, valid, save, true)
+                                            +table-save-button(valid, save, true)
+                                            +unique-feedback(name, uniqueTip)
+
+                            .group-content-empty(ng-if='!(#{scannerModel}.length) && !group.add.length')
+                                | Not defined
+                    .details-row
+                        +java-class('Listener:', modelDeployment + '.URI.listener', '"DeploymentURIListener"', 'true', 'false', 'Deployment event listener', uriDeployment)
+                    .details-row
+                        +checkbox('Check MD5', modelDeployment + '.URI.checkMd5', '"DeploymentURICheckMd5"', 'Exclude files with same md5s from deployment')
+                    .details-row
+                        +checkbox('Encode URI', modelDeployment + '.URI.encodeUri', '"DeploymentURIEncodeUri"', 'URI must be encoded before usage')
+                .panel-details(ng-show=localDeployment)
+                    .details-row
+                        +java-class('Listener:', modelDeployment + '.Local.listener', '"DeploymentLocalListener"', 'true', 'false', 'Deployment event listener', localDeployment)
+                .panel-details(ng-show=customDeployment)
+                    .details-row
+                        +java-class('Class:', modelDeployment + '.Custom.className', '"DeploymentCustom"', 'true', customDeployment, 'DeploymentSpi implementation class', customDeployment)
             .col-sm-6
                 +preview-xml-java(model, 'clusterDeployment')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 3f2d6cb..643ea97 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
@@ -59,10 +59,10 @@ include /app/helpers/jade/mixins.jade
                     .settings-row
                         +java-class('Filter:', modelEventStorage + '.Memory.filter', '"EventStorageFilter"', 'true', 'false',
                         'Filter for events to be recorded<br/>\
-                        Should be implementation of o.a.i.lang.IgnitePredicate&lt;o.a.i.events.Event&gt;')
+                        Should be implementation of o.a.i.lang.IgnitePredicate&lt;o.a.i.events.Event&gt;', eventStorageMemory)
 
                 .settings-row(ng-show=eventStorageCustom)
-                    +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name')
+                    +java-class('Class:', modelEventStorage + '.Custom.className', '"EventStorageCustom"', 'true', eventStorageCustom, 'Event storage implementation class name', eventStorageCustom)
 
             .col-sm-6
                 +preview-xml-java(model, 'clusterEvents')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 aaed8e9..1665659 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
@@ -45,7 +45,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{failoverSpi} && #{failoverSpi}.length > 0' ng-repeat='model in #{failoverSpi} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\
+                                +dropdown-required-autofocus('Failover SPI:', 'model.kind', '"failoverKind" + $index', 'true', 'true', 'Choose Failover SPI', '[\
                                         {value: "JobStealing", label: "Job stealing"},\
                                         {value: "Never", label: "Never"},\
                                         {value: "Always", label: "Always"},\
@@ -68,6 +68,6 @@ include /app/helpers/jade/mixins.jade
                                     'Maximum number of attempts to execute a failed job on another node')
                             .settings-row(ng-show=failoverCustom)
                                 +java-class('SPI implementation', 'model.Custom.class', '"failoverSpiClass" + $index', 'true', failoverCustom,
-                                    'Custom FailoverSpi implementation class name.')
+                                    'Custom FailoverSpi implementation class name.', failoverCustom)
             .col-sm-6
                 +preview-xml-java(model, 'clusterFailover')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 2e567ed..48b1776 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
@@ -27,7 +27,7 @@ div
         +java-class('Curator:', model + '.curator', '"curator"', 'true', 'false',
             'The Curator framework in use<br/>\
             By default generates curator of org.apache.curator. framework.imps.CuratorFrameworkImpl\
-            class with configured connect string, retry policy, and default session and connection timeouts')
+            class with configured connect string, retry policy, and default session and connection timeouts', required)
     .details-row
         +text('Connect string:', model + '.zkConnectionString', '"' + discoveryKind + 'ConnectionString"', required, 'host:port[chroot][,host:port[chroot]]',
             'When "IGNITE_ZK_CONNECTION_STRING" system property is not configured this property will be used')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 5a03de8..5db89f5 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
@@ -21,4 +21,4 @@ include /app/helpers/jade/mixins.jade
 -var required = 'backupItem.discovery.kind === "ZooKeeper" && backupItem.discovery.ZooKeeper.retryPolicy.kind === "Custom"'
 
 .details-row
-    +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name')
+    +java-class('Class name:', retry + '.className', '"customClassName"', 'true', required, 'Custom retry policy implementation class name', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 7fd78bf..9fa9fc9 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
@@ -46,7 +46,7 @@ include /app/helpers/jade/mixins.jade
                         .group-content(ng-show='#{loadBalancingSpi} && #{loadBalancingSpi}.length > 0' ng-repeat='model in #{loadBalancingSpi} track by $index')
                             hr(ng-if='$index != 0')
                             .settings-row
-                                +dropdown-required('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\
+                                +dropdown-required-autofocus('Load balancing:', 'model.kind', '"loadBalancingKind" + $index', 'true', 'true', 'Choose load balancing SPI', '[\
                                         {value: "RoundRobin", label: "Round-robin"},\
                                         {value: "Adaptive", label: "Adaptive"},\
                                         {value: "WeightedRandom", label: "Random"},\
@@ -78,27 +78,30 @@ include /app/helpers/jade/mixins.jade
                                         <li>Default - Default load probing implementation</li>\
                                     </ul>')
                             .settings-row(ng-show='model.kind === "Adaptive" && model.Adaptive.loadProbe.kind')
-                                .panel-details
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "Job"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "Job"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.Job.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average CPU load vs. current')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.CPU.useAverage', '"loadBalancingAdaptiveCPUUseAverage" + $index', 'Use average CPU load vs. current')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +checkbox('Use processors', 'model.Adaptive.loadProbe.CPU.useProcessors', '"loadBalancingAdaptiveCPUUseProcessors" + $index', "divide each node's CPU load by the number of processors on that node")
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "CPU"')
+                                    .details-row
                                         +number-min-max-step('Processor coefficient:', 'model.Adaptive.loadProbe.CPU.processorCoefficient',
                                             '"loadBalancingAdaptiveCPUProcessorCoefficient" + $index', 'true', '1', '0.001', '1', '0.05', 'Coefficient of every CPU')
-                                    .details-row(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"')
+                                .panel-details(ng-show='model.Adaptive.loadProbe.kind === "ProcessingTime"')
+                                    .details-row
                                         +checkbox('Use average', 'model.Adaptive.loadProbe.ProcessingTime.useAverage', '"loadBalancingAdaptiveJobUseAverage" + $index', 'Use average execution time vs. current')
-                                    .details-row(ng-show=loadProbeCustom)
+                                .panel-details(ng-show=loadProbeCustom)
+                                    .details-row
                                         +java-class('Load brobe implementation:', 'model.Adaptive.loadProbe.Custom.className', '"loadBalancingAdaptiveJobUseClass" + $index', 'true', loadProbeCustom,
-                                            'Custom load balancing SPI implementation class name.')
+                                            'Custom load balancing SPI implementation class name.', loadProbeCustom)
                             .settings-row(ng-show='model.kind === "WeightedRandom"')
                                 +number('Node weight:', 'model.WeightedRandom.nodeWeight', '"loadBalancingWRNodeWeight" + $index', 'true', 10, '1', 'Weight of node')
                             .settings-row(ng-show='model.kind === "WeightedRandom"')
                                 +checkbox('Use weights', 'model.WeightedRandom.useWeights', '"loadBalancingWRUseWeights" + $index', 'Node weights should be checked when doing random load balancing')
                             .settings-row(ng-show=loadBalancingCustom)
                                 +java-class('Load balancing SPI implementation:', 'model.Custom.className', '"loadBalancingClass" + $index', 'true', loadBalancingCustom,
-                                    'Custom load balancing SPI implementation class name.')
+                                    'Custom load balancing SPI implementation class name.', loadBalancingCustom)
             .col-sm-6
                 +preview-xml-java(model, 'clusterLoadBalancing')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 385d647..87d2b7d 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
@@ -22,4 +22,4 @@ include /app/helpers/jade/mixins.jade
 
 div
     .details-row
-        +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name')
+        +java-class('Class:', model + '.class', '"customLogger"', 'true', required, 'Logger implementation class name', required)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 85ec073..fbd979c 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
@@ -72,7 +72,7 @@ include /app/helpers/jade/mixins.jade
                                     label.col-xs-12.col-sm-12.col-md-12
                                         .indexField
                                             | {{ $index+1 }})
-                                        +table-remove-conditional-button(trust, enabled, 'Remove trust manager')
+                                        +table-remove-conditional-button(trust, enabled, 'Remove trust manager', 'model')
                                         span(ng-hide='field.edit')
                                             a.labelFormField(ng-click='#{enabled} && (field.edit = true) && (#{model} = model)') {{ model }}
                                         span(ng-if='field.edit')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
new file mode 100644
index 0000000..08cfa71
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary-zipper.service.js
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import Worker from 'worker?inline=true!./summary.worker';
+
+export default ['$q', function($q) {
+    return function({ cluster, data }) {
+        const defer = $q.defer();
+        const worker = new Worker();
+
+        worker.postMessage({ cluster, data });
+
+        worker.onmessage = (e) => {
+            defer.resolve(e.data);
+        };
+
+        worker.onerror = (err) => {
+            defer.reject(err);
+        };
+
+        return defer.promise;
+    };
+}];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 d739c43..cfc6df9 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
@@ -16,15 +16,19 @@
  */
 
 import _ from 'lodash';
-import JSZip from 'jszip';
 import saver from 'file-saver';
 
+const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+
 export default [
-    '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'GeneratorDocker', 'GeneratorPom', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils',
-    function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils) {
+    '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteLoading', '$filter', 'IgniteConfigurationResource', 'JavaTypes', 'IgniteVersion', 'IgniteConfigurationGenerator', 'SpringTransformer', 'JavaTransformer', 'IgniteDockerGenerator', 'IgniteMavenGenerator', 'IgnitePropertiesGenerator', 'IgniteReadmeGenerator', 'IgniteFormUtils', 'IgniteSummaryZipper',
+    function($root, $scope, $http, LegacyUtils, Messages, Loading, $filter, Resource, JavaTypes, Version, generator, spring, java, docker, pom, propsGenerator, readme, FormUtils, SummaryZipper) {
         const ctrl = this;
 
-        $scope.ui = { ready: false };
+        $scope.ui = {
+            isSafari: !!(/constructor/i.test(window.HTMLElement) || window.safari),
+            ready: false
+        };
 
         Loading.start('summaryPage');
 
@@ -223,10 +227,6 @@ export default [
             return false;
         }
 
-        function escapeFileName(name) {
-            return name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
-        }
-
         $scope.selectItem = (cluster) => {
             delete ctrl.cluster;
 
@@ -297,84 +297,19 @@ export default [
 
         // TODO IGNITE-2114: implemented as independent logic for download.
         $scope.downloadConfiguration = function() {
-            const cluster = $scope.cluster;
-
-            const zip = new JSZip();
-
-            if (!ctrl.data)
-                ctrl.data = {};
-
-            if (!ctrl.data.docker)
-                ctrl.data.docker = docker.generate(cluster, 'latest');
-
-            zip.file('Dockerfile', ctrl.data.docker);
-            zip.file('.dockerignore', docker.ignoreFile());
-
-            const cfg = generator.igniteConfiguration(cluster, false);
-            const clientCfg = generator.igniteConfiguration(cluster, true);
-            const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled'));
-
-            const secProps = propsGenerator.generate(cfg);
-
-            if (secProps)
-                zip.file('src/main/resources/secret.properties', secProps);
-
-            const srcPath = 'src/main/java';
-            const resourcesPath = 'src/main/resources';
-
-            const serverXml = `${escapeFileName(cluster.name)}-server.xml`;
-            const clientXml = `${escapeFileName(cluster.name)}-client.xml`;
-
-            const metaPath = `${resourcesPath}/META-INF`;
-
-            zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString());
-            zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString());
-
-            const cfgPath = `${srcPath}/config`;
-
-            zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').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',
-                    'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
-            }
-
-            // Generate loader for caches with configured store.
-            const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory));
-
-            if (_.nonEmpty(cachesToLoad))
-                zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`));
-
-            const startupPath = `${srcPath}/startup`;
-
-            zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`));
-            zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`));
-
-            zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup',
-                'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
-            zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup',
-                'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches));
-
-            zip.file('pom.xml', pom.generate(cluster, Version.productVersion().ignite).asString());
-
-            zip.file('README.txt', readme.generate());
-            zip.file('jdbc-drivers/README.txt', readme.generateJDBC());
-
-            if (_.isEmpty(ctrl.data.pojos))
-                ctrl.data.pojos = java.pojos(cluster.caches);
-
-            for (const pojo of ctrl.data.pojos) {
-                if (pojo.keyClass && JavaTypes.nonBuiltInClass(pojo.keyType))
-                    zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass);
+            if ($scope.isPrepareDownloading)
+                return;
 
-                zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass);
-            }
+            const cluster = $scope.cluster;
 
-            $generatorOptional.optionalContent(zip, cluster);
+            $scope.isPrepareDownloading = true;
 
-            zip.generateAsync({type: 'blob', compression: 'DEFLATE', mimeType: 'application/octet-stream'})
-                .then((blob) => saver.saveAs(blob, escapeFileName(cluster.name) + '-project.zip'));
+            return new SummaryZipper({ cluster, data: ctrl.data || {}, IgniteDemoMode: $root.IgniteDemoMode })
+                .then((data) => {
+                    saver.saveAs(data, escapeFileName(cluster.name) + '-project.zip');
+                })
+                .catch((err) => Messages.showError('Failed to generate project files. ' + err.message))
+                .then(() => $scope.isPrepareDownloading = false);
         };
 
         /**
@@ -393,7 +328,7 @@ export default [
             const dialects = $scope.dialects;
 
             if (dialects.Oracle)
-                window.open('http://www.oracle.com/technetwork/apps-tech/jdbc-112010-090769.html');
+                window.open('http://www.oracle.com/technetwork/database/features/jdbc/default-2280470.html');
 
             if (dialects.DB2)
                 window.open('http://www-01.ibm.com/support/docview.wss?uid=swg21363866');

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
new file mode 100644
index 0000000..6b24001
--- /dev/null
+++ b/modules/web-console/frontend/app/modules/states/configuration/summary/summary.worker.js
@@ -0,0 +1,123 @@
+/*
+ * 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 JSZip from 'jszip';
+
+import IgniteVersion from 'app/modules/configuration/Version.service';
+
+import MavenGenerator from 'app/modules/configuration/generator/Maven.service';
+import DockerGenerator from 'app/modules/configuration/generator/Docker.service';
+import ReadmeGenerator from 'app/modules/configuration/generator/Readme.service';
+import PropertiesGenerator from 'app/modules/configuration/generator/Properties.service';
+import ConfigurationGenerator from 'app/modules/configuration/generator/ConfigurationGenerator';
+
+import JavaTransformer from 'app/modules/configuration/generator/JavaTransformer.service';
+import SpringTransformer from 'app/modules/configuration/generator/SpringTransformer.service';
+
+const Version = new IgniteVersion();
+
+const maven = new MavenGenerator();
+const docker = new DockerGenerator();
+const readme = new ReadmeGenerator();
+const properties = new PropertiesGenerator();
+
+const java = new JavaTransformer[0]();
+const spring = new SpringTransformer[0]();
+
+const generator = new ConfigurationGenerator[0]();
+
+const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+
+// eslint-disable-next-line no-undef
+onmessage = function(e) {
+    const {cluster, data, demo} = e.data;
+
+    const zip = new JSZip();
+
+    if (!data.docker)
+        data.docker = docker.generate(cluster, 'latest');
+
+    zip.file('Dockerfile', data.docker);
+    zip.file('.dockerignore', docker.ignoreFile());
+
+    const cfg = generator.igniteConfiguration(cluster, false);
+    const clientCfg = generator.igniteConfiguration(cluster, true);
+    const clientNearCaches = _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled'));
+
+    const secProps = properties.generate(cfg);
+
+    if (secProps)
+        zip.file('src/main/resources/secret.properties', secProps);
+
+    const srcPath = 'src/main/java';
+    const resourcesPath = 'src/main/resources';
+
+    const serverXml = `${escapeFileName(cluster.name)}-server.xml`;
+    const clientXml = `${escapeFileName(cluster.name)}-client.xml`;
+
+    const metaPath = `${resourcesPath}/META-INF`;
+
+    zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg).asString());
+    zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, clientNearCaches).asString());
+
+    const cfgPath = `${srcPath}/config`;
+
+    zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, 'config', 'ServerConfigurationFactory').asString());
+    zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, 'config', 'ClientConfigurationFactory', clientNearCaches).asString());
+
+    if (java.isDemoConfigured(cluster, demo)) {
+        zip.file(`${srcPath}/demo/DemoStartup.java`, java.nodeStartup(cluster, 'demo.DemoStartup',
+            'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
+    }
+
+    // Generate loader for caches with configured store.
+    const cachesToLoad = _.filter(cluster.caches, (cache) => _.nonNil(cache.cacheStoreFactory));
+
+    if (_.nonEmpty(cachesToLoad))
+        zip.file(`${srcPath}/load/LoadCaches.java`, java.loadCaches(cachesToLoad, 'load', 'LoadCaches', `"${clientXml}"`));
+
+    const startupPath = `${srcPath}/startup`;
+
+    zip.file(`${startupPath}/ServerNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeSpringStartup', `"${serverXml}"`));
+    zip.file(`${startupPath}/ClientNodeSpringStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeSpringStartup', `"${clientXml}"`));
+
+    zip.file(`${startupPath}/ServerNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ServerNodeCodeStartup',
+        'ServerConfigurationFactory.createConfiguration()', 'config.ServerConfigurationFactory'));
+    zip.file(`${startupPath}/ClientNodeCodeStartup.java`, java.nodeStartup(cluster, 'startup.ClientNodeCodeStartup',
+        'ClientConfigurationFactory.createConfiguration()', 'config.ClientConfigurationFactory', clientNearCaches));
+
+    zip.file('pom.xml', maven.generate(cluster, Version.productVersion().ignite).asString());
+
+    zip.file('README.txt', readme.generate());
+    zip.file('jdbc-drivers/README.txt', readme.generateJDBC());
+
+    if (_.isEmpty(data.pojos))
+        data.pojos = java.pojos(cluster.caches);
+
+    for (const pojo of data.pojos) {
+        if (pojo.keyClass)
+            zip.file(`${srcPath}/${pojo.keyType.replace(/\./g, '/')}.java`, pojo.keyClass);
+
+        zip.file(`${srcPath}/${pojo.valueType.replace(/\./g, '/')}.java`, pojo.valueClass);
+    }
+
+    zip.generateAsync({
+        type: 'blob',
+        compression: 'DEFLATE',
+        mimeType: 'application/octet-stream'
+    }).then((blob) => postMessage(blob));
+};

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/user/Auth.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/user/Auth.service.js b/modules/web-console/frontend/app/modules/user/Auth.service.js
index 43e2f92..e0f905d 100644
--- a/modules/web-console/frontend/app/modules/user/Auth.service.js
+++ b/modules/web-console/frontend/app/modules/user/Auth.service.js
@@ -20,12 +20,11 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro
         return {
             forgotPassword(userInfo) {
                 $http.post('/api/v1/password/forgot', userInfo)
-                    .success(() => $state.go('password.send'))
-                    .error((err) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, err)));
+                    .then(() => $state.go('password.send'))
+                    .cacth(({data}) => ErrorPopover.show('forgot_email', Messages.errorMessage(null, data)));
             },
             auth(action, userInfo) {
                 $http.post('/api/v1/' + action, userInfo)
-                    .catch(({data}) => Promise.reject(data))
                     .then(() => {
                         if (action === 'password/forgot')
                             return;
@@ -41,16 +40,16 @@ export default ['Auth', ['$http', '$rootScope', '$state', '$window', 'IgniteErro
                                 $root.gettingStarted.tryShow();
                             });
                     })
-                    .catch((err) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, err)));
+                    .catch((res) => ErrorPopover.show(action + '_email', Messages.errorMessage(null, res)));
             },
             logout() {
                 $http.post('/api/v1/logout')
-                    .success(() => {
+                    .then(() => {
                         User.clean();
 
                         $window.open($state.href('signin'), '_self');
                     })
-                    .error(Messages.showError);
+                    .catch(Messages.showError);
             }
         };
     }]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/services/JavaTypes.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/JavaTypes.service.js b/modules/web-console/frontend/app/services/JavaTypes.service.js
index 679914f..944fea5 100644
--- a/modules/web-console/frontend/app/services/JavaTypes.service.js
+++ b/modules/web-console/frontend/app/services/JavaTypes.service.js
@@ -40,7 +40,7 @@ const VALID_UUID = /^[0-9a-f]{8}-[0-9a-f]{4}-[1-5][0-9a-f]{3}-[89ab][0-9a-f]{3}-
  * Utility service for various check on java types.
  */
 export default class JavaTypes {
-    static $inject = ['igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults'];
+    static $inject = ['IgniteClusterDefaults', 'IgniteCacheDefaults', 'IgniteIGFSDefaults'];
 
     constructor(clusterDflts, cacheDflts, igfsDflts) {
         this.enumClasses = _.uniq(this._enumClassesAcc(_.merge(clusterDflts, cacheDflts, igfsDflts), []));
@@ -101,14 +101,9 @@ export default class JavaTypes {
      * @return {String} Class name.
      */
     shortClassName(clsName) {
-        if (this.isJavaPrimitive(clsName))
-            return clsName;
+        const dotIdx = clsName.lastIndexOf('.');
 
-        const fullClsName = this.fullClassName(clsName);
-
-        const dotIdx = fullClsName.lastIndexOf('.');
-
-        return dotIdx > 0 ? fullClsName.substr(dotIdx + 1) : fullClsName;
+        return dotIdx > 0 ? clsName.substr(dotIdx + 1) : clsName;
     }
 
     /**
@@ -163,7 +158,7 @@ export default class JavaTypes {
      * @param {String} clsName Class name to check.
      * @returns {boolean} 'true' if given class name is java primitive.
      */
-    isJavaPrimitive(clsName) {
+    isPrimitive(clsName) {
         return _.includes(JAVA_PRIMITIVES, clsName);
     }
 


[44/50] [abbrv] ignite git commit: Fixed RAT.

Posted by vo...@apache.org.
Fixed RAT.


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

Branch: refs/heads/ignite-2.0
Commit: dbc8a0f832e7cebf362dd4818947df65075126ee
Parents: 8e7c852
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Dec 23 16:34:21 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Dec 23 16:34:21 2016 +0700

----------------------------------------------------------------------
 modules/platforms/dotnet/build.bat | 11 ++++++++---
 1 file changed, 8 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/dbc8a0f8/modules/platforms/dotnet/build.bat
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.bat b/modules/platforms/dotnet/build.bat
index ef0a96f..5e70c27 100644
--- a/modules/platforms/dotnet/build.bat
+++ b/modules/platforms/dotnet/build.bat
@@ -1,6 +1,10 @@
-:: Licensed 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
+::
+:: 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
 ::
@@ -9,6 +13,7 @@
 :: 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.
+::
 
 :: Apache Ignite.NET build script runner, see build.ps1 for more details.
 powershell -executionpolicy remotesigned -file build.ps1 %*
\ No newline at end of file


[06/50] [abbrv] ignite git commit: IGNITE-4277: Hadoop: better property naming for "partially raw" comparator (2).

Posted by vo...@apache.org.
IGNITE-4277: Hadoop: better property naming for "partially raw" comparator (2).


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

Branch: refs/heads/ignite-2.0
Commit: 57eb47f9c057d15739987500a8b3894d1d739d29
Parents: f8ac0f1
Author: devozerov <vo...@gridgain.com>
Authored: Thu Dec 15 12:29:51 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:56 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/hadoop/HadoopClassLoader.java       | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/57eb47f9/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index 81c1405..1209e74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -372,7 +372,7 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
             // We use "contains" instead of "equals" to handle subclasses properly.
             if (clsName.contains("org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem") ||
                 clsName.contains("org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem") ||
-                clsName.contains("org.apache.ignite.hadoop.io.TextPartialRawComparator") ||
+                clsName.contains("org.apache.ignite.hadoop.io.TextPartiallyRawComparator") ||
                 clsName.contains("org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider"))
                 return true;
         }


[47/50] [abbrv] ignite git commit: Fixed broken links.

Posted by vo...@apache.org.
Fixed broken links.


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

Branch: refs/heads/ignite-2.0
Commit: 6ca86704c36029ff29f11e1dba10588b341aa31a
Parents: fdf1f4b
Author: Vasiliy Sisko <vs...@gridgain.com>
Authored: Fri Dec 23 18:58:47 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 23 18:58:47 2016 +0700

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteLogger.java | 6 +++---
 .../main/java/org/apache/ignite/logger/java/JavaLogger.java    | 4 ++--
 .../testframework/junits/logger/GridTestLog4jLogger.java       | 4 ++--
 .../main/java/org/apache/ignite/logger/log4j/Log4JLogger.java  | 4 ++--
 .../org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java | 2 +-
 5 files changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
index f3afa99..a52b49b 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteLogger.java
@@ -23,8 +23,8 @@ import org.jetbrains.annotations.Nullable;
 /**
  * This interface defines basic logging functionality used throughout the system. We had to
  * abstract it out so that we can use whatever logging is used by the hosting environment.
- * Currently, <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>,
- * <a target=_new href="http://www.jboss.org/developers/guides/logging">JBoss</a>,
+ * Currently, <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>,
+ * <a target=_new href="http://docs.jboss.org/hibernate/orm/4.3/topical/html/logging/Logging">JBoss</a>,
  * <a target=_new href="http://jakarta.apache.org/commons/logging/">JCL</a> and
  * console logging are provided as supported implementations.
  * <p>
@@ -158,4 +158,4 @@ public interface IgniteLogger {
      * @return Name of the file being logged to if one is configured or {@code null} otherwise.
      */
     public String fileName();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
index d5ff5e3..6aa7d38 100644
--- a/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
+++ b/modules/core/src/main/java/org/apache/ignite/logger/java/JavaLogger.java
@@ -86,7 +86,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
  *      ...
  *      cfg.setGridLogger(log);
  * </pre>
- * Please take a look at <a target=_new href="http://java.sun.com/j2se/1.4.2/docs/api20/java/util/logging/Logger.html">Logger javadoc</a>
+ * Please take a look at <a target=_new href="http://docs.oracle.com/javase/7/docs/api/java/util/logging/Logger.html">Logger javadoc</a>
  * for additional information.
  * <p>
  * It's recommended to use Ignite logger injection instead of using/instantiating
@@ -406,4 +406,4 @@ public class JavaLogger implements IgniteLogger, LoggerNodeIdAware {
 
         return null;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
index 74f5160..6a46c7d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/logger/GridTestLog4jLogger.java
@@ -50,7 +50,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
 
 /**
  * Log4j-based implementation for logging. This logger should be used
- * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>-based logging.
+ * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>-based logging.
  * <p>
  * Here is a typical example of configuring log4j logger in Ignite configuration file:
  * <pre name="code" class="xml">
@@ -521,4 +521,4 @@ public class GridTestLog4jLogger implements IgniteLogger, LoggerNodeIdAware {
     @Override public String toString() {
         return S.toString(GridTestLog4jLogger.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
----------------------------------------------------------------------
diff --git a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
index eaae2d4..d5b0f02 100644
--- a/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
+++ b/modules/log4j/src/main/java/org/apache/ignite/logger/log4j/Log4JLogger.java
@@ -50,7 +50,7 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_QUIET;
 
 /**
  * Log4j-based implementation for logging. This logger should be used
- * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/docs/">log4j</a>-based logging.
+ * by loaders that have prefer <a target=_new href="http://logging.apache.org/log4j/1.2/">log4j</a>-based logging.
  * <p>
  * Here is a typical example of configuring log4j logger in Ignite configuration file:
  * <pre name="code" class="xml">
@@ -532,4 +532,4 @@ public class Log4JLogger implements IgniteLogger, LoggerNodeIdAware, Log4jFileAw
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ca86704/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
index 22d42db..30940e4 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/UriDeploymentSpi.java
@@ -111,7 +111,7 @@ import org.jetbrains.annotations.Nullable;
  * {@code META-INF/} entry may contain {@code ignite.xml} file which is a
  * task descriptor file. The purpose of task descriptor XML file is to specify
  * all tasks to be deployed. This file is a regular
- * <a href="http://www.springframework.org/documentation">Spring</a> XML
+ * <a href="https://spring.io/docs">Spring</a> XML
  * definition file.  {@code META-INF/} entry may also contain any other file
  * specified by JAR format.
  * </li>


[15/50] [abbrv] ignite git commit: ignite-2412 Do not call 'asyncOp' for synchronous operations

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index a8219b0..4350b3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -450,61 +450,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return dht.getAndPutIfAbsent(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        return dht.getAndPutIfAbsentAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return dht.putIfAbsent(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        return dht.putIfAbsentAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
     @Nullable @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException {
         return dht.tryGetAndPut(key, val);
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return dht.getAndReplace(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        return dht.getAndReplaceAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return dht.replace(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        return dht.replaceAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        return dht.replace(key, oldVal, newVal);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        return dht.replaceAsync(key, oldVal, newVal);
-    }
-
-    /** {@inheritDoc} */
     @Override public void putAll(Map<? extends K, ? extends V> m)
         throws IgniteCheckedException {
         dht.putAll(m);
@@ -569,6 +519,11 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean remove(K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
+        return dht.remove(key, filter);
+    }
+
+    /** {@inheritDoc} */
     @Override public V getAndRemove(K key) throws IgniteCheckedException {
         return dht.getAndRemove(key);
     }
@@ -602,16 +557,6 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        return dht.remove(key, val);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        return dht.removeAsync(key, val);
-    }
-
-    /** {@inheritDoc} */
     @Override public void removeAll() throws IgniteCheckedException {
         dht.removeAll();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index bc16ff4..a26d2f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -58,7 +58,6 @@ import org.apache.ignite.internal.processors.cache.local.GridLocalCache;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalEx;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.resource.GridResourceIoc;
-import org.apache.ignite.internal.processors.resource.GridResourceProcessor;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridEmbeddedFuture;
@@ -108,6 +107,11 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected void checkJta() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isLocal() {
         return true;
     }
@@ -119,9 +123,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
+    @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         return (V)updateAllInternal(UPDATE,
@@ -138,16 +140,10 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
+    @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        boolean res = (Boolean)updateAllInternal(UPDATE,
+        Boolean res = (Boolean)updateAllInternal(UPDATE,
             Collections.singleton(key),
             Collections.singleton(val),
             null,
@@ -159,8 +155,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
 
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
         return res;
     }
@@ -168,8 +163,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -181,8 +174,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -192,65 +183,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return getAndPut(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        return getAndPutAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return put(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        return putAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return getAndPut(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        return getAndPutAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return put(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        return putAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        A.notNull(oldVal, "oldVal");
-
-        return put(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        return putAsync(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
+    @Override protected void putAll0(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         updateAllInternal(UPDATE,
@@ -264,13 +197,10 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             ctx.writeThrough(),
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
-
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m) {
+    @Override public IgniteInternalFuture<?> putAllAsync0(Map<? extends K, ? extends V> m) {
         return updateAllAsync0(m,
             null,
             null,
@@ -280,8 +210,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public V getAndRemove(K key) throws IgniteCheckedException {
+    @Override protected V getAndRemove0(K key) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         return (V)updateAllInternal(DELETE,
@@ -299,13 +228,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndRemoveAsync(K key) {
+    @Override public IgniteInternalFuture<V> getAndRemoveAsync0(K key) {
         return removeAllAsync0(Collections.singletonList(key), true, false, null);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void removeAll(Collection<? extends K> keys) throws IgniteCheckedException {
+    @Override public void removeAll0(Collection<? extends K> keys) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         updateAllInternal(DELETE,
@@ -322,19 +251,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<Object> removeAllAsync0(Collection<? extends K> keys) {
         return removeAllAsync0(keys, false, false, null).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public boolean remove(K key) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key");
-
+    @Override public boolean remove0(K key, final CacheEntryPredicate filter) throws IgniteCheckedException {
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         Boolean rmv = (Boolean)updateAllInternal(DELETE,
@@ -344,50 +267,23 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
             expiryPerCall(),
             false,
             false,
-            null,
+            filter,
             ctx.writeThrough(),
             ctx.readThrough(),
             opCtx != null && opCtx.isKeepBinary());
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+        assert rmv != null;
 
         return rmv;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key");
-
+    @Override public IgniteInternalFuture<Boolean> removeAsync0(K key, @Nullable CacheEntryPredicate filter) {
         return removeAllAsync0(Collections.singletonList(key), false, false, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        CacheOperationContext opCtx = ctx.operationContextPerCall();
-
-        return (Boolean)updateAllInternal(DELETE,
-            Collections.singleton(key),
-            null,
-            null,
-            expiryPerCall(),
-            false,
-            false,
-            ctx.equalsVal(val),
-            ctx.writeThrough(),
-            ctx.readThrough(),
-            opCtx != null && opCtx.isKeepBinary());
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        return removeAsync(key, ctx.equalsVal(val));
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> removeAllAsync() {
         return ctx.closures().callLocalSafe(new Callable<Void>() {
             @Override public Void call() throws Exception {
@@ -399,11 +295,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
     }
 
     /** {@inheritDoc} */
-
     @SuppressWarnings("unchecked")
-    @Override @Nullable public V get(K key, boolean deserializeBinary, boolean needVer) throws IgniteCheckedException {
-        String taskName = ctx.kernalContext().job().currentTaskName();
-
+    @Override protected V get0(
+        final K key,
+        String taskName,
+        boolean deserializeBinary,
+        boolean needVer) throws IgniteCheckedException
+    {
         Map<K, V> m = getAllInternal(Collections.singleton(key),
             ctx.isSwapOrOffheapEnabled(),
             ctx.readThrough(),
@@ -419,7 +317,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public final Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
+    @Override public final Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
         throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
@@ -794,7 +692,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
+        return asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(op,
                     keys,
@@ -809,11 +707,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
                     keepBinary);
             }
         });
-
-        if (ctx.config().isStatisticsEnabled())
-            fut.listen(new UpdatePutTimeStatClosure(metrics0(), System.nanoTime()));
-
-        return fut;
     }
 
     /**
@@ -835,17 +728,13 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
 
         final boolean readThrough = ctx.readThrough();
 
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
         final ExpiryPolicy expiryPlc = expiryPerCall();
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
+        return asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(DELETE,
                     keys,
@@ -860,11 +749,6 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
                     keepBinary);
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
-
-        return fut;
     }
 
     /**
@@ -1584,10 +1468,7 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected IgniteInternalFuture asyncOp(final Callable<?> op) {
-        if (!asyncToggled)
-            return ctx.closures().callLocalSafe(op);
-
+    private IgniteInternalFuture asyncOp(final Callable<?> op) {
         IgniteInternalFuture fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
index 3e3b84e..648134e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.java
@@ -34,7 +34,7 @@ public class CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest extend
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        cfg.getTransactionConfiguration().setDefaultTxTimeout(Long.MAX_VALUE);
+        cfg.getTransactionConfiguration().setDefaultTxTimeout(5 * 60_000);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
index ec3b808..57c709b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsBlockMessageSystemPoolStarvationSelfTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteFileSystem;
@@ -39,14 +41,11 @@ import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.transactions.TransactionConcurrency;
-import org.apache.ignite.transactions.TransactionIsolation;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
 
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
 
 /**
  * Test to check for system pool starvation due to {@link IgfsBlocksMessage}.
@@ -125,8 +124,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
             @Override public Void call() throws Exception {
                 GridCacheAdapter dataCache = dataCache(attacker);
 
-                try (IgniteInternalTx tx =
-                         dataCache.txStartEx(TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ)) {
+                try (IgniteInternalTx tx = dataCache.txStartEx(PESSIMISTIC, REPEATABLE_READ)) {
                     dataCache.put(DATA_KEY, 0);
 
                     txStartLatch.countDown();
@@ -185,6 +183,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
      * Create IGFS file asynchronously.
      *
      * @param path Path.
+     * @param writeStartLatch Write start latch.
      * @return Future.
      */
     private IgniteInternalFuture<Void> createFileAsync(final IgfsPath path, final CountDownLatch writeStartLatch) {
@@ -265,6 +264,7 @@ public class IgfsBlockMessageSystemPoolStarvationSelfTest extends IgfsCommonAbst
         cfg.setLocalHost("127.0.0.1");
         cfg.setConnectorConfiguration(null);
 
+        cfg.setStripedPoolSize(0);
         cfg.setSystemThreadPoolSize(2);
         cfg.setRebalanceThreadPoolSize(1);
         cfg.setPublicThreadPoolSize(1);


[46/50] [abbrv] ignite git commit: Implemented Visor tasks for Services.

Posted by vo...@apache.org.
Implemented Visor tasks for Services.


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

Branch: refs/heads/ignite-2.0
Commit: fdf1f4bb038eb9b0918367c7a5b7e49fb1e9ec51
Parents: 8cffe90
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Dec 23 18:20:44 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 23 18:20:44 2016 +0700

----------------------------------------------------------------------
 .../visor/service/VisorCancelServiceTask.java   |  70 ++++++++++
 .../visor/service/VisorServiceDescriptor.java   | 132 +++++++++++++++++++
 .../visor/service/VisorServiceTask.java         |  75 +++++++++++
 .../internal/visor/util/VisorTaskUtils.java     |  15 ++-
 .../resources/META-INF/classnames.properties    |  65 +++++++--
 5 files changed, 342 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
new file mode 100644
index 0000000..64987e9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorCancelServiceTask.java
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.service;
+
+import org.apache.ignite.IgniteServices;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+
+/**
+ * Task for cancel services with specified name.
+ */
+@GridInternal
+public class VisorCancelServiceTask extends VisorOneNodeTask<String, Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCancelServiceJob job(String arg) {
+        return new VisorCancelServiceJob(arg, debug);
+    }
+
+    /**
+     * Job for cancel services with specified name.
+     */
+    private static class VisorCancelServiceJob extends VisorJob<String, Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Debug flag.
+         */
+        protected VisorCancelServiceJob(String arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Void run(final String arg) {
+            IgniteServices services = ignite.services();
+
+            services.cancel(arg);
+
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorCancelServiceJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
new file mode 100644
index 0000000..83ec77d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceDescriptor.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.service;
+
+import java.io.Serializable;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.util.VisorTaskUtils;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Data transfer object for {@link ServiceDescriptor} object.
+ */
+public class VisorServiceDescriptor implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Service name. */
+    private String name;
+
+    /** Service class. */
+    private String srvcCls;
+
+    /** Maximum allowed total number of deployed services in the grid, {@code 0} for unlimited. */
+    private int totalCnt;
+
+    /** Maximum allowed number of deployed services on each node. */
+    private int maxPerNodeCnt;
+
+    /** Cache name used for key-to-node affinity calculation. */
+    private String cacheName;
+
+    /** ID of grid node that initiated the service deployment. */
+    private UUID originNodeId;
+
+    /**
+     * Service deployment topology snapshot.
+     * Number of service instances deployed on a node mapped to node ID.
+     */
+    private Map<UUID, Integer> topSnapshot;
+
+    /**
+     * Default constructor.
+     */
+    public VisorServiceDescriptor() {
+        // No-op.
+    }
+
+    /**
+     * Create task result with given parameters
+     *
+     */
+    public VisorServiceDescriptor(ServiceDescriptor srvc) {
+        name = srvc.name();
+        srvcCls = VisorTaskUtils.compactClass(srvc.serviceClass());
+        totalCnt = srvc.totalCount();
+        maxPerNodeCnt = srvc.maxPerNodeCount();
+        cacheName = srvc.cacheName();
+        originNodeId = srvc.originNodeId();
+        topSnapshot = srvc.topologySnapshot();
+    }
+
+    /**
+     * @return Service name.
+     */
+    public String getName() {
+        return name;
+    }
+
+    /**
+     * @return Service class.
+     */
+    public String getServiceClass() {
+        return srvcCls;
+    }
+
+    /**
+     * @return Maximum allowed total number of deployed services in the grid, 0 for unlimited.
+     */
+    public int getTotalCnt() {
+        return totalCnt;
+    }
+
+    /**
+     * @return Maximum allowed number of deployed services on each node.
+     */
+    public int getMaxPerNodeCnt() {
+        return maxPerNodeCnt;
+    }
+
+    /**
+     * @return Cache name used for key-to-node affinity calculation.
+     */
+    public String getCacheName() {
+        return cacheName;
+    }
+
+    /**
+     * @return ID of grid node that initiated the service deployment.
+     */
+    public UUID getOriginNodeId() {
+        return originNodeId;
+    }
+
+    /**
+     * @return Service deployment topology snapshot. Number of service instances deployed on a node mapped to node ID.
+     */
+    public Map<UUID, Integer> getTopologySnapshot() {
+        return topSnapshot;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorServiceDescriptor.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
new file mode 100644
index 0000000..1b3495c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/service/VisorServiceTask.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.service;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.apache.ignite.services.ServiceDescriptor;
+
+/**
+ * Task for collect topology service configuration.
+ */
+@GridInternal
+public class VisorServiceTask extends VisorOneNodeTask<Void, Collection<VisorServiceDescriptor>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorServiceJob job(Void arg) {
+        return new VisorServiceJob(arg, debug);
+    }
+
+    /**
+     * Job for collect topology service configuration.
+     */
+    private static class VisorServiceJob extends VisorJob<Void, Collection<VisorServiceDescriptor>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Debug flag.
+         */
+        protected VisorServiceJob(Void arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<VisorServiceDescriptor> run(final Void arg) {
+            Collection<ServiceDescriptor> services = ignite.services().serviceDescriptors();
+
+            Collection<VisorServiceDescriptor> res = new ArrayList<>(services.size());
+
+            for (ServiceDescriptor srvc: services)
+                res.add(new VisorServiceDescriptor(srvc));
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(VisorServiceJob.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
index 1e9346c..3f5003a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorTaskUtils.java
@@ -270,6 +270,19 @@ public class VisorTaskUtils {
     /**
      * Compact class names.
      *
+     * @param cls Class object for compact.
+     * @return Compacted string.
+     */
+    @Nullable public static String compactClass(Class cls) {
+        if (cls == null)
+            return null;
+
+        return U.compact(cls.getName());
+    }
+
+    /**
+     * Compact class names.
+     *
      * @param obj Object for compact.
      * @return Compacted string.
      */
@@ -277,7 +290,7 @@ public class VisorTaskUtils {
         if (obj == null)
             return null;
 
-        return U.compact(obj.getClass().getName());
+        return compactClass(obj.getClass());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fdf1f4bb/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 4c9596c..4d0b931 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -294,12 +294,17 @@ org.apache.ignite.internal.jdbc2.JdbcDatabaseMetadata$UpdateMetadataTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$1
 org.apache.ignite.internal.jdbc2.JdbcQueryTask$QueryResult
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$1
+org.apache.ignite.internal.jdbc2.JdbcQueryTaskV2$QueryResult
+org.apache.ignite.internal.jdbc2.JdbcSqlFieldsQuery
 org.apache.ignite.internal.managers.GridManagerAdapter$1$1
 org.apache.ignite.internal.managers.checkpoint.GridCheckpointManager$CheckpointSet
 org.apache.ignite.internal.managers.checkpoint.GridCheckpointRequest
 org.apache.ignite.internal.managers.communication.GridIoManager$ConcurrentHashMap0
 org.apache.ignite.internal.managers.communication.GridIoMessage
 org.apache.ignite.internal.managers.communication.GridIoUserMessage
+org.apache.ignite.internal.managers.communication.IgniteIoTestMessage
 org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean
 org.apache.ignite.internal.managers.deployment.GridDeploymentPerVersionStore$2
 org.apache.ignite.internal.managers.deployment.GridDeploymentRequest
@@ -387,20 +392,20 @@ org.apache.ignite.internal.processors.cache.GridCacheAdapter$3
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$30
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$32
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$4
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$48
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$49
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$50
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$51
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$52
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$53
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$54
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$55
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$57
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$58
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$58$1
+org.apache.ignite.internal.processors.cache.GridCacheAdapter$59
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$6
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$60
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$61
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$62
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$63
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$64
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$65
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$66
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$67
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$69
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$70
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$70$1
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$71
-org.apache.ignite.internal.processors.cache.GridCacheAdapter$72
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$9
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1
 org.apache.ignite.internal.processors.cache.GridCacheAdapter$AsyncOp$1$1
@@ -719,8 +724,11 @@ org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtFor
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtForceKeysResponse
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemandMessage
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$1$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$2
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$3
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$4$1
+org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$5$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$1
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander$DemandWorker$2
 org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionExchangeId
@@ -1110,6 +1118,12 @@ org.apache.ignite.internal.processors.hadoop.HadoopJobStatus
 org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan
 org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo
 org.apache.ignite.internal.processors.hadoop.HadoopTaskType
+org.apache.ignite.internal.processors.hadoop.message.HadoopMessage
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse
+org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage
 org.apache.ignite.internal.processors.igfs.IgfsAckMessage
 org.apache.ignite.internal.processors.igfs.IgfsAttributes
 org.apache.ignite.internal.processors.igfs.IgfsBlockKey
@@ -1207,6 +1221,7 @@ org.apache.ignite.internal.processors.platform.cache.PlatformCacheEntryProcessor
 org.apache.ignite.internal.processors.platform.cache.PlatformCachePartialUpdateException
 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinity$1
 org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunction
+org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryFilter
 org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryImpl
@@ -1243,6 +1258,9 @@ 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
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$CleanupCompletionListener
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkCacheExtension$RemoveOldEntriesRunnable
+org.apache.ignite.internal.processors.platform.entityframework.PlatformDotNetEntityFrameworkIncreaseVersionProcessor
 org.apache.ignite.internal.processors.platform.events.PlatformEventFilterListenerImpl
 org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter
 org.apache.ignite.internal.processors.platform.messaging.PlatformMessageFilterImpl
@@ -1265,6 +1283,7 @@ 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.IgniteSQLException
 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
@@ -1330,6 +1349,9 @@ org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructure
 org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryFieldsMetaResult
 org.apache.ignite.internal.processors.rest.handlers.query.CacheQueryResult
 org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler$QueryCursorIterator
+org.apache.ignite.internal.processors.rest.handlers.redis.GridRedisRestCommandHandler$1
+org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisGenericException
+org.apache.ignite.internal.processors.rest.handlers.redis.exception.GridRedisTypeException
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$2
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler$ExeCallable
 org.apache.ignite.internal.processors.rest.handlers.task.GridTaskResultRequest
@@ -1341,6 +1363,9 @@ org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioList
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpMemcachedNioListener$2
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1
 org.apache.ignite.internal.processors.rest.protocols.tcp.GridTcpRestNioListener$1$1
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisCommand
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisMessage
+org.apache.ignite.internal.processors.rest.protocols.tcp.redis.GridRedisNioListener$1
 org.apache.ignite.internal.processors.rest.request.RestQueryRequest$QueryType
 org.apache.ignite.internal.processors.service.GridServiceAssignments
 org.apache.ignite.internal.processors.service.GridServiceAssignmentsKey
@@ -1585,10 +1610,13 @@ org.apache.ignite.internal.util.lang.IgniteReducer2X
 org.apache.ignite.internal.util.lang.IgniteReducer3
 org.apache.ignite.internal.util.lang.IgniteReducer3X
 org.apache.ignite.internal.util.lang.IgniteReducerX
+org.apache.ignite.internal.util.lang.IgniteSingletonIterator
 org.apache.ignite.internal.util.nio.GridNioEmbeddedFuture$1
 org.apache.ignite.internal.util.nio.GridNioException
 org.apache.ignite.internal.util.nio.GridNioMessageTracker
 org.apache.ignite.internal.util.nio.GridNioServer$NioOperation
+org.apache.ignite.internal.util.nio.GridNioServer$RandomBalancer
+org.apache.ignite.internal.util.nio.GridNioServer$SizeBasedBalancer
 org.apache.ignite.internal.util.nio.GridNioSessionMetaKey
 org.apache.ignite.internal.util.nio.ssl.GridNioSslHandler
 org.apache.ignite.internal.util.offheap.GridOffHeapEvent
@@ -1801,6 +1829,11 @@ org.apache.ignite.internal.visor.query.VisorQueryResult
 org.apache.ignite.internal.visor.query.VisorQueryResultEx
 org.apache.ignite.internal.visor.query.VisorQueryScanSubstringFilter
 org.apache.ignite.internal.visor.query.VisorQueryTask
+org.apache.ignite.internal.visor.service.VisorCancelServiceTask
+org.apache.ignite.internal.visor.service.VisorCancelServiceTask$VisorCancelServiceJob
+org.apache.ignite.internal.visor.service.VisorServiceDescriptor
+org.apache.ignite.internal.visor.service.VisorServiceTask
+org.apache.ignite.internal.visor.service.VisorServiceTask$VisorServiceJob
 org.apache.ignite.internal.visor.util.VisorClusterGroupEmptyException
 org.apache.ignite.internal.visor.util.VisorEventMapper
 org.apache.ignite.internal.visor.util.VisorExceptionWrapper
@@ -1858,12 +1891,15 @@ org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointData
 org.apache.ignite.spi.collision.jobstealing.JobStealingRequest
 org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi$PriorityGridCollisionJobContextComparator
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$1
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$10
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$11
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosure$1
-org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$8
-org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$9
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$2$ConnectClosureNew$1
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeClosure
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage
+org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeMessage2
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$HandshakeTimeoutException
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$NodeIdMessage
 org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi$RecoveryLastReceivedMessage
@@ -1923,3 +1959,4 @@ org.apache.ignite.transactions.TransactionOptimisticException
 org.apache.ignite.transactions.TransactionRollbackException
 org.apache.ignite.transactions.TransactionState
 org.apache.ignite.transactions.TransactionTimeoutException
+org.apache.ignite.util.AttributeNodeFilter


[22/50] [abbrv] ignite git commit: Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

Posted by vo...@apache.org.
Fixed 'singleRmv' flag for cache.remove(k, v) operation in tx cache.

(cherry picked from commit 126ab60)


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

Branch: refs/heads/ignite-2.0
Commit: 800579c9ca193f7660e35f241c255904c573eef1
Parents: e82eefe
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:15:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:18:28 2016 +0300

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


http://git-wip-us.apache.org/repos/asf/ignite/blob/800579c9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index f235f6a..3b4557d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2948,6 +2948,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /**
      * @param key Key.
+     * @param filter Filter.
      * @return {@code True} if entry was removed.
      * @throws IgniteCheckedException If failed.
      */
@@ -2959,7 +2960,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     Collections.singletonList(key),
                     /*retval*/false,
                     filter,
-                    /*singleRmv*/true).get().success();
+                    /*singleRmv*/filter == null).get().success();
             }
 
             @Override public String toString() {


[37/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
deleted file mode 100644
index 296b942..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-java.js
+++ /dev/null
@@ -1,3617 +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.
- */
-
-// Java generation entry point.
-const $generatorJava = {};
-
-/**
- * Translate some value to valid java code.
- *
- * @param val Value to convert.
- * @param type Value type.
- * @returns {*} String with value that will be valid for java.
- */
-$generatorJava.toJavaCode = function(val, type) {
-    if (val === null)
-        return 'null';
-
-    if (type === 'raw')
-        return val;
-
-    if (type === 'class')
-        return val + '.class';
-
-    if (type === 'float')
-        return val + 'f';
-
-    if (type === 'path')
-        return '"' + val.replace(/\\/g, '\\\\') + '"';
-
-    if (type)
-        return type + '.' + val;
-
-    if (typeof (val) === 'string')
-        return '"' + val.replace('"', '\\"') + '"';
-
-    if (typeof (val) === 'number' || typeof (val) === 'boolean')
-        return String(val);
-
-    return 'Unknown type: ' + typeof (val) + ' (' + val + ')';
-};
-
-/**
- * @param propName Property name.
- * @param setterName Optional concrete setter name.
- * @returns Property setter with name by java conventions.
- */
-$generatorJava.setterName = function(propName, setterName) {
-    return setterName ? setterName : $generatorCommon.toJavaName('set', propName);
-};
-
-// Add constructor argument
-$generatorJava.constructorArg = function(obj, propName, dflt, notFirst, opt) {
-    const v = (obj ? obj[propName] : null) || dflt;
-
-    if ($generatorCommon.isDefinedAndNotEmpty(v))
-        return (notFirst ? ', ' : '') + $generatorJava.toJavaCode(v);
-    else if (!opt)
-        return notFirst ? ', null' : 'null';
-
-    return '';
-};
-
-/**
- * Add variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param varFullActualType Variable actual full class name to be added to imports.
- * @param varFullGenericType1 Optional full class name of first generic.
- * @param varFullGenericType2 Optional full class name of second generic.
- * @param subClass If 'true' then variable will be declared as anonymous subclass.
- */
-$generatorJava.declareVariable = function(res, varName, varFullType, varFullActualType, varFullGenericType1, varFullGenericType2, subClass) {
-    res.emptyLineIfNeeded();
-
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    if (varFullActualType && varFullGenericType1) {
-        const varActualType = res.importClass(varFullActualType);
-        const varGenericType1 = res.importClass(varFullGenericType1);
-        let varGenericType2 = null;
-
-        if (varFullGenericType2)
-            varGenericType2 = res.importClass(varFullGenericType2);
-
-        res.line((varNew ? (varType + '<' + varGenericType1 + (varGenericType2 ? ', ' + varGenericType2 : '') + '> ') : '') +
-            varName + ' = new ' + varActualType + '<>();');
-    }
-    else
-        res.line((varNew ? (varType + ' ') : '') + varName + ' = new ' + varType + '()' + (subClass ? ' {' : ';'));
-
-    if (!subClass)
-        res.needEmptyLine = true;
-
-    return varName;
-};
-
-/**
- * Add local variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- */
-$generatorJava.declareVariableLocal = function(res, varName, varFullType) {
-    const varType = res.importClass(varFullType);
-
-    res.line(varType + ' ' + varName + ' = new ' + varType + '();');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Add custom variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param varExpr Custom variable creation expression.
- * @param modifier Additional variable modifier.
- */
-$generatorJava.declareVariableCustom = function(res, varName, varFullType, varExpr, modifier) {
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    res.line((varNew ? ((modifier ? modifier + ' ' : '') + varType + ' ') : '') + varName + ' = ' + varExpr + ';');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Add array variable declaration.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param varFullType Variable full class name to be added to imports.
- * @param length Array length.
- */
-$generatorJava.declareVariableArray = function(res, varName, varFullType, length) {
-    const varType = res.importClass(varFullType);
-
-    const varNew = !res.vars[varName];
-
-    if (varNew)
-        res.vars[varName] = true;
-
-    res.line((varNew ? (varType + '[] ') : '') + varName + ' = new ' + varType + '[' + length + '];');
-
-    res.needEmptyLine = true;
-};
-
-/**
- * Clear list of declared variables.
- *
- * @param res
- */
-$generatorJava.resetVariables = function(res) {
-    res.vars = {};
-};
-
-/**
- * Add property via setter / property name.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional info about property data type.
- * @param setterName Optional special setter name.
- * @param dflt Optional default value.
- */
-$generatorJava.property = function(res, varName, obj, propName, dataType, setterName, dflt) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            const missDflt = _.isNil(dflt);
-
-            // Add to result if no default provided or value not equals to default.
-            if (missDflt || (!missDflt && val !== dflt)) {
-                res.line(varName + '.' + $generatorJava.setterName(propName, setterName) +
-                    '(' + $generatorJava.toJavaCode(val, dataType) + ');');
-
-                return true;
-            }
-        }
-    }
-
-    return false;
-};
-
-/**
- * Add enum property via setter / property name.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Name of enum class
- * @param setterName Optional special setter name.
- * @param dflt Optional default value.
- */
-$generatorJava.enumProperty = function(res, varName, obj, propName, dataType, setterName, dflt) {
-    const val = obj[propName];
-
-    if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-        const missDflt = _.isNil(dflt);
-
-        // Add to result if no default provided or value not equals to default.
-        if (missDflt || (!missDflt && val !== dflt)) {
-            res.line(varName + '.' + $generatorJava.setterName(propName, setterName) +
-                '(' + $generatorJava.toJavaCode(val, dataType ? res.importClass(dataType) : null) + ');');
-
-            return true;
-        }
-    }
-
-    return false;
-};
-
-// Add property for class name.
-$generatorJava.classNameProperty = function(res, varName, obj, propName) {
-    const val = obj[propName];
-
-    if (!_.isNil(val)) {
-        res.line(varName + '.' + $generatorJava.setterName(propName) +
-            '("' + $generatorCommon.JavaTypes.fullClassName(val) + '");');
-    }
-};
-
-/**
- * Add list property.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional data type.
- * @param setterName Optional setter name.
- */
-$generatorJava.listProperty = function(res, varName, obj, propName, dataType, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.importClass('java.util.Arrays');
-
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => $generatorJava.toJavaCode(v, dataType)), '(Arrays.asList(', '))');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add function with varargs arguments.
- *
- * @param res Resulting output with generated code.
- * @param fx Function name.
- * @param quote Whether to quote arguments.
- * @param args Array with arguments.
- * @param startBlock Optional start block string.
- * @param endBlock Optional end block string.
- * @param startQuote Start quote string.
- * @param endQuote End quote string.
- */
-$generatorJava.fxVarArgs = function(res, fx, quote, args, startBlock = '(', endBlock = ')', startQuote = '"', endQuote = '"') {
-    const quoteArg = (arg) => quote ? startQuote + arg + endQuote : arg;
-
-    if (args.length === 1)
-        res.append(fx + startBlock + quoteArg(args[0]) + endBlock + ';');
-    else {
-        res.startBlock(fx + startBlock);
-
-        const len = args.length - 1;
-
-        _.forEach(args, (arg, ix) => res.line(quoteArg(arg) + (ix < len ? ', ' : '')));
-
-        res.endBlock(endBlock + ';');
-    }
-};
-
-/**
- * Add array property.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param setterName Optional setter name.
- */
-$generatorJava.arrayProperty = function(res, varName, obj, propName, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => 'new ' + res.importClass(v) + '()'), '({ ', ' });');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add multi-param property (setter with several arguments).
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param obj Source object with data.
- * @param propName Property name to take from source object.
- * @param dataType Optional data type.
- * @param setterName Optional setter name.
- */
-$generatorJava.multiparamProperty = function(res, varName, obj, propName, dataType, setterName) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        $generatorJava.fxVarArgs(res, varName + '.' + $generatorJava.setterName(propName, setterName), false,
-            _.map(val, (v) => $generatorJava.toJavaCode(dataType === 'class' ? res.importClass(v) : v, dataType)));
-    }
-};
-
-/**
- * Add complex bean.
- *
- * @param res Resulting output with generated code.
- * @param varName Variable name.
- * @param bean
- * @param beanPropName Bean property name.
- * @param beanVarName
- * @param beanClass Bean class.
- * @param props
- * @param createBeanAlthoughNoProps If 'true' then create empty bean.
- */
-$generatorJava.beanProperty = function(res, varName, bean, beanPropName, beanVarName, beanClass, props, createBeanAlthoughNoProps) {
-    if (bean && $generatorCommon.hasProperty(bean, props)) {
-        res.emptyLineIfNeeded();
-
-        $generatorJava.declareVariable(res, beanVarName, beanClass);
-
-        _.forIn(props, function(descr, propName) {
-            if (props.hasOwnProperty(propName)) {
-                if (descr) {
-                    switch (descr.type) {
-                        case 'list':
-                            $generatorJava.listProperty(res, beanVarName, bean, propName, descr.elementsType, descr.setterName);
-                            break;
-
-                        case 'array':
-                            $generatorJava.arrayProperty(res, beanVarName, bean, propName, descr.setterName);
-                            break;
-
-                        case 'enum':
-                            $generatorJava.enumProperty(res, beanVarName, bean, propName, descr.enumClass, descr.setterName, descr.dflt);
-                            break;
-
-                        case 'float':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'float', descr.setterName);
-                            break;
-
-                        case 'path':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'path', descr.setterName);
-                            break;
-
-                        case 'raw':
-                            $generatorJava.property(res, beanVarName, bean, propName, 'raw', descr.setterName);
-                            break;
-
-                        case 'propertiesAsList':
-                            const val = bean[propName];
-
-                            if (val && val.length > 0) {
-                                $generatorJava.declareVariable(res, descr.propVarName, 'java.util.Properties');
-
-                                _.forEach(val, function(nameAndValue) {
-                                    const eqIndex = nameAndValue.indexOf('=');
-
-                                    if (eqIndex >= 0) {
-                                        res.line(descr.propVarName + '.setProperty(' +
-                                            '"' + nameAndValue.substring(0, eqIndex) + '", ' +
-                                            '"' + nameAndValue.substr(eqIndex + 1) + '");');
-                                    }
-                                });
-
-                                res.needEmptyLine = true;
-
-                                res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(' + descr.propVarName + ');');
-                            }
-                            break;
-
-                        case 'bean':
-                            if ($generatorCommon.isDefinedAndNotEmpty(bean[propName]))
-                                res.line(beanVarName + '.' + $generatorJava.setterName(propName) + '(new ' + res.importClass(bean[propName]) + '());');
-
-                            break;
-
-                        default:
-                            $generatorJava.property(res, beanVarName, bean, propName, null, descr.setterName, descr.dflt);
-                    }
-                }
-                else
-                    $generatorJava.property(res, beanVarName, bean, propName);
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(' + beanVarName + ');');
-
-        res.needEmptyLine = true;
-    }
-    else if (createBeanAlthoughNoProps) {
-        res.emptyLineIfNeeded();
-        res.line(varName + '.' + $generatorJava.setterName(beanPropName) + '(new ' + res.importClass(beanClass) + '());');
-
-        res.needEmptyLine = true;
-    }
-};
-
-/**
- * Add eviction policy.
- *
- * @param res Resulting output with generated code.
- * @param varName Current using variable name.
- * @param evtPlc Data to add.
- * @param propName Name in source data.
- */
-$generatorJava.evictionPolicy = function(res, varName, evtPlc, propName) {
-    if (evtPlc && evtPlc.kind) {
-        const evictionPolicyDesc = $generatorCommon.EVICTION_POLICIES[evtPlc.kind];
-
-        const obj = evtPlc[evtPlc.kind.toUpperCase()];
-
-        $generatorJava.beanProperty(res, varName, obj, propName, propName,
-            evictionPolicyDesc.className, evictionPolicyDesc.fields, true);
-    }
-};
-
-// Generate cluster general group.
-$generatorJava.clusterGeneral = function(cluster, clientNearCfg, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.declareVariable(res, 'cfg', 'org.apache.ignite.configuration.IgniteConfiguration');
-
-    $generatorJava.property(res, 'cfg', cluster, 'name', null, 'setGridName');
-    res.needEmptyLine = true;
-
-    $generatorJava.property(res, 'cfg', cluster, 'localHost');
-    res.needEmptyLine = true;
-
-    if (clientNearCfg) {
-        res.line('cfg.setClientMode(true);');
-
-        res.needEmptyLine = true;
-    }
-
-    if (cluster.discovery) {
-        const d = cluster.discovery;
-
-        $generatorJava.declareVariable(res, 'discovery', 'org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi');
-
-        switch (d.kind) {
-            case 'Multicast':
-                $generatorJava.beanProperty(res, 'discovery', d.Multicast, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
-                    {
-                        multicastGroup: null,
-                        multicastPort: null,
-                        responseWaitTime: null,
-                        addressRequestAttempts: null,
-                        localAddress: null,
-                        addresses: {type: 'list'}
-                    }, true);
-
-                break;
-
-            case 'Vm':
-                $generatorJava.beanProperty(res, 'discovery', d.Vm, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
-                    {addresses: {type: 'list'}}, true);
-
-                break;
-
-            case 'S3':
-                $generatorJava.beanProperty(res, 'discovery', d.S3, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder', {bucketName: null}, true);
-
-                break;
-
-            case 'Cloud':
-                $generatorJava.beanProperty(res, 'discovery', d.Cloud, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
-                    {
-                        credential: null,
-                        credentialPath: null,
-                        identity: null,
-                        provider: null,
-                        regions: {type: 'list'},
-                        zones: {type: 'list'}
-                    }, true);
-
-                break;
-
-            case 'GoogleStorage':
-                $generatorJava.beanProperty(res, 'discovery', d.GoogleStorage, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
-                    {
-                        projectName: null,
-                        bucketName: null,
-                        serviceAccountP12FilePath: null,
-                        serviceAccountId: null
-                    }, true);
-
-                break;
-
-            case 'Jdbc':
-                $generatorJava.declareVariable(res, 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder');
-                $generatorJava.property(res, 'ipFinder', d.Jdbc, 'initSchema');
-
-                const datasource = d.Jdbc;
-                if (datasource.dataSourceBean && datasource.dialect) {
-                    res.needEmptyLine = !datasource.initSchema;
-
-                    res.line('ipFinder.setDataSource(DataSources.INSTANCE_' + datasource.dataSourceBean + ');');
-                }
-
-                res.needEmptyLine = true;
-
-                res.line('discovery.setIpFinder(ipFinder);');
-
-                break;
-
-            case 'SharedFs':
-                $generatorJava.beanProperty(res, 'discovery', d.SharedFs, 'ipFinder', 'ipFinder',
-                    'org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder', {path: null}, true);
-
-                break;
-
-            case 'ZooKeeper':
-                const finderVar = 'ipFinder';
-
-                $generatorJava.declareVariable(res, 'ipFinder', 'org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder');
-
-                if (d.ZooKeeper) {
-                    if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator))
-                        res.line(finderVar + '.setCurator(new ' + res.importClass(d.ZooKeeper.curator) + '());');
-
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'zkConnectionString');
-
-                    if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) {
-                        const kind = d.ZooKeeper.retryPolicy.kind;
-                        const retryPolicy = d.ZooKeeper.retryPolicy[kind];
-
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.ExponentialBackoffRetry') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxSleepMs', null, true, true) + '));');
-
-                                break;
-
-                            case 'BoundedExponentialBackoff':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.BoundedExponentialBackoffRetry') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'baseSleepTimeMs', 1000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxSleepTimeMs', 2147483647, true) +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxRetries', 10, true) + '));');
-
-                                break;
-
-                            case 'UntilElapsed':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryUntilElapsed') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'maxElapsedTimeMs', 60000) +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));');
-
-                                break;
-
-                            case 'NTimes':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryNTimes') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'n', 10) +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetries', 1000, true) + '));');
-
-                                break;
-
-                            case 'OneTime':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryOneTime') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'sleepMsBetweenRetry', 1000) + '));');
-
-                                break;
-
-                            case 'Forever':
-                                res.line(finderVar + '.setRetryPolicy(new ' + res.importClass('org.apache.curator.retry.RetryForever') + '(' +
-                                    $generatorJava.constructorArg(retryPolicy, 'retryIntervalMs', 1000) + '));');
-
-                                break;
-
-                            case 'Custom':
-                                if (retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className))
-                                    res.line(finderVar + '.setRetryPolicy(new ' + res.importClass(retryPolicy.className) + '());');
-
-                                break;
-
-                            default:
-                        }
-                    }
-
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'basePath', null, null, '/services');
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'serviceName', null, null, 'ignite');
-                    $generatorJava.property(res, finderVar, d.ZooKeeper, 'allowDuplicateRegistrations', null, null, false);
-                }
-
-                res.line('discovery.setIpFinder(ipFinder);');
-
-                break;
-
-            default:
-                res.line('Unknown discovery kind: ' + d.kind);
-        }
-
-        res.needEmptyLine = false;
-
-        $generatorJava.clusterDiscovery(d, res);
-
-        res.emptyLineIfNeeded();
-
-        res.line('cfg.setDiscoverySpi(discovery);');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate atomics group.
-$generatorJava.clusterAtomics = function(atomics, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) {
-        res.startSafeBlock();
-
-        $generatorJava.declareVariable(res, 'atomicCfg', 'org.apache.ignite.configuration.AtomicConfiguration');
-
-        $generatorJava.enumProperty(res, 'atomicCfg', atomics, 'cacheMode', 'org.apache.ignite.cache.CacheMode', null, 'PARTITIONED');
-
-        const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED';
-
-        let hasData = cacheMode !== 'PARTITIONED';
-
-        hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'atomicSequenceReserveSize', null, null, 1000) || hasData;
-
-        if (cacheMode === 'PARTITIONED')
-            hasData = $generatorJava.property(res, 'atomicCfg', atomics, 'backups', null, null, 0) || hasData;
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setAtomicConfiguration(atomicCfg);');
-
-        res.needEmptyLine = true;
-
-        if (!hasData)
-            res.rollbackSafeBlock();
-    }
-
-    return res;
-};
-
-// Generate binary group.
-$generatorJava.clusterBinary = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.binaryIsDefined(binary)) {
-        const varName = 'binary';
-
-        $generatorJava.declareVariable(res, varName, 'org.apache.ignite.configuration.BinaryConfiguration');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper))
-            res.line(varName + '.setIdMapper(new ' + res.importClass(binary.idMapper) + '());');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.nameMapper))
-            res.line(varName + '.setNameMapper(new ' + res.importClass(binary.nameMapper) + '());');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.serializer))
-            res.line(varName + '.setSerializer(new ' + res.importClass(binary.serializer) + '());');
-
-        res.needEmptyLine = $generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.serializer);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) {
-            const arrVar = 'types';
-
-            $generatorJava.declareVariable(res, arrVar, 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.binary.BinaryTypeConfiguration');
-
-            _.forEach(binary.typeConfigurations, function(type) {
-                if ($generatorCommon.isDefinedAndNotEmpty(type.typeName))
-                    res.line(arrVar + '.add(' + $generatorJava.binaryTypeFunctionName(type.typeName) + '());'); // TODO IGNITE-2269 Replace using of separated methods for binary type configurations to extended constructors.
-            });
-
-            res.needEmptyLine = true;
-
-            res.line(varName + '.setTypeConfigurations(' + arrVar + ');');
-
-            res.needEmptyLine = true;
-        }
-
-        $generatorJava.property(res, varName, binary, 'compactFooter', null, null, true);
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setBinaryConfiguration(' + varName + ');');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache key configurations.
-$generatorJava.clusterCacheKeyConfiguration = function(keyCfgs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName);
-
-    if (_.isEmpty(keyCfgs))
-        return res;
-
-    $generatorJava.declareVariableArray(res, 'keyConfigurations', 'org.apache.ignite.cache.CacheKeyConfiguration', keyCfgs.length);
-
-    const cacheKeyCfg = res.importClass('org.apache.ignite.cache.CacheKeyConfiguration');
-
-    _.forEach(keyCfgs, (cfg, idx) => {
-        res.needEmptyLine = true;
-
-        res.line(`keyConfigurations[${idx}] = new ${cacheKeyCfg}("${cfg.typeName}", "${cfg.affinityKeyFieldName}");`);
-
-        res.needEmptyLine = true;
-    });
-
-    res.line('cfg.setCacheKeyConfiguration(keyConfigurations);');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Construct binary type configuration factory method name.
-$generatorJava.binaryTypeFunctionName = function(typeName) {
-    const dotIdx = typeName.lastIndexOf('.');
-
-    const shortName = dotIdx > 0 ? typeName.substr(dotIdx + 1) : typeName;
-
-    return $generatorCommon.toJavaName('binaryType', shortName);
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Generate factory method for specified BinaryTypeConfiguration.
-$generatorJava.binaryTypeConfiguration = function(type, res) {
-    const typeName = type.typeName;
-
-    res.line('/**');
-    res.line(' * Create binary type configuration for ' + typeName + '.');
-    res.line(' *');
-    res.line(' * @return Configured binary type.');
-    res.line(' */');
-    res.startBlock('private static BinaryTypeConfiguration ' + $generatorJava.binaryTypeFunctionName(typeName) + '() {');
-
-    $generatorJava.resetVariables(res);
-
-    const typeVar = 'typeCfg';
-
-    $generatorJava.declareVariable(res, typeVar, 'org.apache.ignite.binary.BinaryTypeConfiguration');
-
-    $generatorJava.property(res, typeVar, type, 'typeName');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.idMapper))
-        res.line(typeVar + '.setIdMapper(new ' + res.importClass(type.idMapper) + '());');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.nameMapper))
-        res.line(typeVar + '.setNameMapper(new ' + res.importClass(type.nameMapper) + '());');
-
-    if ($generatorCommon.isDefinedAndNotEmpty(type.serializer))
-        res.line(typeVar + '.setSerializer(new ' + res.importClass(type.serializer) + '());');
-
-    $generatorJava.property(res, typeVar, type, 'enum', null, null, false);
-
-    res.needEmptyLine = true;
-
-    res.line('return ' + typeVar + ';');
-    res.endBlock('}');
-
-    res.needEmptyLine = true;
-};
-
-// TODO IGNITE-2269 Remove specified methods after implamentation of extended constructors.
-// Generates binary type configuration factory methods.
-$generatorJava.binaryTypeConfigurations = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(binary)) {
-        _.forEach(binary.typeConfigurations, function(type) {
-            $generatorJava.binaryTypeConfiguration(type, res);
-        });
-    }
-
-    return res;
-};
-
-// Generate collision group.
-$generatorJava.clusterCollision = function(collision, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (collision && collision.kind && collision.kind !== 'Noop') {
-        const spi = collision[collision.kind];
-
-        if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) {
-            const varName = 'collisionSpi';
-
-            switch (collision.kind) {
-                case 'JobStealing':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'activeJobsThreshold', null, null, 95);
-                    $generatorJava.property(res, varName, spi, 'waitJobsThreshold', null, null, 0);
-                    $generatorJava.property(res, varName, spi, 'messageExpireTime', null, null, 1000);
-                    $generatorJava.property(res, varName, spi, 'maximumStealingAttempts', null, null, 5);
-                    $generatorJava.property(res, varName, spi, 'stealingEnabled', null, null, true);
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) {
-                        res.line(varName + '.' + $generatorJava.setterName('externalCollisionListener') +
-                            '(new ' + res.importClass(spi.externalCollisionListener) + '());');
-                    }
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) {
-                        const stealingAttrsVar = 'stealingAttrs';
-
-                        res.needEmptyLine = true;
-
-                        $generatorJava.declareVariable(res, stealingAttrsVar, 'java.util.Map', 'java.util.HashMap', 'String', 'java.io.Serializable');
-
-                        _.forEach(spi.stealingAttributes, function(attr) {
-                            res.line(stealingAttrsVar + '.put("' + attr.name + '", "' + attr.value + '");');
-                        });
-
-                        res.needEmptyLine = true;
-
-                        res.line(varName + '.setStealingAttributes(' + stealingAttrsVar + ');');
-                    }
-
-                    break;
-
-                case 'FifoQueue':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'parallelJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'waitingJobsNumber');
-
-                    break;
-
-                case 'PriorityQueue':
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi');
-
-                    $generatorJava.property(res, varName, spi, 'parallelJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'waitingJobsNumber');
-                    $generatorJava.property(res, varName, spi, 'priorityAttributeKey', null, null, 'grid.task.priority');
-                    $generatorJava.property(res, varName, spi, 'jobPriorityAttributeKey', null, null, 'grid.job.priority');
-                    $generatorJava.property(res, varName, spi, 'defaultPriority', null, null, 0);
-                    $generatorJava.property(res, varName, spi, 'starvationIncrement', null, null, 1);
-                    $generatorJava.property(res, varName, spi, 'starvationPreventionEnabled', null, null, true);
-
-                    break;
-
-                case 'Custom':
-                    $generatorJava.declareVariable(res, varName, spi.class);
-
-                    break;
-
-                default:
-            }
-
-            res.needEmptyLine = true;
-
-            res.line('cfg.setCollisionSpi(' + varName + ');');
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    return res;
-};
-
-// Generate communication group.
-$generatorJava.clusterCommunication = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const cfg = $generatorCommon.COMMUNICATION_CONFIGURATION;
-
-    $generatorJava.beanProperty(res, 'cfg', cluster.communication, 'communicationSpi', 'commSpi', cfg.className, cfg.fields);
-
-    res.needEmptyLine = false;
-
-    $generatorJava.property(res, 'cfg', cluster, 'networkTimeout', null, null, 5000);
-    $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryDelay', null, null, 1000);
-    $generatorJava.property(res, 'cfg', cluster, 'networkSendRetryCount', null, null, 3);
-    $generatorJava.property(res, 'cfg', cluster, 'segmentCheckFrequency');
-    $generatorJava.property(res, 'cfg', cluster, 'waitForSegmentOnStart', null, null, false);
-    $generatorJava.property(res, 'cfg', cluster, 'discoveryStartupDelay', null, null, 60000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate REST access group.
-$generatorJava.clusterConnector = function(connector, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(connector) && connector.enabled) {
-        const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION);
-
-        if (connector.sslEnabled) {
-            cfg.fields.sslClientAuth = {dflt: false};
-            cfg.fields.sslFactory = {type: 'bean'};
-        }
-
-        $generatorJava.beanProperty(res, 'cfg', connector, 'connectorConfiguration', 'clientCfg',
-            cfg.className, cfg.fields, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate deployment group.
-$generatorJava.clusterDeployment = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.enumProperty(res, 'cfg', cluster, 'deploymentMode', 'org.apache.ignite.configuration.DeploymentMode', null, 'SHARED');
-
-    res.softEmptyLine();
-
-    const p2pEnabled = cluster.peerClassLoadingEnabled;
-
-    if (!_.isNil(p2pEnabled)) {
-        $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingEnabled', null, null, false);
-
-        if (p2pEnabled) {
-            $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingMissedResourcesCacheSize', null, null, 100);
-            $generatorJava.property(res, 'cfg', cluster, 'peerClassLoadingThreadPoolSize', null, null, 2);
-            $generatorJava.multiparamProperty(res, 'cfg', cluster, 'peerClassLoadingLocalClassPathExclude');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate discovery group.
-$generatorJava.clusterDiscovery = function(disco, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (disco) {
-        $generatorJava.property(res, 'discovery', disco, 'localAddress');
-        $generatorJava.property(res, 'discovery', disco, 'localPort', null, null, 47500);
-        $generatorJava.property(res, 'discovery', disco, 'localPortRange', null, null, 100);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'addressResolver', 'addressResolver', disco.addressResolver, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'socketTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'ackTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'maxAckTimeout', null, null, 600000);
-        $generatorJava.property(res, 'discovery', disco, 'networkTimeout', null, null, 5000);
-        $generatorJava.property(res, 'discovery', disco, 'joinTimeout', null, null, 0);
-        $generatorJava.property(res, 'discovery', disco, 'threadPriority', null, null, 10);
-        $generatorJava.property(res, 'discovery', disco, 'heartbeatFrequency', null, null, 2000);
-        $generatorJava.property(res, 'discovery', disco, 'maxMissedHeartbeats', null, null, 1);
-        $generatorJava.property(res, 'discovery', disco, 'maxMissedClientHeartbeats', null, null, 5);
-        $generatorJava.property(res, 'discovery', disco, 'topHistorySize', null, null, 1000);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.listener)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'listener', 'listener', disco.listener, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'dataExchange', 'dataExchange', disco.dataExchange, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'metricsProvider', 'metricsProvider', disco.metricsProvider, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'reconnectCount', null, null, 10);
-        $generatorJava.property(res, 'discovery', disco, 'statisticsPrintFrequency', null, null, 0);
-        $generatorJava.property(res, 'discovery', disco, 'ipFinderCleanFrequency', null, null, 60000);
-
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator)) {
-            $generatorJava.beanProperty(res, 'discovery', disco, 'authenticator', 'authenticator', disco.authenticator, {}, true);
-            res.needEmptyLine = false;
-        }
-
-        $generatorJava.property(res, 'discovery', disco, 'forceServerMode', null, null, false);
-        $generatorJava.property(res, 'discovery', disco, 'clientReconnectDisabled', null, null, false);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate events group.
-$generatorJava.clusterEvents = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups');
-
-        if (cluster.includeEventTypes.length === 1) {
-            const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]});
-            const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value);
-
-            res.line('cfg.setIncludeEventTypes(' + evts + ');');
-        }
-        else {
-            _.forEach(cluster.includeEventTypes, function(value, ix) {
-                const evtGrp = _.find(evtGrps, {value});
-                const evts = res.importStatic(evtGrp.class + '.' + evtGrp.value);
-
-                if (ix === 0)
-                    res.line('int[] events = new int[' + evts + '.length');
-                else
-                    res.line('    + ' + evts + '.length');
-            });
-
-            res.line('];');
-
-            res.needEmptyLine = true;
-
-            res.line('int k = 0;');
-
-            _.forEach(cluster.includeEventTypes, function(value, idx) {
-                res.needEmptyLine = true;
-
-                const evtGrp = _.find(evtGrps, {value});
-                const evts = res.importStatic(evtGrp.class + '.' + value);
-
-                res.line('System.arraycopy(' + evts + ', 0, events, k, ' + evts + '.length);');
-
-                if (idx < cluster.includeEventTypes.length - 1)
-                    res.line('k += ' + evts + '.length;');
-            });
-
-            res.needEmptyLine = true;
-
-            res.line('cfg.setIncludeEventTypes(events);');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate failover group.
-$generatorJava.clusterFailover = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) {
-        return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')));
-    }) >= 0) {
-        const arrayVarName = 'failoverSpiList';
-
-        $generatorJava.declareVariable(res, arrayVarName, 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.spi.failover.FailoverSpi');
-
-        _.forEach(cluster.failoverSpi, function(spi) {
-            if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) {
-                const varName = 'failoverSpi';
-
-                const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts');
-
-                if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) {
-                    const spiCls = res.importClass($generatorCommon.failoverSpiClass(spi));
-
-                    $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.spi.failover.FailoverSpi', 'new ' + spiCls + '()');
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi[spi.kind].maximumFailoverAttempts))
-                        res.line('((' + spiCls + ') ' + varName + ').setMaximumFailoverAttempts(' + spi[spi.kind].maximumFailoverAttempts + ');');
-
-                    res.needEmptyLine = true;
-
-                    res.line(arrayVarName + '.add(' + varName + ');');
-                }
-                else
-                    res.line(arrayVarName + '.add(new ' + res.importClass($generatorCommon.failoverSpiClass(spi)) + '());');
-
-                res.needEmptyLine = true;
-            }
-        });
-
-        res.line('cfg.setFailoverSpi(' + arrayVarName + '.toArray(new FailoverSpi[' + arrayVarName + '.size()]));');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorJava.clusterLogger = function(logger, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.loggerConfigured(logger)) {
-        const varName = 'logger';
-
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2':
-                $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j2.Log4J2Logger',
-                    'new Log4J2Logger(' + $generatorJava.toJavaCode(log.path, 'path') + ')');
-
-                res.needEmptyLine = true;
-
-                if ($generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line(varName + '.setLevel(' + res.importClass('org.apache.logging.log4j.Level') + '.' + log.level + ');');
-
-                break;
-
-            case 'Null':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.NullLogger');
-
-                break;
-
-            case 'Java':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.java.JavaLogger');
-
-                break;
-
-            case 'JCL':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.jcl.JclLogger');
-
-                break;
-
-            case 'SLF4J':
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.slf4j.Slf4jLogger');
-
-                break;
-
-            case 'Log4j':
-                if (log.mode === 'Default')
-                    $generatorJava.declareVariable(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger');
-                else {
-                    $generatorJava.declareVariableCustom(res, varName, 'org.apache.ignite.logger.log4j.Log4JLogger',
-                        'new Log4JLogger(' + $generatorJava.toJavaCode(log.path, 'path') + ')');
-                }
-
-                if ($generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line(varName + '.setLevel(' + res.importClass('org.apache.log4j.Level') + '.' + log.level + ');');
-
-                break;
-
-            case 'Custom':
-                $generatorJava.declareVariable(res, varName, log.class);
-
-                break;
-
-            default:
-        }
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setGridLogger(' + varName + ');');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorJava.clusterMarshaller = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const marshaller = cluster.marshaller;
-
-    if (marshaller && marshaller.kind) {
-        const marshallerDesc = $generatorCommon.MARSHALLERS[marshaller.kind];
-
-        $generatorJava.beanProperty(res, 'cfg', marshaller[marshaller.kind], 'marshaller', 'marshaller',
-            marshallerDesc.className, marshallerDesc.fields, true);
-
-        $generatorJava.beanProperty(res, 'marshaller', marshaller[marshaller.kind], marshallerDesc.className, marshallerDesc.fields, true);
-    }
-
-    $generatorJava.property(res, 'cfg', cluster, 'marshalLocalJobs', null, null, false);
-    $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheKeepAliveTime', null, null, 10000);
-    $generatorJava.property(res, 'cfg', cluster, 'marshallerCacheThreadPoolSize', null, 'setMarshallerCachePoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate metrics group.
-$generatorJava.clusterMetrics = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'metricsExpireTime');
-    $generatorJava.property(res, 'cfg', cluster, 'metricsHistorySize', null, null, 10000);
-    $generatorJava.property(res, 'cfg', cluster, 'metricsLogFrequency', null, null, 60000);
-    $generatorJava.property(res, 'cfg', cluster, 'metricsUpdateFrequency', null, null, 2000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate swap group.
-$generatorJava.clusterSwap = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') {
-        $generatorJava.beanProperty(res, 'cfg', cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi', 'swapSpi',
-            $generatorCommon.SWAP_SPACE_SPI.className, $generatorCommon.SWAP_SPACE_SPI.fields, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate time group.
-$generatorJava.clusterTime = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'clockSyncSamples', null, null, 8);
-    $generatorJava.property(res, 'cfg', cluster, 'clockSyncFrequency', null, null, 120000);
-    $generatorJava.property(res, 'cfg', cluster, 'timeServerPortBase', null, null, 31100);
-    $generatorJava.property(res, 'cfg', cluster, 'timeServerPortRange', null, null, 100);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate ODBC configuration group.
-$generatorJava.clusterODBC = function(odbc, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (odbc && odbc.odbcEnabled) {
-        $generatorJava.beanProperty(res, 'cfg', odbc, 'odbcConfiguration', 'odbcConfiguration',
-            $generatorCommon.ODBC_CONFIGURATION.className, $generatorCommon.ODBC_CONFIGURATION.fields, true);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate thread pools group.
-$generatorJava.clusterPools = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'cfg', cluster, 'publicThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'systemThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'managementThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'igfsThreadPoolSize');
-    $generatorJava.property(res, 'cfg', cluster, 'rebalanceThreadPoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate transactions group.
-$generatorJava.clusterTransactions = function(transactionConfiguration, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.beanProperty(res, 'cfg', transactionConfiguration, 'transactionConfiguration',
-        'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION.className,
-        $generatorCommon.TRANSACTION_CONFIGURATION.fields, false);
-
-    return res;
-};
-
-// Generate user attributes group.
-$generatorJava.clusterUserAttributes = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) {
-        $generatorJava.declareVariable(res, 'attributes', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(cluster.attributes, function(attr) {
-            res.line('attributes.put("' + attr.name + '", "' + attr.value + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('cfg.setUserAttributes(attributes);');
-
-        res.needEmptyLine = true;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-
-// Generate cache general group.
-$generatorJava.cacheGeneral = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'name');
-
-    $generatorJava.enumProperty(res, varName, cache, 'cacheMode', 'org.apache.ignite.cache.CacheMode');
-    $generatorJava.enumProperty(res, varName, cache, 'atomicityMode', 'org.apache.ignite.cache.CacheAtomicityMode');
-
-    if (cache.cacheMode === 'PARTITIONED' && $generatorJava.property(res, varName, cache, 'backups'))
-        $generatorJava.property(res, varName, cache, 'readFromBackup');
-
-    $generatorJava.property(res, varName, cache, 'copyOnRead');
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL')
-        $generatorJava.property(res, varName, cache, 'invalidate');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache memory group.
-$generatorJava.cacheMemory = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.enumProperty(res, varName, cache, 'memoryMode', 'org.apache.ignite.cache.CacheMemoryMode', null, 'ONHEAP_TIERED');
-
-    if (cache.memoryMode !== 'OFFHEAP_VALUES')
-        $generatorJava.property(res, varName, cache, 'offHeapMaxMemory', null, null, -1);
-
-    res.softEmptyLine();
-
-    $generatorJava.evictionPolicy(res, varName, cache.evictionPolicy, 'evictionPolicy');
-
-    $generatorJava.property(res, varName, cache, 'startSize', null, null, 1500000);
-    $generatorJava.property(res, varName, cache, 'swapEnabled', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache query & indexing group.
-$generatorJava.cacheQuery = function(cache, domains, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'sqlSchema');
-    $generatorJava.property(res, varName, cache, 'sqlOnheapRowCacheSize', null, null, 10240);
-    $generatorJava.property(res, varName, cache, 'longQueryWarningTimeout', null, null, 3000);
-
-    const indexedTypes = _.reduce(domains, (acc, domain) => {
-        if (domain.queryMetadata === 'Annotations') {
-            acc.push(domain.keyType);
-            acc.push(domain.valueType);
-        }
-
-        return acc;
-    }, []);
-
-    if (indexedTypes.length > 0) {
-        res.softEmptyLine();
-
-        $generatorJava.multiparamProperty(res, varName, {indexedTypes}, 'indexedTypes', 'class');
-    }
-
-    res.softEmptyLine();
-
-    $generatorJava.multiparamProperty(res, varName, cache, 'sqlFunctionClasses', 'class');
-
-    res.softEmptyLine();
-
-    $generatorJava.property(res, varName, cache, 'snapshotableIndex', null, null, false);
-    $generatorJava.property(res, varName, cache, 'sqlEscapeAll', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * Generate cache store datasource.
- *
- * @param storeFactory Factory to generate data source for.
- * @param res Resulting output with generated code.
- */
-$generatorJava.cacheStoreDataSource = function(storeFactory, res) {
-    const dialect = storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect;
-
-    if (dialect) {
-        const varName = 'dataSource';
-
-        const dataSourceBean = storeFactory.dataSourceBean;
-
-        const varType = res.importClass($generatorCommon.dataSourceClassName(dialect));
-
-        res.line('public static final ' + varType + ' INSTANCE_' + dataSourceBean + ' = create' + dataSourceBean + '();');
-
-        res.needEmptyLine = true;
-
-        res.startBlock('private static ' + varType + ' create' + dataSourceBean + '() {');
-        if (dialect === 'Oracle')
-            res.startBlock('try {');
-
-        $generatorJava.resetVariables(res);
-
-        $generatorJava.declareVariable(res, varName, varType);
-
-        switch (dialect) {
-            case 'Generic':
-                res.line(varName + '.setJdbcUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-
-                break;
-
-            case 'DB2':
-                res.line(varName + '.setServerName(props.getProperty("' + dataSourceBean + '.jdbc.server_name"));');
-                res.line(varName + '.setPortNumber(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.port_number")));');
-                res.line(varName + '.setDatabaseName(props.getProperty("' + dataSourceBean + '.jdbc.database_name"));');
-                res.line(varName + '.setDriverType(Integer.valueOf(props.getProperty("' + dataSourceBean + '.jdbc.driver_type")));');
-
-                break;
-
-            case 'PostgreSQL':
-                res.line(varName + '.setUrl(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-
-                break;
-
-            default:
-                res.line(varName + '.setURL(props.getProperty("' + dataSourceBean + '.jdbc.url"));');
-        }
-
-        res.line(varName + '.setUser(props.getProperty("' + dataSourceBean + '.jdbc.username"));');
-        res.line(varName + '.setPassword(props.getProperty("' + dataSourceBean + '.jdbc.password"));');
-
-        res.needEmptyLine = true;
-
-        res.line('return dataSource;');
-
-        if (dialect === 'Oracle') {
-            res.endBlock('}');
-            res.startBlock('catch (' + res.importClass('java.sql.SQLException') + ' ex) {');
-            res.line('throw new Error(ex);');
-            res.endBlock('}');
-        }
-
-        res.endBlock('}');
-
-        res.needEmptyLine = true;
-
-        return dataSourceBean;
-    }
-
-    return null;
-};
-
-$generatorJava.clusterDataSources = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const datasources = [];
-
-    let storeFound = false;
-
-    function startSourcesFunction() {
-        if (!storeFound) {
-            res.line('/** Helper class for datasource creation. */');
-            res.startBlock('public static class DataSources {');
-
-            storeFound = true;
-        }
-    }
-
-    _.forEach(cluster.caches, function(cache) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            const beanClassName = $generatorJava.dataSourceClassName(res, storeFactory);
-
-            if (beanClassName && !_.includes(datasources, beanClassName)) {
-                datasources.push(beanClassName);
-
-                if (factoryKind === 'CacheJdbcPojoStoreFactory' || factoryKind === 'CacheJdbcBlobStoreFactory') {
-                    startSourcesFunction();
-
-                    $generatorJava.cacheStoreDataSource(storeFactory, res);
-                }
-            }
-        }
-    });
-
-    if (cluster.discovery.kind === 'Jdbc') {
-        const datasource = cluster.discovery.Jdbc;
-
-        if (datasource.dataSourceBean && datasource.dialect) {
-            const beanClassName = $generatorJava.dataSourceClassName(res, datasource);
-
-            if (beanClassName && !_.includes(datasources, beanClassName)) {
-                startSourcesFunction();
-
-                $generatorJava.cacheStoreDataSource(datasource, res);
-            }
-        }
-    }
-
-    if (storeFound)
-        res.endBlock('}');
-
-    return res;
-};
-
-/**
- * Generate cache store group.
- *
- * @param cache Cache descriptor.
- * @param domains Domain model descriptors.
- * @param cacheVarName Cache variable name.
- * @param res Resulting output with generated code.
- * @returns {*} Java code for cache store configuration.
- */
-$generatorJava.cacheStore = function(cache, domains, cacheVarName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!cacheVarName)
-        cacheVarName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            const storeFactoryDesc = $generatorCommon.STORE_FACTORIES[factoryKind];
-
-            const varName = 'storeFactory' + storeFactoryDesc.suffix;
-
-            if (factoryKind === 'CacheJdbcPojoStoreFactory') {
-                // Generate POJO store factory.
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory', null, null, null, true);
-                res.deep++;
-
-                res.line('/** {@inheritDoc} */');
-                res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStore') + ' create() {');
-
-                res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');');
-
-                res.needEmptyLine = true;
-
-                res.line('return super.create();');
-                res.endBlock('}');
-                res.endBlock('};');
-
-                res.needEmptyLine = true;
-
-                res.line(varName + '.setDialect(new ' +
-                    res.importClass($generatorCommon.jdbcDialectClassName(storeFactory.dialect)) + '());');
-
-                res.needEmptyLine = true;
-
-                if (storeFactory.sqlEscapeAll) {
-                    res.line(varName + '.setSqlEscapeAll(true);');
-
-                    res.needEmptyLine = true;
-                }
-
-                const domainConfigs = _.filter(domains, function(domain) {
-                    return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-                        $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable);
-                });
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-                    $generatorJava.declareVariable(res, 'jdbcTypes', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.store.jdbc.JdbcType');
-
-                    res.needEmptyLine = true;
-
-                    _.forEach(domainConfigs, function(domain) {
-                        if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable))
-                            res.line('jdbcTypes.add(jdbcType' + $generatorJava.extractType(domain.valueType) + '(' + cacheVarName + '.getName()));');
-                    });
-
-                    res.needEmptyLine = true;
-
-                    res.line(varName + '.setTypes(jdbcTypes.toArray(new JdbcType[jdbcTypes.size()]));');
-
-                    res.needEmptyLine = true;
-                }
-
-                res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');');
-            }
-            else if (factoryKind === 'CacheJdbcBlobStoreFactory') {
-                // Generate POJO store factory.
-                $generatorJava.declareVariable(res, varName, 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory', null, null, null, storeFactory.connectVia === 'DataSource');
-
-                if (storeFactory.connectVia === 'DataSource') {
-                    res.deep++;
-
-                    res.line('/** {@inheritDoc} */');
-                    res.startBlock('@Override public ' + res.importClass('org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStore') + ' create() {');
-
-                    res.line('setDataSource(DataSources.INSTANCE_' + storeFactory.dataSourceBean + ');');
-
-                    res.needEmptyLine = true;
-
-                    res.line('return super.create();');
-                    res.endBlock('}');
-                    res.endBlock('};');
-
-                    res.needEmptyLine = true;
-
-                    $generatorJava.property(res, varName, storeFactory, 'initSchema');
-                    $generatorJava.property(res, varName, storeFactory, 'createTableQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'loadQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'insertQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'updateQuery');
-                    $generatorJava.property(res, varName, storeFactory, 'deleteQuery');
-                }
-                else {
-                    $generatorJava.property(res, varName, storeFactory, 'connectionUrl');
-
-                    if (storeFactory.user) {
-                        $generatorJava.property(res, varName, storeFactory, 'user');
-                        res.line(varName + '.setPassword(props.getProperty("ds.' + storeFactory.user + '.password"));');
-                    }
-                }
-
-                res.needEmptyLine = true;
-
-                res.line(cacheVarName + '.setCacheStoreFactory(' + varName + ');');
-            }
-            else
-                $generatorJava.beanProperty(res, cacheVarName, storeFactory, 'cacheStoreFactory', varName, storeFactoryDesc.className, storeFactoryDesc.fields, true);
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    res.softEmptyLine();
-
-    $generatorJava.property(res, cacheVarName, cache, 'storeKeepBinary', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'loadPreviousValue', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'readThrough', null, null, false);
-    $generatorJava.property(res, cacheVarName, cache, 'writeThrough', null, null, false);
-
-    res.softEmptyLine();
-
-    if (cache.writeBehindEnabled) {
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindEnabled', null, null, false);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindBatchSize', null, null, 512);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushSize', null, null, 10240);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushFrequency', null, null, 5000);
-        $generatorJava.property(res, cacheVarName, cache, 'writeBehindFlushThreadCount', null, null, 1);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache node filter group.
-$generatorJava.cacheNodeFilter = function(cache, igfss, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    switch (_.get(cache, 'nodeFilter.kind')) {
-        case 'IGFS':
-            const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs);
-
-            if (foundIgfs) {
-                const predClsName = res.importClass('org.apache.ignite.internal.processors.igfs.IgfsNodePredicate');
-
-                res.line(`${varName}.setNodeFilter(new ${predClsName}("${foundIgfs.name}"));`);
-            }
-
-            break;
-
-        case 'OnNodes':
-            const nodes = cache.nodeFilter.OnNodes.nodeIds;
-
-            if ($generatorCommon.isDefinedAndNotEmpty(nodes)) {
-                const startQuote = res.importClass('java.util.UUID') + '.fromString("';
-
-                $generatorJava.fxVarArgs(res, varName + '.setNodeFilter(new ' +
-                    res.importClass('org.apache.ignite.internal.util.lang.GridNodePredicate'), true, nodes, '(', '))',
-                    startQuote, '")');
-            }
-
-            break;
-
-        case 'Custom':
-            res.line(varName + '.setNodeFilter(new ' + res.importClass(cache.nodeFilter.Custom.className) + '());');
-
-            break;
-
-        default: break;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache concurrency group.
-$generatorJava.cacheConcurrency = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'maxConcurrentAsyncOperations', null, null, 500);
-    $generatorJava.property(res, varName, cache, 'defaultLockTimeout', null, null, 0);
-    $generatorJava.enumProperty(res, varName, cache, 'atomicWriteOrderMode', 'org.apache.ignite.cache.CacheAtomicWriteOrderMode');
-    $generatorJava.enumProperty(res, varName, cache, 'writeSynchronizationMode', 'org.apache.ignite.cache.CacheWriteSynchronizationMode', null, 'PRIMARY_SYNC');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache rebalance group.
-$generatorJava.cacheRebalance = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheMode !== 'LOCAL') {
-        $generatorJava.enumProperty(res, varName, cache, 'rebalanceMode', 'org.apache.ignite.cache.CacheRebalanceMode', null, 'ASYNC');
-        $generatorJava.property(res, varName, cache, 'rebalanceThreadPoolSize', null, null, 1);
-        $generatorJava.property(res, varName, cache, 'rebalanceBatchSize', null, null, 524288);
-        $generatorJava.property(res, varName, cache, 'rebalanceBatchesPrefetchCount', null, null, 2);
-        $generatorJava.property(res, varName, cache, 'rebalanceOrder', null, null, 0);
-        $generatorJava.property(res, varName, cache, 'rebalanceDelay', null, null, 0);
-        $generatorJava.property(res, varName, cache, 'rebalanceTimeout', null, null, 10000);
-        $generatorJava.property(res, varName, cache, 'rebalanceThrottle', null, null, 0);
-    }
-
-    res.softEmptyLine();
-
-    if (cache.igfsAffinnityGroupSize) {
-        res.line(varName + '.setAffinityMapper(new ' + res.importClass('org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper') + '(' + cache.igfsAffinnityGroupSize + '));');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache server near cache group.
-$generatorJava.cacheServerNearCache = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) {
-        res.needEmptyLine = true;
-
-        if (cache.nearConfiguration) {
-            $generatorJava.declareVariable(res, 'nearCfg', 'org.apache.ignite.configuration.NearCacheConfiguration');
-
-            res.needEmptyLine = true;
-
-            if (cache.nearConfiguration.nearStartSize) {
-                $generatorJava.property(res, 'nearCfg', cache.nearConfiguration, 'nearStartSize', null, null, 375000);
-
-                res.needEmptyLine = true;
-            }
-
-            if (cache.nearConfiguration.nearEvictionPolicy && cache.nearConfiguration.nearEvictionPolicy.kind) {
-                $generatorJava.evictionPolicy(res, 'nearCfg', cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy');
-
-                res.needEmptyLine = true;
-            }
-
-            res.line(varName + '.setNearConfiguration(nearCfg);');
-
-            res.needEmptyLine = true;
-        }
-    }
-
-    return res;
-};
-
-// Generate cache statistics group.
-$generatorJava.cacheStatistics = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!varName)
-        varName = $generatorJava.nextVariableName('cache', cache);
-
-    $generatorJava.property(res, varName, cache, 'statisticsEnabled', null, null, false);
-    $generatorJava.property(res, varName, cache, 'managementEnabled', null, null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model query fields.
-$generatorJava.domainModelQueryFields = function(res, domain) {
-    const fields = domain.fields;
-
-    if (fields && fields.length > 0) {
-        $generatorJava.declareVariable(res, 'fields', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(fields, function(field) {
-            res.line('fields.put("' + field.name + '", "' + $generatorCommon.JavaTypes.fullClassName(field.className) + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setFields(fields);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model query aliases.
-$generatorJava.domainModelQueryAliases = function(res, domain) {
-    const aliases = domain.aliases;
-
-    if (aliases && aliases.length > 0) {
-        $generatorJava.declareVariable(res, 'aliases', 'java.util.Map', 'java.util.HashMap', 'java.lang.String', 'java.lang.String');
-
-        _.forEach(aliases, function(alias) {
-            res.line('aliases.put("' + alias.field + '", "' + alias.alias + '");');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setAliases(aliases);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model indexes.
-$generatorJava.domainModelQueryIndexes = function(res, domain) {
-    const indexes = domain.indexes;
-
-    if (indexes && indexes.length > 0) {
-        res.needEmptyLine = true;
-
-        $generatorJava.declareVariable(res, 'indexes', 'java.util.List', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryIndex');
-
-        _.forEach(indexes, function(index) {
-            const fields = index.fields;
-
-            // One row generation for 1 field index.
-            if (fields && fields.length === 1) {
-                const field = index.fields[0];
-
-                res.line('indexes.add(new ' + res.importClass('org.apache.ignite.cache.QueryIndex') +
-                    '("' + field.name + '", ' +
-                    res.importClass('org.apache.ignite.cache.QueryIndexType') + '.' + index.indexType + ', ' +
-                    field.direction + ', "' + index.name + '"));');
-            }
-            else {
-                res.needEmptyLine = true;
-
-                $generatorJava.declareVariable(res, 'index', 'org.apache.ignite.cache.QueryIndex');
-
-                $generatorJava.property(res, 'index', index, 'name');
-                $generatorJava.enumProperty(res, 'index', index, 'indexType', 'org.apache.ignite.cache.QueryIndexType');
-
-                res.needEmptyLine = true;
-
-                if (fields && fields.length > 0) {
-                    $generatorJava.declareVariable(res, 'indFlds', 'java.util.LinkedHashMap', 'java.util.LinkedHashMap', 'String', 'Boolean');
-
-                    _.forEach(fields, function(field) {
-                        res.line('indFlds.put("' + field.name + '", ' + field.direction + ');');
-                    });
-
-                    res.needEmptyLine = true;
-
-                    res.line('index.setFields(indFlds);');
-
-                    res.needEmptyLine = true;
-                }
-
-                res.line('indexes.add(index);');
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.line('qryMeta.setIndexes(indexes);');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model db fields.
-$generatorJava.domainModelDatabaseFields = function(res, domain, fieldProperty) {
-    const dbFields = domain[fieldProperty];
-
-    if (dbFields && dbFields.length > 0) {
-        res.needEmptyLine = true;
-
-        res.importClass('java.sql.Types');
-
-        res.startBlock('jdbcType.' + $generatorCommon.toJavaName('set', fieldProperty) + '(');
-
-        const lastIx = dbFields.length - 1;
-
-        res.importClass('org.apache.ignite.cache.store.jdbc.JdbcTypeField');
-
-        _.forEach(dbFields, function(field, ix) {
-            res.line('new JdbcTypeField(' +
-                'Types.' + field.databaseFieldType + ', ' + '"' + field.databaseFieldName + '", ' +
-                res.importClass(field.javaFieldType) + '.class, ' + '"' + field.javaFieldName + '"' + ')' + (ix < lastIx ? ',' : ''));
-        });
-
-        res.endBlock(');');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model general group.
-$generatorJava.domainModelGeneral = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    switch ($generatorCommon.domainQueryMetadata(domain)) {
-        case 'Annotations':
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) {
-                const types = [];
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType))
-                    types.push($generatorJava.toJavaCode(res.importClass(domain.keyType), 'class'));
-                else
-                    types.push('???');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType))
-                    types.push($generatorJava.toJavaCode(res.importClass(domain.valueType), 'class'));
-                else
-                    types.push('???');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(types))
-                    $generatorJava.fxVarArgs(res, 'cache.setIndexedTypes', false, types);
-            }
-
-            break;
-
-        case 'Configuration':
-            $generatorJava.classNameProperty(res, 'jdbcTypes', domain, 'keyType');
-            $generatorJava.property(res, 'jdbcTypes', domain, 'valueType');
-
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.fields)) {
-                res.needEmptyLine = true;
-
-                $generatorJava.classNameProperty(res, 'qryMeta', domain, 'keyType');
-                $generatorJava.property(res, 'qryMeta', domain, 'valueType');
-            }
-
-            break;
-
-        default:
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model for query group.
-$generatorJava.domainModelQuery = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') {
-        $generatorJava.domainModelQueryFields(res, domain);
-        $generatorJava.domainModelQueryAliases(res, domain);
-        $generatorJava.domainModelQueryIndexes(res, domain);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate domain model for store group.
-$generatorJava.domainStore = function(domain, withTypes, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.property(res, 'jdbcType', domain, 'databaseSchema');
-    $generatorJava.property(res, 'jdbcType', domain, 'databaseTable');
-
-    if (withTypes) {
-        $generatorJava.classNameProperty(res, 'jdbcType', domain, 'keyType');
-        $generatorJava.property(res, 'jdbcType', domain, 'valueType');
-    }
-
-    $generatorJava.domainModelDatabaseFields(res, domain, 'keyFields');
-    $generatorJava.domainModelDatabaseFields(res, domain, 'valueFields');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model configs.
-$generatorJava.cacheDomains = function(domains, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const domainConfigs = _.filter(domains, function(domain) {
-        return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-            $generatorCommon.isDefinedAndNotEmpty(domain.fields);
-    });
-
-    // Generate domain model configs.
-    if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-        $generatorJava.declareVariable(res, 'queryEntities', 'java.util.Collection', 'java.util.ArrayList', 'org.apache.ignite.cache.QueryEntity');
-
-        _.forEach(domainConfigs, function(domain) {
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.fields))
-                res.line('queryEntities.add(queryEntity' + $generatorJava.extractType(domain.valueType) + '());');
-        });
-
-        res.needEmptyLine = true;
-
-        res.line(varName + '.setQueryEntities(queryEntities);');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorJava.cache = function(cache, varName, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorJava.cacheGeneral(cache, varName, res);
-    $generatorJava.cacheMemory(cache, varName, res);
-    $generatorJava.cacheQuery(cache, cache.domains, varName, res);
-    $generatorJava.cacheStore(cache, cache.domains, varName, res);
-
-    const igfs = _.get(cache, 'nodeFilter.IGFS.instance');
-
-    $generatorJava.cacheNodeFilter(cache, igfs ? [igfs] : [], varName, res);
-    $generatorJava.cacheConcurrency(cache, varName, res);
-    $generatorJava.cacheRebalance(cache, varName, res);
-    $generatorJava.cacheServerNearCache(cache, varName, res);
-    $generatorJava.cacheStatistics(cache, varName, res);
-    $generatorJava.cacheDomains(cache.domains, varName, res);
-};
-
-// Generation of cache domain model in separate methods.
-$generatorJava.clusterDomains = function(caches, res) {
-    const domains = [];
-
-    const typeVarName = 'jdbcType';
-    const metaVarName = 'qryMeta';
-
-    _.forEach(caches, function(cache) {
-        _.forEach(cache.domains, function(domain) {
-            if (_.isNil(_.find(domains, function(m) {
-                return m === domain.valueType;
-            }))) {
-                $generatorJava.resetVariables(res);
-
-                const type = $generatorJava.extractType(domain.valueType);
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.databaseTable)) {
-                    res.line('/**');
-                    res.line(' * Create JDBC type for ' + type + '.');
-                    res.line(' *');
-                    res.line(' * @param cacheName Cache name.');
-                    res.line(' * @return Configured JDBC type.');
-                    res.line(' */');
-                    res.startBlock('private static JdbcType jdbcType' + type + '(String cacheName) {');
-
-                    $generatorJava.declareVariable(res, typeVarName, 'org.apache.ignite.cache.store.jdbc.JdbcType');
-
-                    res.needEmptyLine = true;
-
-                    res.line(typeVarName + '.setCacheName(cacheName);');
-
-                    $generatorJava.domainStore(domain, true, res);
-
-                    res.needEmptyLine = true;
-
-                    res.line('return ' + typeVarName + ';');
-                    res.endBlock('}');
-
-                    res.needEmptyLine = true;
-                }
-
-                if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-                    $generatorCommon.isDefinedAndNotEmpty(domain.fields)) {
-                    res.line('/**');
-                    res.line(' * Create SQL Query descriptor for ' + type + '.');
-                    res.line(' *');
-                    res.line(' * @return Configured query entity.');
-                    res.line(' */');
-                    res.startBlock('private static QueryEntity queryEntity' + type + '() {');
-
-                    $generatorJava.declareVariable(res, metaVarName, 'org.apache.ignite.cache.QueryEntity');
-
-                    $generatorJava.classNameProperty(res, metaVarName, domain, 'keyType');
-                    $generatorJava.property(res, metaVarName, domain, 'valueType');
-
-                    res.needEmptyLine = true;
-
-                    $generatorJava.domainModelQuery(domain, res);
-
-                    res.emptyLineIfNeeded();
-                    res.line('return ' + metaVarName + ';');
-
-                    res.needEmptyLine = true;
-
-                    res.endBlock('}');
-                }
-
-                domains.push(domain.valueType);
-            }
-        });
-    });
-};
-
-/**
- * @param prefix Variable prefix.
- * @param obj Object to process.
- * @param names Known names to generate next unique name.
- */
-$generatorJava.nextVariableName = function(prefix, obj, names) {
-    let nextName = $generatorCommon.toJavaName(prefix, obj.name);
-
-    let ix = 0;
-
-    const checkNextName = (name) => name === nextName + (ix === 0 ? '' : '_' + ix);
-
-    while (_.find(names, (name) => checkNextName(name)))
-        ix++;
-
-    if (ix > 0)
-        nextName = nextName + '_' + ix;
-
-    return nextName;
-};
-
-// Generate cluster caches.
-$generatorJava.clusterCaches = function(caches, igfss, isSrvCfg, res) {
-    function clusterCache(cache, names) {
-        res.emptyLineIfNeeded();
-
-        const cacheName = $generatorJava.nextVariableName('cache', cache, names);
-
-        $generatorJava.resetVariables(res);
-
-        const hasDatasource = $generatorCommon.cacheHasDatasource(cache);
-
-        res.line('/**');
-        res.line(' * Create configuration for cache "' + cache.name + '".');
-        res.line(' *');
-        res.line(' * @return Configured cache.');
-
-        if (hasDatasource)
-            res.line(' * @throws Exception if failed to create cache configuration.');
-
-        res.line(' */');
-        res.startBlock('public static CacheConfiguration ' + cacheName + '()' + (hasDatasource ? ' throws Exception' : '') + ' {');
-
-        $generatorJava.declareVariable(res, cacheName, 'org.apache.ignite.configuration.CacheConfiguration');
-
-        $generatorJava.cache(cache, cacheName, res);
-
-        res.line('return ' + cacheName + ';');
-        res.endBlock('}');
-
-        names.push(cacheName);
-
-        res.needEmptyLine = true;
-    }
-
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const names = [];
-
-    if ($generatorCommon.isDefinedAndNotEmpty(caches)) {
-        res.emptyLineIfNeeded();
-
-        _.forEach(caches, function(cache) {
-            clusterCache(cache, names);
-        });
-
-        res.needEmptyLine = true;
-    }
-
-    if (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss)) {
-        res.emptyLineIfNeeded();
-
-        _.forEach(igfss, function(igfs) {
-            clusterCache($generatorCommon.igfsDataCache(igfs), names);
-            clusterCache($generatorCommon.igfsMetaCache(igfs), names);
-        });
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cluster caches.
-$generatorJava.clusterCacheUse = function(caches, igfss, res) {
-    function clusterCacheInvoke(cache, names) {
-        names.push($generatorJava.nextVariableName('cache', cache, names));
-    }
-
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const cacheNames = [];
-
-    _.forEach(caches, function(cache) {
-        clusterCacheInvoke(cache, cacheNames);
-    });
-
-    const igfsNames = [];
-
-    _.forEach(igfss, function(igfs) {
-        clusterCacheInvoke($generatorCommon.igfsDataCache(igfs), igfsNames);
-        clusterCacheInvoke($generatorCommon.igfsMetaCache(igfs), igfsNames);
-    });
-
-    const allCacheNames = cacheNames.concat(igfsNames);
-
-    if (allCacheNames.length) {
-        res.line('cfg.setCacheConfiguration(' + allCacheNames.join('(), ') + '());');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Get class name from fully specified class path.
-$generatorJava.extractType = function(fullType) {
-    return fullType.substring(fullType.lastIndexOf('.') + 1);
-};
-
-/**
- * Generate java class code.
- *
- * @param domain Domain model object.
- * @param key If 'true' then key class should be generated.
- * @param pkg Package name.
- * @param useConstructor If 'true' then empty and full constructors should be generat

<TRUNCATED>

[02/50] [abbrv] ignite git commit: IGNITE-4341: Hadoop: added Terasort to unit tests. This closes #1302. This closes #1321.

Posted by vo...@apache.org.
IGNITE-4341: Hadoop: added Terasort to unit tests. This closes #1302. This closes #1321.


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

Branch: refs/heads/ignite-2.0
Commit: b44baf1e8c42c57fa4e241d5943593fa4ae42f12
Parents: ffe53eb
Author: iveselovskiy <iv...@gridgain.com>
Authored: Mon Dec 12 16:52:47 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:29 2016 +0300

----------------------------------------------------------------------
 modules/hadoop/pom.xml                          |   7 +
 .../hadoop/impl/fs/HadoopFileSystemsUtils.java  |  11 +
 .../impl/v2/HadoopV2JobResourceManager.java     |  25 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |  13 +-
 .../impl/HadoopAbstractWordCountTest.java       |   6 +-
 .../hadoop/impl/HadoopFileSystemsTest.java      |   9 +
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |   4 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |   4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         | 376 +++++++++++++++++++
 .../client/HadoopClientProtocolSelfTest.java    |   4 +-
 .../collections/HadoopSkipListSelfTest.java     |  14 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   2 +
 .../testsuites/IgniteHadoopTestSuite.java       |   3 +
 13 files changed, 450 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index d0b0481..db302d7 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -95,6 +95,13 @@
         </dependency>
 
         <dependency>
+            <groupId>org.apache.hadoop</groupId>
+            <artifactId>hadoop-mapreduce-examples</artifactId>
+            <version>${hadoop.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
             <groupId>org.gridgain</groupId>
             <artifactId>ignite-shmem</artifactId>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java
index 5115cb4..37902f0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/fs/HadoopFileSystemsUtils.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal.processors.hadoop.impl.fs;
 
+import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsConstants;
 import org.jetbrains.annotations.Nullable;
 
@@ -48,4 +50,13 @@ public class HadoopFileSystemsUtils {
     public static String disableFsCachePropertyName(@Nullable String scheme) {
         return String.format("fs.%s.impl.disable.cache", scheme);
     }
+
+    /**
+     * Clears Hadoop {@link FileSystem} cache.
+     *
+     * @throws IOException On error.
+     */
+    public static void clearFileSystemCache() throws IOException {
+        FileSystem.closeAll();
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java
index 3984f83..52e394b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2JobResourceManager.java
@@ -123,7 +123,9 @@ class HadoopV2JobResourceManager {
 
             JobConf cfg = ctx.getJobConf();
 
-            String mrDir = cfg.get("mapreduce.job.dir");
+            Collection<URL> clsPathUrls = new ArrayList<>();
+
+            String mrDir = cfg.get(MRJobConfig.MAPREDUCE_JOB_DIR);
 
             if (mrDir != null) {
                 stagingDir = new Path(new URI(mrDir));
@@ -144,28 +146,23 @@ class HadoopV2JobResourceManager {
 
                 File jarJobFile = new File(jobLocDir, "job.jar");
 
-                Collection<URL> clsPathUrls = new ArrayList<>();
-
                 clsPathUrls.add(jarJobFile.toURI().toURL());
 
                 rsrcSet.add(jarJobFile);
                 rsrcSet.add(new File(jobLocDir, "job.xml"));
-
-                processFiles(jobLocDir, ctx.getCacheFiles(), download, false, null, MRJobConfig.CACHE_LOCALFILES);
-                processFiles(jobLocDir, ctx.getCacheArchives(), download, true, null, MRJobConfig.CACHE_LOCALARCHIVES);
-                processFiles(jobLocDir, ctx.getFileClassPaths(), download, false, clsPathUrls, null);
-                processFiles(jobLocDir, ctx.getArchiveClassPaths(), download, true, clsPathUrls, null);
-
-                if (!clsPathUrls.isEmpty()) {
-                    clsPath = new URL[clsPathUrls.size()];
-
-                    clsPathUrls.toArray(clsPath);
-                }
             }
             else if (!jobLocDir.mkdirs())
                 throw new IgniteCheckedException("Failed to create local job directory: "
                     + jobLocDir.getAbsolutePath());
 
+            processFiles(jobLocDir, ctx.getCacheFiles(), download, false, null, MRJobConfig.CACHE_LOCALFILES);
+            processFiles(jobLocDir, ctx.getCacheArchives(), download, true, null, MRJobConfig.CACHE_LOCALARCHIVES);
+            processFiles(jobLocDir, ctx.getFileClassPaths(), download, false, clsPathUrls, null);
+            processFiles(jobLocDir, ctx.getArchiveClassPaths(), download, true, clsPathUrls, null);
+
+            if (!clsPathUrls.isEmpty())
+                clsPath = clsPathUrls.toArray(new URL[clsPathUrls.size()]);
+
             setLocalFSWorkingDirectory(jobLocDir);
         }
         catch (URISyntaxException | IOException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java
index 12351c6..5666cbc 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractSelfTest.java
@@ -83,7 +83,9 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
     private static String initCp;
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
+    @Override protected final void beforeTestsStarted() throws Exception {
+        HadoopFileSystemsUtils.clearFileSystemCache();
+
         // Add surefire classpath to regular classpath.
         initCp = System.getProperty("java.class.path");
 
@@ -93,6 +95,15 @@ public abstract class HadoopAbstractSelfTest extends GridCommonAbstractTest {
             System.setProperty("java.class.path", initCp + File.pathSeparatorChar + surefireCp);
 
         super.beforeTestsStarted();
+
+        beforeTestsStarted0();
+    }
+
+    /**
+     * Performs additional initialization in the beginning of test class execution.
+     */
+    protected void beforeTestsStarted0() throws Exception {
+        // noop
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java
index 3cb8f91..84e6aee 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopAbstractWordCountTest.java
@@ -49,9 +49,7 @@ public abstract class HadoopAbstractWordCountTest extends HadoopAbstractSelfTest
     protected IgfsEx igfs;
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
+    @Override protected void beforeTestsStarted0() throws Exception {
         Configuration cfg = new Configuration();
 
         setupFileSystems(cfg);
@@ -62,6 +60,8 @@ public abstract class HadoopAbstractWordCountTest extends HadoopAbstractSelfTest
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
         igfs = (IgfsEx)startGrids(gridCount()).fileSystem(igfsName);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java
index 252d6cb..7680690 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopFileSystemsTest.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopFileSystemsUtils;
+import org.apache.ignite.internal.processors.hadoop.impl.fs.HadoopLocalFileSystemV1;
 import org.apache.ignite.testframework.GridTestUtils;
 
 /**
@@ -37,11 +38,15 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
         startGrids(gridCount());
     }
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
         stopAllGrids(true);
     }
 
@@ -70,6 +75,10 @@ public class HadoopFileSystemsTest extends HadoopAbstractSelfTest {
         cfg.set(HadoopFileSystemsUtils.LOC_FS_WORK_DIR_PROP,
             new Path(new Path(uri), "user/" + System.getProperty("user.name")).toString());
 
+        FileSystem fs = FileSystem.get(uri, cfg);
+
+        assertTrue(fs instanceof HadoopLocalFileSystemV1);
+
         final CountDownLatch changeUserPhase = new CountDownLatch(THREAD_COUNT);
         final CountDownLatch changeDirPhase = new CountDownLatch(THREAD_COUNT);
         final CountDownLatch changeAbsDirPhase = new CountDownLatch(THREAD_COUNT);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
index a3bf49c..91ad5ec 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopJobTrackerSelfTest.java
@@ -65,9 +65,7 @@ public class HadoopJobTrackerSelfTest extends HadoopAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
+    @Override protected void beforeTestsStarted0() throws Exception {
         startGrids(gridCount());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java
index 027f921..9d45b03 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTaskExecutionSelfTest.java
@@ -83,9 +83,7 @@ public class HadoopTaskExecutionSelfTest extends HadoopAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
+    @Override protected void beforeTestsStarted0() throws Exception {
         startGrids(gridCount());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
new file mode 100644
index 0000000..0cc9564
--- /dev/null
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.impl;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.URI;
+import java.util.UUID;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.examples.terasort.TeraGen;
+import org.apache.hadoop.examples.terasort.TeraInputFormat;
+import org.apache.hadoop.examples.terasort.TeraOutputFormat;
+import org.apache.hadoop.examples.terasort.TeraSort;
+import org.apache.hadoop.examples.terasort.TeraValidate;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.Partitioner;
+import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.HadoopConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+
+import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo;
+
+/**
+ * Implements TeraSort Hadoop sample as a unit test.
+ */
+public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
+    /** Copy of Hadoop constant of package-private visibility. */
+    public static final String PARTITION_FILENAME = getPartitionFileNameConstant();
+
+    /**  Out destination dir. */
+    protected final String generateOutDir = getFsBase() + "/tera-generated";
+
+    /** Sort destination dir. */
+    protected final String sortOutDir = getFsBase() + "/tera-sorted";
+
+    /** Validation destination dir. */
+    protected final String validateOutDir = getFsBase() + "/tera-validated";
+
+    /**
+     * Extracts value of Hadoop package-private constant.
+     *
+     * @return TeraInputFormat.PARTITION_FILENAME.
+     */
+    private static String getPartitionFileNameConstant() {
+        try {
+            Field f = TeraInputFormat.class.getDeclaredField("PARTITION_FILENAME");
+
+            f.setAccessible(true);
+
+            return (String)f.get(null);
+        }
+        catch (Exception e) {
+            throw new IgniteException(e);
+        }
+    }
+
+    /**
+     * Gets base directory.
+     * Note that this directory will be completely deleted in the and of the test.
+     * @return The base directory.
+     */
+    protected String getFsBase() {
+        return "file:///tmp/" + getUser() + "/hadoop-terasort-test";
+    }
+
+    /**
+     * @return Full input data size, in bytes.
+     */
+    protected long dataSizeBytes() {
+        return 100_000_000;
+    }
+
+    /**
+     * Desired number of maps in TeraSort job.
+     * @return The number of maps.
+     */
+    protected int numMaps() {
+        return gridCount() * 10;
+    }
+
+    /**
+     * Desired number of reduces in TeraSort job.
+     * @return The number of reduces.
+     */
+    protected int numReduces() {
+        return gridCount() * 8;
+    }
+
+    /**
+     * The user to run Hadoop job on behalf of.
+     * @return The user to run Hadoop job on behalf of.
+     */
+    protected String getUser() {
+        return System.getProperty("user.name");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids(true);
+
+        // Delete files used:
+        getFileSystem().delete(new Path(getFsBase()), true);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected final boolean igfsEnabled() {
+        return false;
+    }
+
+    /**
+     * Does actual test TeraSort job Through Ignite API
+     */
+    protected final void teraSort() throws Exception {
+        System.out.println("TeraSort ===============================================================");
+
+        getFileSystem().delete(new Path(sortOutDir), true);
+
+        final JobConf jobConf = new JobConf();
+
+        jobConf.setUser(getUser());
+
+        jobConf.set("fs.defaultFS", getFsBase());
+
+        log().info("Desired number of reduces: " + numReduces());
+
+        jobConf.set("mapreduce.job.reduces", String.valueOf(numReduces()));
+
+        log().info("Desired number of maps: " + numMaps());
+
+        final long splitSize = dataSizeBytes() / numMaps();
+
+        log().info("Desired split size: " + splitSize);
+
+        // Force the split to be of the desired size:
+        jobConf.set("mapred.min.split.size", String.valueOf(splitSize));
+        jobConf.set("mapred.max.split.size", String.valueOf(splitSize));
+
+        Job job = setupConfig(jobConf);
+
+        HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 1);
+
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+
+        fut.get();
+    }
+
+    /**
+     * Gets the file system we work upon.
+     * @return The file system.
+     * @throws Exception
+     */
+    FileSystem getFileSystem() throws Exception{
+        return FileSystem.get(new URI(getFsBase()), new Configuration());
+    }
+
+    /**
+     * Represents the data generation stage.
+     * @throws Exception
+     */
+    private void teraGenerate() throws Exception {
+        System.out.println("TeraGenerate ===============================================================");
+
+        getFileSystem().delete(new Path(generateOutDir), true);
+
+        final long numLines = dataSizeBytes() / 100; // TeraGen makes 100 bytes ber line
+
+        if (numLines < 1)
+            throw new IllegalStateException("Data size is too small: " + dataSizeBytes());
+
+        // Generate input data:
+        int res = ToolRunner.run(new Configuration(), new TeraGen(), new String[] {"-Dmapreduce.framework.name=local",
+            String.valueOf(numLines), generateOutDir});
+
+        assertEquals(0, res);
+
+        FileStatus[] fileStatuses = getFileSystem().listStatus(new Path(generateOutDir));
+
+        long sumLen = 0;
+
+        for (FileStatus fs: fileStatuses)
+            sumLen += fs.getLen();
+
+        assertEquals(dataSizeBytes(), sumLen); // Ensure correct size data is generated.
+    }
+
+    /**
+     * Creates Job instance and sets up necessary properties for it.
+     * @param conf The Job config.
+     * @return The job.
+     * @throws Exception On error.
+     */
+    private Job setupConfig(JobConf conf) throws Exception {
+        Job job = Job.getInstance(conf);
+
+        Path inputDir = new Path(generateOutDir);
+        Path outputDir = new Path(sortOutDir);
+
+        boolean useSimplePartitioner = TeraSort.getUseSimplePartitioner(job);
+
+        TeraInputFormat.setInputPaths(job, inputDir);
+        FileOutputFormat.setOutputPath(job, outputDir);
+
+        job.setJobName("TeraSort");
+
+        job.setOutputKeyClass(Text.class);
+        job.setOutputValueClass(Text.class);
+
+        job.setInputFormatClass(TeraInputFormat.class);
+        job.setOutputFormatClass(TeraOutputFormat.class);
+
+        if (useSimplePartitioner)
+            job.setPartitionerClass(TeraSort.SimplePartitioner.class);
+        else {
+            long start = System.currentTimeMillis();
+
+            Path partFile = new Path(outputDir, PARTITION_FILENAME);
+
+            URI partUri = new URI(partFile.toString() + "#" + PARTITION_FILENAME);
+
+            try {
+                TeraInputFormat.writePartitionFile(job, partFile);
+            } catch (Throwable e) {
+                throw new RuntimeException(e);
+            }
+
+            job.addCacheFile(partUri);
+
+            long end = System.currentTimeMillis();
+
+            System.out.println("Spent " + (end - start) + "ms computing partitions. " +
+                "Partition file added to distributed cache: " + partUri);
+
+            job.setPartitionerClass(getTeraSortTotalOrderPartitioner()/*TeraSort.TotalOrderPartitioner.class*/);
+        }
+
+        job.getConfiguration().setInt("dfs.replication", TeraSort.getOutputReplication(job));
+
+        /* TeraOutputFormat.setFinalSync(job, true); */
+        Method m = TeraOutputFormat.class.getDeclaredMethod("setFinalSync", JobContext.class, boolean.class);
+        m.setAccessible(true);
+        m.invoke(null, job, true);
+
+        return job;
+    }
+
+    /**
+     * Extracts package-private TeraSort total order partitioner class.
+     *
+     * @return The class.
+     */
+    @SuppressWarnings("unchecked")
+    private Class<? extends Partitioner> getTeraSortTotalOrderPartitioner() {
+        Class[] classes = TeraSort.class.getDeclaredClasses();
+
+        Class<? extends Partitioner> totalOrderPartitionerCls = null;
+
+        for (Class<?> x: classes) {
+            if ("TotalOrderPartitioner".equals(x.getSimpleName())) {
+                totalOrderPartitionerCls = (Class<? extends Partitioner>)x;
+
+                break;
+            }
+        }
+
+        if (totalOrderPartitionerCls == null)
+            throw new IllegalStateException("Failed to find TeraSort total order partitioner class.");
+
+        return totalOrderPartitionerCls;
+    }
+
+    /**
+     * Implements validation phase of the sample.
+     * @throws Exception
+     */
+    private void teraValidate() throws Exception {
+        System.out.println("TeraValidate ===============================================================");
+
+        getFileSystem().delete(new Path(validateOutDir), true);
+
+        // Generate input data:
+        int res = ToolRunner.run(new Configuration(), new TeraValidate(),
+            new String[] {"-Dmapreduce.framework.name=local", sortOutDir, validateOutDir});
+
+        assertEquals(0, res);
+
+        FileStatus[] fileStatuses = getFileSystem().listStatus(new Path(validateOutDir), new PathFilter() {
+            @Override public boolean accept(Path path) {
+                // Typically name is "part-r-00000":
+                return path.getName().startsWith("part-r-");
+            }
+        });
+
+        // TeraValidate has only 1 reduce, so should be only 1 result file:
+        assertEquals(1, fileStatuses.length);
+
+        // The result file must contain only 1 line with the checksum, like this:
+        // "checksum        7a27e2d0d55de",
+        // typically it has length of 23 bytes.
+        // If sorting was not correct, the result contains list of K-V pairs that are not ordered correctly.
+        // In such case the size of the output will be much larger.
+        long len = fileStatuses[0].getLen();
+
+        assertTrue("TeraValidate length: " + len, len >= 16 && len <= 32);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        getFileSystem().delete(new Path(getFsBase()), true);
+
+        startGrids(gridCount());
+    }
+
+    /**
+     * Runs generate/sort/validate phases of the terasort sample.
+     * @throws Exception
+     */
+    public void testTeraSort() throws Exception {
+        teraGenerate();
+
+        teraSort();
+
+        teraValidate();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration igc = super.getConfiguration(gridName);
+
+        HadoopConfiguration hc = createHadoopConfiguration();
+
+        igc.setHadoopConfiguration(hc);
+
+        return igc;
+    }
+
+    /**
+     * Creates Hadoop configuration for the test.
+     * @return The {@link HadoopConfiguration}.
+     */
+    protected HadoopConfiguration createHadoopConfiguration() {
+        HadoopConfiguration hadoopCfg = new HadoopConfiguration();
+
+        // See org.apache.ignite.configuration.HadoopConfiguration.DFLT_MAX_TASK_QUEUE_SIZE
+        hadoopCfg.setMaxTaskQueueSize(30_000);
+
+        return hadoopCfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
index 7156a3d..44fc46e 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/client/HadoopClientProtocolSelfTest.java
@@ -99,9 +99,7 @@ public class HadoopClientProtocolSelfTest extends HadoopAbstractSelfTest {
     }
 
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        super.beforeTestsStarted();
-
+    @Override protected void beforeTestsStarted0() throws Exception {
         startGrids(gridCount());
 
         setupLockFile.delete();

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java
index 111ea78..1138803 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopSkipListSelfTest.java
@@ -85,6 +85,9 @@ public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
         }
     }
 
+    /**
+     * @throws Exception On error.
+     */
     public void testMapSimple() throws Exception {
         GridUnsafeMemory mem = new GridUnsafeMemory(0);
 
@@ -139,7 +142,16 @@ public class HadoopSkipListSelfTest extends HadoopAbstractMapTest {
         assertEquals(0, mem.allocatedSize());
     }
 
-    private void check(HadoopMultimap m, Multimap<Integer, Integer> mm, final Multimap<Integer, Integer> vis, HadoopTaskContext taskCtx)
+    /**
+     * Check.
+     * @param m The multimap.
+     * @param mm The multimap storing expectations.
+     * @param vis The multimap to store visitor results.
+     * @param taskCtx The task context.
+     * @throws Exception On error.
+     */
+    private void check(HadoopMultimap m, Multimap<Integer, Integer> mm, final Multimap<Integer, Integer> vis,
+        HadoopTaskContext taskCtx)
         throws Exception {
         final HadoopTaskInput in = m.input(taskCtx);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
index 7c43500..5f64ce7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/taskexecutor/external/HadoopExternalTaskExecutionSelfTest.java
@@ -57,6 +57,8 @@ public class HadoopExternalTaskExecutionSelfTest extends HadoopAbstractSelfTest
     @Override protected void beforeTest() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-404");
 
+        super.beforeTest();
+
         startGrids(gridCount());
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b44baf1e/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 959bc59..6046cc1 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -23,6 +23,7 @@ import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
 import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.processors.hadoop.HadoopTestClassLoader;
+import org.apache.ignite.internal.processors.hadoop.impl.HadoopTeraSortTest;
 import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolEmbeddedSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolMultipleServersSelfTest;
 import org.apache.ignite.internal.processors.hadoop.impl.client.HadoopClientProtocolSelfTest;
@@ -123,6 +124,8 @@ public class IgniteHadoopTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(ldr.loadClass(KerberosHadoopFileSystemFactorySelfTest.class.getName())));
 
+        suite.addTest(new TestSuite(ldr.loadClass(HadoopTeraSortTest.class.getName())));
+
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyTest.class.getName())));
         suite.addTest(new TestSuite(ldr.loadClass(HadoopSnappyFullMapReduceTest.class.getName())));
 


[32/50] [abbrv] ignite git commit: IGNITE-4439: Added missing SUID.

Posted by vo...@apache.org.
IGNITE-4439: Added missing SUID.


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

Branch: refs/heads/ignite-2.0
Commit: 1e2469fa80aeb554b12ff5e7c3465260f900dfc4
Parents: e2ac666
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 23 12:25:21 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 23 12:25:21 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/util/AttributeNodeFilter.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1e2469fa/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
index e2b972b..fed0d43 100644
--- a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
@@ -65,6 +65,9 @@ import org.jetbrains.annotations.Nullable;
  * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}.
  */
 public class AttributeNodeFilter implements IgnitePredicate<ClusterNode> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
     /** Attributes. */
     private final Map<String, Object> attrs;
 


[07/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index b0353a6..5345a9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.spi.communication.tcp;
 
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractConfigTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
 
@@ -47,6 +45,9 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "unacknowledgedMessagesBufferSize", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", 0);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", Integer.MAX_VALUE);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index a74a07a..0c57105 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -109,7 +109,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     /**
      * @param useShmem Use shared mem.
      */
-    protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
+    GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
         super(false);
 
         this.useShmem = useShmem;
@@ -370,17 +370,17 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
             Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
 
             for (GridNioSession ses : sessions) {
-                final GridNioRecoveryDescriptor snd = ses.recoveryDescriptor();
+                final GridNioRecoveryDescriptor snd = ses.outRecoveryDescriptor();
 
                 if (snd != null) {
                     GridTestUtils.waitForCondition(new GridAbsPredicate() {
                         @Override public boolean apply() {
-                            return snd.messagesFutures().isEmpty();
+                            return snd.messagesRequests().isEmpty();
                         }
                     }, 10_000);
 
-                    assertEquals("Unexpected messages: " + snd.messagesFutures(), 0,
-                        snd.messagesFutures().size());
+                    assertEquals("Unexpected messages: " + snd.messagesRequests(), 0,
+                        snd.messagesRequests().size());
                 }
             }
         }
@@ -547,11 +547,18 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         }
 
         for (CommunicationSpi spi : spis.values()) {
-            final ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+            final ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
 
             assert GridTestUtils.waitForCondition(new PA() {
                 @Override public boolean apply() {
-                    return clients.isEmpty();
+                    for (GridCommunicationClient[] clients0 : clients.values()) {
+                        for (GridCommunicationClient client : clients0) {
+                            if (client != null)
+                                return false;
+                        }
+                    }
+
+                    return true;
                 }
             }, getTestTimeout()) : "Clients: " + clients;
         }
@@ -583,4 +590,4 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         spis.clear();
         nodes.clear();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index 34872c6..12c2edb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -173,7 +173,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
                     boolean found = false;
 
                     for (GridNioSession ses : sessions) {
-                        final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+                        final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
                         if (recoveryDesc != null) {
                             found = true;
@@ -188,12 +188,12 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
 
                             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                                 @Override public boolean apply() {
-                                    return recoveryDesc.messagesFutures().isEmpty();
+                                    return recoveryDesc.messagesRequests().isEmpty();
                                 }
                             }, 10_000);
 
-                            assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
-                                recoveryDesc.messagesFutures().size());
+                            assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+                                recoveryDesc.messagesRequests().size());
 
                             break;
                         }
@@ -361,6 +361,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
         spi.setAckSendThreshold(ackCnt);
         spi.setMessageQueueLimit(queueLimit);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
index 95c9e40..b1aa119 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
@@ -33,6 +33,7 @@ public class GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest extends Gri
         spi.setAckSendThreshold(5);
         spi.setSocketSendBuffer(512);
         spi.setSocketReceiveBuffer(512);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..8e43937
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+/**
+ *
+ */
+public class GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest extends GridTcpCommunicationSpiRecoverySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean usePairedConnections() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 3234d74..065a3d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -56,6 +56,7 @@ import org.eclipse.jetty.util.ConcurrentHashSet;
 /**
  *
  */
+@SuppressWarnings("unchecked")
 @GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
 public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi> extends GridSpiAbstractTest<T> {
     /** */
@@ -305,14 +306,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 sentCnt.incrementAndGet();
@@ -331,7 +332,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     for (int j = 0; j < 100; j++) {
                         spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
@@ -415,14 +421,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 expCnt1.incrementAndGet();
@@ -441,12 +447,16 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     // Wait when session is closed, then try to open new connection from node1.
                     GridTestUtils.waitForCondition(new GridAbsPredicate() {
-                        @Override
-                        public boolean apply() {
+                        @Override public boolean apply() {
                             return ses1.closeTime() != 0;
                         }
                     }, awaitForSocketWriteTimeout());
@@ -532,14 +542,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 sentCnt.incrementAndGet();
@@ -558,7 +568,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     sndFut.get();
 
@@ -605,11 +620,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
     /**
      * @param spi SPI.
+     * @param in {@code True} if need find inbound session.
      * @return Session.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private GridNioSession communicationSession(TcpCommunicationSpi spi) throws Exception {
+    private GridNioSession communicationSession(TcpCommunicationSpi spi, boolean in) throws Exception {
         final GridNioServer srv = U.field(spi, "nioSrvr");
 
         GridTestUtils.waitForCondition(new GridAbsPredicate() {
@@ -622,9 +638,21 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
         Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
 
-        assertEquals(1, sessions.size());
+        for (GridNioSession ses : sessions) {
+            if (in == ses.accepted())
+                return ses;
+        }
+
+        fail("Failed to find session");
 
-        return sessions.iterator().next();
+        return null;
+    }
+
+    /**
+     * @return {@code True}.
+     */
+    protected boolean usePairedConnections() {
+        return true;
     }
 
     /**
@@ -642,6 +670,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
         spi.setSocketWriteTimeout(1000);
         spi.setSocketSendBuffer(512);
         spi.setSocketReceiveBuffer(512);
+        spi.setConnectionsPerNode(1);
+        spi.setUsePairedConnections(usePairedConnections());
 
         return spi;
     }
@@ -748,9 +778,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
             spi.spiStop();
         }
 
-        for (IgniteTestResources rsrcs : spiRsrcs) {
+        for (IgniteTestResources rsrcs : spiRsrcs)
             rsrcs.stopThreads();
-        }
 
         spis.clear();
         nodes.clear();

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
index 25e3611..c4930a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
@@ -187,7 +187,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
                     boolean found = false;
 
                     for (GridNioSession ses : sessions) {
-                        final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+                        final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
                         if (recoveryDesc != null) {
                             found = true;
@@ -202,12 +202,12 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
 
                             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                                 @Override public boolean apply() {
-                                    return recoveryDesc.messagesFutures().isEmpty();
+                                    return recoveryDesc.messagesRequests().isEmpty();
                                 }
                             }, 10_000);
 
-                            assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
-                                recoveryDesc.messagesFutures().size());
+                            assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+                                recoveryDesc.messagesRequests().size());
 
                             break;
                         }
@@ -397,6 +397,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
         spi.setAckSendThreshold(ackCnt);
         spi.setMessageQueueLimit(queueLimit);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index f9e2ff4..143159d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -63,7 +63,9 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
-                U.allPluginProviders());
+                null,
+                U.allPluginProviders()
+        );
 
         GridTestUtils.setFieldValue(grid(), "cfg", config());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 14e5833..17757ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAto
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWrityOrderOffHeapMultiNodeFullApiSelfTest;
@@ -77,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredFullApiSelfTest;
@@ -228,6 +230,10 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);
         suite.addTestSuite(GridCachePartitionedFullApiMultithreadedSelfTest.class);
 
+        // Disabled striped pool.
+        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.class);
+
         // Other.
         suite.addTestSuite(GridCacheClearSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 435fcfb..554bb3d 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -39,6 +39,10 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerW
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
+import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
+import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;
@@ -128,7 +132,10 @@ import org.apache.ignite.internal.processors.cache.context.IgniteCacheTxExecutio
 import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicNearUpdateTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheTxNearUpdateTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEntrySetIterationPreloadingSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecovery10ConnectionsTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecovery10ConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageRecoveryIdleConnectionTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageWriteTimeoutTest;
@@ -259,7 +266,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
-            suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+        suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
         suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
         suite.addTestSuite(DataStreamerImplSelfTest.class);
         suite.addTestSuite(DataStreamerTimeoutTest.class);
@@ -292,10 +299,13 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class);
         suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class);
         suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class);
+        suite.addTestSuite(IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.class);
+        suite.addTestSuite(IgniteCacheAtomicMessageRecovery10ConnectionsTest.class);
         suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class);
         suite.addTestSuite(IgniteCacheMessageWriteTimeoutTest.class);
         suite.addTestSuite(IgniteCacheMessageRecoveryIdleConnectionTest.class);
         suite.addTestSuite(IgniteCacheConnectionRecoveryTest.class);
+        suite.addTestSuite(IgniteCacheConnectionRecovery10ConnectionsTest.class);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionAtomicSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredAtomicSelfTest.class, ignoredTests);
@@ -325,6 +335,11 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheTxFastFinishTest.class);
 
+        suite.addTestSuite(IgniteVariousConnectionNumberTest.class);
+        suite.addTestSuite(IgniteCommunicationBalanceTest.class);
+        suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class);
+        suite.addTestSuite(IgniteIoTestMessagesTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
index c557fbb..11fcfda 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithrea
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithreadedShmemTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryAckSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest;
+import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest;
@@ -50,6 +51,7 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryAckSelfTest.class));
         suite.addTest(new TestSuite(IgniteTcpCommunicationRecoveryAckClosureSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySslSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index a725534..9542372 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -240,7 +240,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                     try {
                         // Must process query callback in a separate thread to avoid deadlocks.
-                        evtProcSvc.submit(new EventHandler() {
+                        evtProcSvc.execute(new EventHandler() {
                             @Override protected void body() throws IgniteCheckedException {
                                 processJobMetadataUpdates(evts);
                             }
@@ -264,7 +264,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                 try {
                     // Must process discovery callback in a separate thread to avoid deadlock.
-                    evtProcSvc.submit(new EventHandler() {
+                    evtProcSvc.execute(new EventHandler() {
                         @Override protected void body() {
                             processNodeLeft((DiscoveryEvent)evt);
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index bc047e7..ff58509 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -639,6 +639,7 @@ public class HadoopExternalCommunication {
                         .logger(log.getLogger(GridNioServer.class))
                         .selectorCount(selectorsCnt)
                         .gridName(gridName)
+                        .serverName("hadoop")
                         .tcpNoDelay(tcpNoDelay)
                         .directBuffer(directBuf)
                         .byteOrder(ByteOrder.nativeOrder())
@@ -1305,11 +1306,11 @@ public class HadoopExternalCommunication {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
                 log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
 
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
index a8de999..3f33fb7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
@@ -190,7 +190,7 @@ public class HadoopIpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
             assert ses == HadoopIpcToNioAdapter.this.ses : "ses=" + ses +
                 ", this.ses=" + HadoopIpcToNioAdapter.this.ses;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
index eeca564..24bba88 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
@@ -57,10 +57,10 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
         assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
 
-        return proceedSessionWrite(ses, U.marshal(marsh, msg));
+        return proceedSessionWrite(ses, U.marshal(marsh, msg), fut);
     }
 
     @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
@@ -84,4 +84,4 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
----------------------------------------------------------------------
diff --git a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
index f46ee93..221538c 100644
--- a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
+++ b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
@@ -169,7 +169,7 @@ public class KafkaStreamer<T, K, V> extends StreamAdapter<T, K, V> {
 
         // Now create an object to consume the messages.
         for (final KafkaStream<K, V> stream : streams) {
-            executor.submit(new Runnable() {
+            executor.execute(new Runnable() {
                 @Override public void run() {
                     while (!stopped) {
                         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
index 0b10a75..369e318 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
@@ -122,7 +122,11 @@ public class ClassesGenerator {
             for (String err : errs)
                 sb.append("    ").append(err).append('\n');
 
-            throw new Exception(sb.toString().trim());
+            String msg = sb.toString().trim();
+
+            System.out.println(msg);
+
+            throw new Exception(msg);
         }
 
         PrintStream out = new PrintStream(new File(basePath,
@@ -246,4 +250,4 @@ public class ClassesGenerator {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
index 54ef00d..07549d5 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
@@ -30,7 +30,7 @@ import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionOptimisticException;
 import org.apache.ignite.transactions.TransactionRollbackException;
-import org.apache.ignite.yardstick.cache.IgniteSqlQueryBenchmark;
+import org.apache.ignite.yardstick.cache.IgnitePutBenchmark;
 import org.yardstickframework.BenchmarkDriver;
 import org.yardstickframework.BenchmarkDriverStartUp;
 
@@ -89,13 +89,13 @@ public class IgniteBenchmarkUtils {
     public static void main(String[] args) throws Exception {
         final String cfg = "modules/yardstick/config/ignite-localhost-config.xml";
 
-        final Class<? extends BenchmarkDriver> benchmark = IgniteSqlQueryBenchmark.class;
+        final Class<? extends BenchmarkDriver> benchmark = IgnitePutBenchmark.class;
 
         final int threads = 1;
 
         final boolean clientDriverNode = true;
 
-        final int extraNodes = 2;
+        final int extraNodes = 4;
 
         final int warmUp = 5;
         final int duration = 5;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
index a25f975..c8022e2 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
@@ -72,7 +72,7 @@ public class CacheEntryEventProbe implements BenchmarkProbe {
 
                 buildingService = Executors.newSingleThreadExecutor();
 
-                buildingService.submit(new Runnable() {
+                buildingService.execute(new Runnable() {
                     @Override public void run() {
                         try {
                             while (!Thread.currentThread().isInterrupted()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
new file mode 100644
index 0000000..bee45e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestBenchmark extends IgniteAbstractBenchmark {
+    /** */
+    private List<ClusterNode> targetNodes;
+
+    /** */
+    private IgniteKernal ignite;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        ignite = (IgniteKernal)ignite();
+
+        targetNodes = new ArrayList<>();
+
+        ClusterNode loc = ignite().cluster().localNode();
+
+        Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+        for (ClusterNode node : nodes) {
+            if (!loc.equals(node))
+                targetNodes.add(node);
+        }
+
+        if (targetNodes.isEmpty())
+            throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+        BenchmarkUtils.println(cfg, "Initialized target nodes: " + targetNodes + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+        ignite.sendIoTest(node, null, false).get();
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
new file mode 100644
index 0000000..8791c83
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
@@ -0,0 +1,61 @@
+/*
+ * 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.yardstick.io;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ *
+ */
+public abstract class IgniteIoTestAbstractBenchmark extends IgniteAbstractBenchmark {
+    /** */
+    protected final List<ClusterNode> targetNodes = new ArrayList<>();
+
+    /** */
+    protected IgniteKernal ignite;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        ignite = (IgniteKernal)ignite();
+
+        ClusterNode loc = ignite().cluster().localNode();
+
+        Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+        for (ClusterNode node : nodes) {
+            if (!loc.equals(node))
+                targetNodes.add(node);
+        }
+
+        if (targetNodes.isEmpty())
+            throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+        BenchmarkUtils.println(cfg, "Initialized target nodes: " + F.nodeIds(targetNodes) + ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
new file mode 100644
index 0000000..9011910
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestSendAllBenchmark extends IgniteIoTestAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ignite.sendIoTest(targetNodes, null, false).get();
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
new file mode 100644
index 0000000..88368e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
@@ -0,0 +1,35 @@
+/*
+ * 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.yardstick.io;
+
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ *
+ */
+public class IgniteIoTestSendRandomBenchmark extends IgniteIoTestAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+        ignite.sendIoTest(node, null, false).get();
+
+        return true;
+    }
+}


[42/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 5887832..8770bf6 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/ConfigurationGenerator.js
@@ -19,1776 +19,1825 @@ import DFLT_DIALECTS from 'app/data/dialects.json';
 
 import { EmptyBean, Bean } from './Beans';
 
-export default ['JavaTypes', 'igniteClusterDefaults', 'igniteCacheDefaults', 'igniteIgfsDefaults', (JavaTypes, clusterDflts, cacheDflts, igfsDflts) => {
-    class ConfigurationGenerator {
-        static igniteConfigurationBean(cluster) {
-            return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts);
-        }
+import IgniteClusterDefaults from './defaults/Cluster.service';
+import IgniteCacheDefaults from './defaults/Cache.service';
+import IgniteIGFSDefaults from './defaults/IGFS.service';
 
-        static igfsConfigurationBean(igfs) {
-            return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts);
-        }
+import JavaTypes from '../../../services/JavaTypes.service';
 
-        static cacheConfigurationBean(cache) {
-            return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts);
-        }
+const clusterDflts = new IgniteClusterDefaults();
+const cacheDflts = new IgniteCacheDefaults();
+const igfsDflts = new IgniteIGFSDefaults();
 
-        static domainConfigurationBean(domain) {
-            return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts);
-        }
+const javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
 
-        static discoveryConfigurationBean(discovery) {
-            return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery);
-        }
+export default class IgniteConfigurationGenerator {
+    static igniteConfigurationBean(cluster) {
+        return new Bean('org.apache.ignite.configuration.IgniteConfiguration', 'cfg', cluster, clusterDflts);
+    }
 
-        /**
-         * Function to generate ignite configuration.
-         *
-         * @param {Object} cluster Cluster to process.
-         * @param {Boolean} client
-         * @return {Bean} Generated ignite configuration.
-         */
-        static igniteConfiguration(cluster, client) {
-            const cfg = this.igniteConfigurationBean(cluster);
-
-            this.clusterGeneral(cluster, cfg, client);
-            this.clusterAtomics(cluster.atomicConfiguration, cfg);
-            this.clusterBinary(cluster.binaryConfiguration, cfg);
-            this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg);
-            this.clusterCheckpoint(cluster, cluster.caches, cfg);
-            this.clusterCollision(cluster.collision, cfg);
-            this.clusterCommunication(cluster, cfg);
-            this.clusterConnector(cluster.connector, cfg);
-            this.clusterDeployment(cluster, cfg);
-            this.clusterEvents(cluster, cfg);
-            this.clusterFailover(cluster, cfg);
-            this.clusterLoadBalancing(cluster, cfg);
-            this.clusterLogger(cluster.logger, cfg);
-            this.clusterODBC(cluster.odbc, cfg);
-            this.clusterMarshaller(cluster, cfg);
-            this.clusterMetrics(cluster, cfg);
-            this.clusterSwap(cluster, cfg);
-            this.clusterTime(cluster, cfg);
-            this.clusterPools(cluster, cfg);
-            this.clusterTransactions(cluster.transactionConfiguration, cfg);
-            this.clusterSsl(cluster, cfg);
-            this.clusterUserAttributes(cluster, cfg);
-
-            this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg);
-
-            if (!client)
-                this.clusterIgfss(cluster.igfss, cfg);
+    static igfsConfigurationBean(igfs) {
+        return new Bean('org.apache.ignite.configuration.FileSystemConfiguration', 'igfs', igfs, igfsDflts);
+    }
 
-            return cfg;
-        }
+    static cacheConfigurationBean(cache) {
+        return new Bean('org.apache.ignite.configuration.CacheConfiguration', 'ccfg', cache, cacheDflts);
+    }
 
-        static dialectClsName(dialect) {
-            return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect');
-        }
+    static domainConfigurationBean(domain) {
+        return new Bean('org.apache.ignite.cache.QueryEntity', 'qryEntity', domain, cacheDflts);
+    }
 
-        static dataSourceBean(id, dialect) {
-            let dsBean;
+    static discoveryConfigurationBean(discovery) {
+        return new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery', discovery, clusterDflts.discovery);
+    }
 
-            switch (dialect) {
-                case 'Generic':
-                    dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {})
-                        .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database');
+    /**
+     * Function to generate ignite configuration.
+     *
+     * @param {Object} cluster Cluster to process.
+     * @param {Boolean} client
+     * @return {Bean} Generated ignite configuration.
+     */
+    static igniteConfiguration(cluster, client) {
+        const cfg = this.igniteConfigurationBean(cluster);
+
+        this.clusterGeneral(cluster, cfg, client);
+        this.clusterAtomics(cluster.atomicConfiguration, cfg);
+        this.clusterBinary(cluster.binaryConfiguration, cfg);
+        this.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, cfg);
+        this.clusterCheckpoint(cluster, cluster.caches, cfg);
+        this.clusterCollision(cluster.collision, cfg);
+        this.clusterCommunication(cluster, cfg);
+        this.clusterConnector(cluster.connector, cfg);
+        this.clusterDeployment(cluster, cfg);
+        this.clusterEvents(cluster, cfg);
+        this.clusterFailover(cluster, cfg);
+        this.clusterLoadBalancing(cluster, cfg);
+        this.clusterLogger(cluster.logger, cfg);
+        this.clusterODBC(cluster.odbc, cfg);
+        this.clusterMarshaller(cluster, cfg);
+        this.clusterMetrics(cluster, cfg);
+        this.clusterSwap(cluster, cfg);
+        this.clusterTime(cluster, cfg);
+        this.clusterPools(cluster, cfg);
+        this.clusterTransactions(cluster.transactionConfiguration, cfg);
+        this.clusterSsl(cluster, cfg);
+        this.clusterUserAttributes(cluster, cfg);
+
+        this.clusterCaches(cluster, cluster.caches, cluster.igfss, client, cfg);
+
+        if (!client)
+            this.clusterIgfss(cluster.igfss, cfg);
+
+        return cfg;
+    }
 
-                    break;
-                case 'Oracle':
-                    dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]');
+    static dialectClsName(dialect) {
+        return DFLT_DIALECTS[dialect] || 'Unknown database: ' + (dialect || 'Choose JDBC dialect');
+    }
 
-                    break;
-                case 'DB2':
-                    dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {})
-                        .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_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');
+    static dataSourceBean(id, dialect) {
+        let dsBean;
+
+        switch (dialect) {
+            case 'Generic':
+                dsBean = new Bean('com.mchange.v2.c3p0.ComboPooledDataSource', id, {})
+                    .property('jdbcUrl', `${id}.jdbc.url`, 'jdbc:your_database');
+
+                break;
+            case 'Oracle':
+                dsBean = new Bean('oracle.jdbc.pool.OracleDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:oracle:thin:@[host]:[port]:[database]');
+
+                break;
+            case 'DB2':
+                dsBean = new Bean('com.ibm.db2.jcc.DB2DataSource', id, {})
+                    .property('serverName', `${id}.jdbc.server_name`, 'YOUR_DATABASE_SERVER_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':
+                dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]');
+
+                break;
+            case 'MySQL':
+                dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]');
+
+                break;
+            case 'PostgreSQL':
+                dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {})
+                    .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]');
+
+                break;
+            case 'H2':
+                dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {})
+                    .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]');
+
+                break;
+            default:
+        }
 
-                    break;
-                case 'SQLServer':
-                    dsBean = new Bean('com.microsoft.sqlserver.jdbc.SQLServerDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:sqlserver://[host]:[port][;databaseName=database]');
+        if (dsBean) {
+            dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME')
+                .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD');
+        }
 
-                    break;
-                case 'MySQL':
-                    dsBean = new Bean('com.mysql.jdbc.jdbc2.optional.MysqlDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:mysql://[host]:[port]/[database]');
+        return dsBean;
+    }
 
-                    break;
-                case 'PostgreSQL':
-                    dsBean = new Bean('org.postgresql.ds.PGPoolingDataSource', id, {})
-                        .property('url', `${id}.jdbc.url`, 'jdbc:postgresql://[host]:[port]/[database]');
+    // Generate general section.
+    static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) {
+        if (client)
+            cfg.prop('boolean', 'clientMode', true);
 
-                    break;
-                case 'H2':
-                    dsBean = new Bean('org.h2.jdbcx.JdbcDataSource', id, {})
-                        .property('URL', `${id}.jdbc.url`, 'jdbc:h2:tcp://[host]/[database]');
+        cfg.stringProperty('name', 'gridName')
+            .stringProperty('localHost');
 
-                    break;
-                default:
-            }
-
-            if (dsBean) {
-                dsBean.property('user', `${id}.jdbc.username`, 'YOUR_USER_NAME')
-                    .property('password', `${id}.jdbc.password`, 'YOUR_PASSWORD');
-            }
+        if (_.isNil(cluster.discovery))
+            return cfg;
 
-            return dsBean;
-        }
+        const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery',
+            cluster.discovery, clusterDflts.discovery);
 
-        // Generate general section.
-        static clusterGeneral(cluster, cfg = this.igniteConfigurationBean(cluster), client = false) {
-            if (client)
-                cfg.prop('boolean', 'clientMode', true);
+        let ipFinder;
 
-            cfg.stringProperty('name', 'gridName')
-                .stringProperty('localHost');
+        switch (discovery.valueOf('kind')) {
+            case 'Vm':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
+                    'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm);
 
-            if (_.isNil(cluster.discovery))
-                return cfg;
+                ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses);
 
-            const discovery = new Bean('org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi', 'discovery',
-                cluster.discovery, clusterDflts.discovery);
+                break;
+            case 'Multicast':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
+                    'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast);
 
-            let ipFinder;
+                ipFinder.stringProperty('multicastGroup')
+                    .intProperty('multicastPort')
+                    .intProperty('responseWaitTime')
+                    .intProperty('addressRequestAttempts')
+                    .stringProperty('localAddress')
+                    .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses);
 
-            switch (discovery.valueOf('kind')) {
-                case 'Vm':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder',
-                        'ipFinder', cluster.discovery.Vm, clusterDflts.discovery.Vm);
+                break;
+            case 'S3':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder',
+                    'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3);
 
-                    ipFinder.collectionProperty('addrs', 'addresses', cluster.discovery.Vm.addresses);
+                ipFinder.stringProperty('bucketName');
 
-                    break;
-                case 'Multicast':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder',
-                        'ipFinder', cluster.discovery.Multicast, clusterDflts.discovery.Multicast);
+                break;
+            case 'Cloud':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
+                    'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud);
 
-                    ipFinder.stringProperty('multicastGroup')
-                        .intProperty('multicastPort')
-                        .intProperty('responseWaitTime')
-                        .intProperty('addressRequestAttempts')
-                        .stringProperty('localAddress')
-                        .collectionProperty('addrs', 'addresses', cluster.discovery.Multicast.addresses);
+                ipFinder.stringProperty('credential')
+                    .pathProperty('credentialPath')
+                    .stringProperty('identity')
+                    .stringProperty('provider')
+                    .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions)
+                    .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones);
 
-                    break;
-                case 'S3':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder',
-                        'ipFinder', cluster.discovery.S3, clusterDflts.discovery.S3);
+                break;
+            case 'GoogleStorage':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
+                    'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage);
 
-                    ipFinder.stringProperty('bucketName');
+                ipFinder.stringProperty('projectName')
+                    .stringProperty('bucketName')
+                    .pathProperty('serviceAccountP12FilePath')
+                    .stringProperty('serviceAccountId');
 
-                    break;
-                case 'Cloud':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder',
-                        'ipFinder', cluster.discovery.Cloud, clusterDflts.discovery.Cloud);
+                break;
+            case 'Jdbc':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder',
+                    'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc);
 
-                    ipFinder.stringProperty('credential')
-                        .pathProperty('credentialPath')
-                        .stringProperty('identity')
-                        .stringProperty('provider')
-                        .collectionProperty('regions', 'regions', cluster.discovery.Cloud.regions)
-                        .collectionProperty('zones', 'zones', cluster.discovery.Cloud.zones);
+                ipFinder.intProperty('initSchema');
 
-                    break;
-                case 'GoogleStorage':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder',
-                        'ipFinder', cluster.discovery.GoogleStorage, clusterDflts.discovery.GoogleStorage);
+                if (ipFinder.includes('dataSourceBean', 'dialect')) {
+                    const id = ipFinder.valueOf('dataSourceBean');
 
-                    ipFinder.stringProperty('projectName')
-                        .stringProperty('bucketName')
-                        .pathProperty('serviceAccountP12FilePath')
-                        .stringProperty('serviceAccountId');
+                    ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect')));
+                }
 
-                    break;
-                case 'Jdbc':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder',
-                        'ipFinder', cluster.discovery.Jdbc, clusterDflts.discovery.Jdbc);
+                break;
+            case 'SharedFs':
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder',
+                    'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs);
+
+                ipFinder.pathProperty('path');
+
+                break;
+            case 'ZooKeeper':
+                const src = cluster.discovery.ZooKeeper;
+                const dflt = clusterDflts.discovery.ZooKeeper;
+
+                ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder',
+                    'ipFinder', src, dflt);
+
+                ipFinder.emptyBeanProperty('curator')
+                    .stringProperty('zkConnectionString');
+
+                const kind = _.get(src, 'retryPolicy.kind');
+
+                if (kind) {
+                    const policy = src.retryPolicy;
+
+                    let retryPolicyBean;
+
+                    switch (kind) {
+                        case 'ExponentialBackoff':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null,
+                                policy.ExponentialBackoff, dflt.ExponentialBackoff)
+                                .intConstructorArgument('baseSleepTimeMs')
+                                .intConstructorArgument('maxRetries')
+                                .intConstructorArgument('maxSleepMs');
+
+                            break;
+                        case 'BoundedExponentialBackoff':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry',
+                                null, policy.BoundedExponentialBackoff, dflt.BoundedExponentialBackoffRetry)
+                                .intConstructorArgument('baseSleepTimeMs')
+                                .intConstructorArgument('maxSleepTimeMs')
+                                .intConstructorArgument('maxRetries');
+
+                            break;
+                        case 'UntilElapsed':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null,
+                                policy.UntilElapsed, dflt.UntilElapsed)
+                                .intConstructorArgument('maxElapsedTimeMs')
+                                .intConstructorArgument('sleepMsBetweenRetries');
+
+                            break;
+
+                        case 'NTimes':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null,
+                                policy.NTimes, dflt.NTimes)
+                                .intConstructorArgument('n')
+                                .intConstructorArgument('sleepMsBetweenRetries');
+
+                            break;
+                        case 'OneTime':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null,
+                                policy.OneTime, dflt.OneTime)
+                                .intConstructorArgument('sleepMsBetweenRetry');
+
+                            break;
+                        case 'Forever':
+                            retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null,
+                                policy.Forever, dflt.Forever)
+                                .intConstructorArgument('retryIntervalMs');
+
+                            break;
+                        case 'Custom':
+                            const className = _.get(policy, 'Custom.className');
+
+                            if (_.nonEmpty(className))
+                                retryPolicyBean = new EmptyBean(className);
+
+                            break;
+                        default:
+                            // No-op.
+                    }
 
-                    ipFinder.intProperty('initSchema');
+                    if (retryPolicyBean)
+                        ipFinder.beanProperty('retryPolicy', retryPolicyBean);
+                }
 
-                    if (ipFinder.includes('dataSourceBean', 'dialect')) {
-                        const id = ipFinder.valueOf('dataSourceBean');
+                ipFinder.pathProperty('basePath', '/services')
+                    .stringProperty('serviceName')
+                    .boolProperty('allowDuplicateRegistrations');
 
-                        ipFinder.dataSource(id, 'dataSource', this.dataSourceBean(id, ipFinder.valueOf('dialect')));
-                    }
+                break;
+            default:
+                // No-op.
+        }
 
-                    break;
-                case 'SharedFs':
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder',
-                        'ipFinder', cluster.discovery.SharedFs, clusterDflts.discovery.SharedFs);
+        if (ipFinder)
+            discovery.beanProperty('ipFinder', ipFinder);
 
-                    ipFinder.pathProperty('path');
+        this.clusterDiscovery(cluster.discovery, cfg, discovery);
 
-                    break;
-                case 'ZooKeeper':
-                    const src = cluster.discovery.ZooKeeper;
-                    const dflt = clusterDflts.discovery.ZooKeeper;
+        return cfg;
+    }
 
-                    ipFinder = new Bean('org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder',
-                        'ipFinder', src, dflt);
+    static igfsDataCache(igfs) {
+        return this.cacheConfiguration({
+            name: igfs.name + '-data',
+            cacheMode: 'PARTITIONED',
+            atomicityMode: 'TRANSACTIONAL',
+            writeSynchronizationMode: 'FULL_SYNC',
+            backups: 0,
+            igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
+        });
+    }
 
-                    ipFinder.emptyBeanProperty('curator')
-                        .stringProperty('zkConnectionString');
+    static igfsMetaCache(igfs) {
+        return this.cacheConfiguration({
+            name: igfs.name + '-meta',
+            cacheMode: 'REPLICATED',
+            atomicityMode: 'TRANSACTIONAL',
+            writeSynchronizationMode: 'FULL_SYNC'
+        });
+    }
 
-                    const kind = _.get(src, 'retryPolicy.kind');
+    static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) {
+        const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache));
 
-                    if (kind) {
-                        const policy = src.retryPolicy;
+        if (!client) {
+            _.forEach(igfss, (igfs) => {
+                ccfgs.push(this.igfsDataCache(igfs));
+                ccfgs.push(this.igfsMetaCache(igfs));
+            });
+        }
 
-                        let retryPolicyBean;
+        cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration');
 
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.ExponentialBackoffRetry', null,
-                                    policy.ExponentialBackoff, dflt.ExponentialBackoff)
-                                    .intConstructorArgument('baseSleepTimeMs')
-                                    .intConstructorArgument('maxRetries')
-                                    .intConstructorArgument('maxSleepMs');
+        return cfg;
+    }
 
-                                break;
-                            case 'BoundedExponentialBackoff':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.BoundedExponentialBackoffRetry',
-                                    null, policy.BoundedExponentialBackoffRetry, dflt.BoundedExponentialBackoffRetry)
-                                    .intConstructorArgument('baseSleepTimeMs')
-                                    .intConstructorArgument('maxSleepTimeMs')
-                                    .intConstructorArgument('maxRetries');
+    // Generate atomics group.
+    static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) {
+        const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg',
+            atomics, clusterDflts.atomics);
 
-                                break;
-                            case 'UntilElapsed':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryUntilElapsed', null,
-                                    policy.UntilElapsed, dflt.UntilElapsed)
-                                    .intConstructorArgument('maxElapsedTimeMs')
-                                    .intConstructorArgument('sleepMsBetweenRetries');
+        acfg.enumProperty('cacheMode')
+            .intProperty('atomicSequenceReserveSize');
 
-                                break;
+        if (acfg.valueOf('cacheMode') === 'PARTITIONED')
+            acfg.intProperty('backups');
 
-                            case 'NTimes':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryNTimes', null,
-                                    policy.NTimes, dflt.NTimes)
-                                    .intConstructorArgument('n')
-                                    .intConstructorArgument('sleepMsBetweenRetries');
+        if (acfg.isEmpty())
+            return cfg;
 
-                                break;
-                            case 'OneTime':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryOneTime', null,
-                                    policy.OneTime, dflt.OneTime)
-                                    .intConstructorArgument('sleepMsBetweenRetry');
+        cfg.beanProperty('atomicConfiguration', acfg);
 
-                                break;
-                            case 'Forever':
-                                retryPolicyBean = new Bean('org.apache.curator.retry.RetryForever', null,
-                                    policy.Forever, dflt.Forever)
-                                    .intConstructorArgument('retryIntervalMs');
+        return cfg;
+    }
 
-                                break;
-                            case 'Custom':
-                                if (_.nonEmpty(policy.Custom.className))
-                                    retryPolicyBean = new EmptyBean(policy.Custom.className);
+    // Generate binary group.
+    static clusterBinary(binary, cfg = this.igniteConfigurationBean()) {
+        const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg',
+            binary, clusterDflts.binary);
 
-                                break;
-                            default:
-                                // No-op.
-                        }
+        binaryCfg.emptyBeanProperty('idMapper')
+            .emptyBeanProperty('nameMapper')
+            .emptyBeanProperty('serializer');
 
-                        if (retryPolicyBean)
-                            ipFinder.beanProperty('retryPolicy', retryPolicyBean);
-                    }
+        const typeCfgs = [];
 
-                    ipFinder.pathProperty('basePath', '/services')
-                        .stringProperty('serviceName')
-                        .boolProperty('allowDuplicateRegistrations');
+        _.forEach(binary.typeConfigurations, (type) => {
+            const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration',
+                javaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations);
 
-                    break;
-                default:
-                    // No-op.
-            }
+            typeCfg.stringProperty('typeName')
+                .emptyBeanProperty('idMapper')
+                .emptyBeanProperty('nameMapper')
+                .emptyBeanProperty('serializer')
+                .intProperty('enum');
 
-            if (ipFinder)
-                discovery.beanProperty('ipFinder', ipFinder);
+            if (typeCfg.nonEmpty())
+                typeCfgs.push(typeCfg);
+        });
 
-            this.clusterDiscovery(cluster.discovery, cfg, discovery);
+        binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration')
+            .boolProperty('compactFooter');
 
+        if (binaryCfg.isEmpty())
             return cfg;
-        }
-
-        static igfsDataCache(igfs) {
-            return this.cacheConfiguration({
-                name: igfs.name + '-data',
-                cacheMode: 'PARTITIONED',
-                atomicityMode: 'TRANSACTIONAL',
-                writeSynchronizationMode: 'FULL_SYNC',
-                backups: 0,
-                igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
-            });
-        }
 
-        static igfsMetaCache(igfs) {
-            return this.cacheConfiguration({
-                name: igfs.name + '-meta',
-                cacheMode: 'REPLICATED',
-                atomicityMode: 'TRANSACTIONAL',
-                writeSynchronizationMode: 'FULL_SYNC'
-            });
-        }
+        cfg.beanProperty('binaryConfiguration', binaryCfg);
 
-        static clusterCaches(cluster, caches, igfss, client, cfg = this.igniteConfigurationBean(cluster)) {
-            const ccfgs = _.map(caches, (cache) => this.cacheConfiguration(cache));
+        return cfg;
+    }
 
-            if (!client) {
-                _.forEach(igfss, (igfs) => {
-                    ccfgs.push(this.igfsDataCache(igfs));
-                    ccfgs.push(this.igfsMetaCache(igfs));
-                });
+    // Generate cache key configurations.
+    static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) {
+        const items = _.reduce(keyCfgs, (acc, keyCfg) => {
+            if (keyCfg.typeName && keyCfg.affinityKeyFieldName) {
+                acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg)
+                    .stringConstructorArgument('typeName')
+                    .stringConstructorArgument('affinityKeyFieldName'));
             }
 
-            cfg.varArgProperty('ccfgs', 'cacheConfiguration', ccfgs, 'org.apache.ignite.configuration.CacheConfiguration');
+            return acc;
+        }, []);
 
+        if (_.isEmpty(items))
             return cfg;
-        }
 
-        // Generate atomics group.
-        static clusterAtomics(atomics, cfg = this.igniteConfigurationBean()) {
-            const acfg = new Bean('org.apache.ignite.configuration.AtomicConfiguration', 'atomicCfg',
-                atomics, clusterDflts.atomics);
+        cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items,
+            'org.apache.ignite.cache.CacheKeyConfiguration');
 
-            acfg.enumProperty('cacheMode')
-                .intProperty('atomicSequenceReserveSize');
+        return cfg;
+    }
 
-            if (acfg.valueOf('cacheMode') === 'PARTITIONED')
-                acfg.intProperty('backups');
+    // Generate checkpoint configurations.
+    static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) {
+        const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => {
+            switch (_.get(spi, 'kind')) {
+                case 'FS':
+                    const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi',
+                        'checkpointSpiFs', spi.FS);
 
-            if (acfg.isEmpty())
-                return cfg;
+                    fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths'))
+                        .emptyBeanProperty('checkpointListener');
 
-            cfg.beanProperty('atomicConfiguration', acfg);
+                    return fsBean;
 
-            return cfg;
-        }
+                case 'Cache':
+                    const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi',
+                        'checkpointSpiCache', spi.Cache);
 
-        // Generate binary group.
-        static clusterBinary(binary, cfg = this.igniteConfigurationBean()) {
-            const binaryCfg = new Bean('org.apache.ignite.configuration.BinaryConfiguration', 'binaryCfg',
-                binary, clusterDflts.binary);
+                    const curCache = _.get(spi, 'Cache.cache');
 
-            binaryCfg.emptyBeanProperty('idMapper')
-                .emptyBeanProperty('nameMapper')
-                .emptyBeanProperty('serializer');
+                    const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache));
 
-            const typeCfgs = [];
+                    if (cache)
+                        cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name);
 
-            _.forEach(binary.typeConfigurations, (type) => {
-                const typeCfg = new Bean('org.apache.ignite.binary.BinaryTypeConfiguration',
-                    JavaTypes.toJavaName('binaryType', type.typeName), type, clusterDflts.binary.typeConfigurations);
+                    cacheBean.stringProperty('cacheName')
+                        .emptyBeanProperty('checkpointListener');
 
-                typeCfg.stringProperty('typeName')
-                    .emptyBeanProperty('idMapper')
-                    .emptyBeanProperty('nameMapper')
-                    .emptyBeanProperty('serializer')
-                    .intProperty('enum');
+                    return cacheBean;
 
-                if (typeCfg.nonEmpty())
-                    typeCfgs.push(typeCfg);
-            });
+                case 'S3':
+                    const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi',
+                        'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3);
 
-            binaryCfg.collectionProperty('types', 'typeConfigurations', typeCfgs, 'org.apache.ignite.binary.BinaryTypeConfiguration')
-                .boolProperty('compactFooter');
+                    let credentialsBean = null;
 
-            if (binaryCfg.isEmpty())
-                return cfg;
+                    switch (_.get(spi.S3, 'awsCredentials.kind')) {
+                        case 'Basic':
+                            credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {});
 
-            cfg.beanProperty('binaryConfiguration', binaryCfg);
+                            credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY');
 
-            return cfg;
-        }
+                            break;
 
-        // Generate cache key configurations.
-        static clusterCacheKeyConfiguration(keyCfgs, cfg = this.igniteConfigurationBean()) {
-            const items = _.reduce(keyCfgs, (acc, keyCfg) => {
-                if (keyCfg.typeName && keyCfg.affinityKeyFieldName) {
-                    acc.push(new Bean('org.apache.ignite.cache.CacheKeyConfiguration', null, keyCfg)
-                        .stringConstructorArgument('typeName')
-                        .stringConstructorArgument('affinityKeyFieldName'));
-                }
+                        case 'Properties':
+                            credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {});
 
-                return acc;
-            }, []);
+                            const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties)
+                                .pathConstructorArgument('path');
 
-            if (_.isEmpty(items))
-                return cfg;
+                            if (fileBean.nonEmpty())
+                                credentialsBean.beanConstructorArgument('file', fileBean);
 
-            cfg.arrayProperty('cacheKeyConfiguration', 'cacheKeyConfiguration', items,
-                'org.apache.ignite.cache.CacheKeyConfiguration');
+                            break;
 
-            return cfg;
-        }
+                        case 'Anonymous':
+                            credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {});
 
-        // Generate checkpoint configurations.
-        static clusterCheckpoint(cluster, caches, cfg = this.igniteConfigurationBean()) {
-            const cfgs = _.filter(_.map(cluster.checkpointSpi, (spi) => {
-                switch (_.get(spi, 'kind')) {
-                    case 'FS':
-                        const fsBean = new Bean('org.apache.ignite.spi.checkpoint.sharedfs.SharedFsCheckpointSpi',
-                            'checkpointSpiFs', spi.FS);
+                            break;
 
-                        fsBean.collectionProperty('directoryPaths', 'directoryPaths', _.get(spi, 'FS.directoryPaths'))
-                            .emptyBeanProperty('checkpointListener');
+                        case 'BasicSession':
+                            credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {});
 
-                        return fsBean;
+                            // TODO 2054 Arguments in one line is very long string.
+                            credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.secretKey')
+                                .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken');
 
-                    case 'Cache':
-                        const cacheBean = new Bean('org.apache.ignite.spi.checkpoint.cache.CacheCheckpointSpi',
-                            'checkpointSpiCache', spi.Cache);
+                            break;
 
-                        const curCache = _.get(spi, 'Cache.cache');
+                        case 'Custom':
+                            const className = _.get(spi.S3.awsCredentials, 'Custom.className');
 
-                        const cache = _.find(caches, (c) => curCache && (c._id === curCache || _.get(c, 'cache._id') === curCache));
+                            if (className)
+                                credentialsBean = new Bean(className, 'awsCredentials', {});
 
-                        if (cache)
-                            cacheBean.prop('java.lang.String', 'cacheName', cache.name || cache.cache.name);
+                            break;
 
-                        cacheBean.stringProperty('cacheName')
-                            .emptyBeanProperty('checkpointListener');
+                        default:
+                            break;
+                    }
 
-                        return cacheBean;
+                    if (credentialsBean)
+                        s3Bean.beanProperty('awsCredentials', credentialsBean);
 
-                    case 'S3':
-                        const s3Bean = new Bean('org.apache.ignite.spi.checkpoint.s3.S3CheckpointSpi',
-                            'checkpointSpiS3', spi.S3, clusterDflts.checkpointSpi.S3);
+                    s3Bean.stringProperty('bucketNameSuffix');
 
-                        let credentialsBean = null;
+                    const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration,
+                        clusterDflts.checkpointSpi.S3.clientConfiguration);
 
-                        switch (_.get(spi.S3, 'awsCredentials.kind')) {
-                            case 'Basic':
-                                credentialsBean = new Bean('com.amazonaws.auth.BasicAWSCredentials', 'awsCredentials', {});
+                    clientBean.enumProperty('protocol')
+                        .intProperty('maxConnections')
+                        .stringProperty('userAgent');
 
-                                credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey', 'YOUR_S3_ACCESS_KEY')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.secretKey', 'YOUR_S3_SECRET_KEY');
+                    const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration)
+                        .factoryMethod('getByName')
+                        .stringConstructorArgument('localAddress');
 
-                                break;
+                    if (locAddr.nonEmpty())
+                        clientBean.beanProperty('localAddress', locAddr);
 
-                            case 'Properties':
-                                credentialsBean = new Bean('com.amazonaws.auth.PropertiesCredentials', 'awsCredentials', {});
+                    clientBean.stringProperty('proxyHost')
+                        .intProperty('proxyPort')
+                        .stringProperty('proxyUsername');
 
-                                const fileBean = new Bean('java.io.File', '', spi.S3.awsCredentials.Properties)
-                                    .pathConstructorArgument('path');
+                    const userName = clientBean.valueOf('proxyUsername');
 
-                                if (fileBean.nonEmpty())
-                                    credentialsBean.beanConstructorArgument('file', fileBean);
+                    if (userName)
+                        clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`);
 
-                                break;
+                    clientBean.stringProperty('proxyDomain')
+                        .stringProperty('proxyWorkstation');
 
-                            case 'Anonymous':
-                                credentialsBean = new Bean('com.amazonaws.auth.AnonymousAWSCredentials', 'awsCredentials', {});
+                    const retryPolicy = spi.S3.clientConfiguration.retryPolicy;
 
-                                break;
+                    if (retryPolicy) {
+                        const kind = retryPolicy.kind;
 
-                            case 'BasicSession':
-                                credentialsBean = new Bean('com.amazonaws.auth.BasicSessionCredentials', 'awsCredentials', {});
+                        const policy = retryPolicy[kind];
 
-                                // TODO 2054 Arguments in one line is very long string.
-                                credentialsBean.propertyConstructorArgument('checkpoint.s3.credentials.accessKey')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.secretKey')
-                                    .propertyConstructorArgument('checkpoint.s3.credentials.sessionToken');
+                        let retryBean;
+
+                        switch (kind) {
+                            case 'Default':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY',
+                                    honorMaxErrorRetryInClientConfig: true
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constantConstructorArgument('maxErrorRetry')
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
                                 break;
 
-                            case 'Custom':
-                                const className = _.get(spi.S3.awsCredentials, 'Custom.className');
+                            case 'DefaultMaxRetries':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
+                                    honorMaxErrorRetryInClientConfig: false
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                                credentialsBean = new Bean(className, 'awsCredentials', {});
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
                                 break;
 
-                            default:
-                                break;
-                        }
+                            case 'DynamoDB':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY',
+                                    honorMaxErrorRetryInClientConfig: true
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                        if (credentialsBean)
-                            s3Bean.beanProperty('awsCredentials', credentialsBean);
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constantConstructorArgument('maxErrorRetry')
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        s3Bean.stringProperty('bucketNameSuffix');
+                                break;
 
-                        const clientBean = new Bean('com.amazonaws.ClientConfiguration', 'clientCfg', spi.S3.clientConfiguration,
-                            clusterDflts.checkpointSpi.S3.clientConfiguration);
+                            case 'DynamoDBMaxRetries':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
+                                    retryCondition: 'DEFAULT_RETRY_CONDITION',
+                                    backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
+                                    maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
+                                    honorMaxErrorRetryInClientConfig: false
+                                }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
 
-                        clientBean.enumProperty('protocol')
-                            .intProperty('maxConnections')
-                            .stringProperty('userAgent');
+                                retryBean.constantConstructorArgument('retryCondition')
+                                    .constantConstructorArgument('backoffStrategy')
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        const locAddr = new Bean('java.net.InetAddress', '', spi.S3.clientConfiguration)
-                            .factoryMethod('getByName')
-                            .stringConstructorArgument('localAddress');
+                                break;
 
-                        if (locAddr.nonEmpty())
-                            clientBean.beanProperty('localAddress', locAddr);
+                            case 'Custom':
+                                retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy);
 
-                        clientBean.stringProperty('proxyHost')
-                            .intProperty('proxyPort')
-                            .stringProperty('proxyUsername');
+                                retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null)
+                                    .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null)
+                                    .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
+                                    .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
 
-                        const userName = clientBean.valueOf('proxyUsername');
+                                break;
 
-                        if (userName)
-                            clientBean.property('proxyPassword', `checkpoint.s3.proxy.${userName}.password`);
+                            default:
+                                break;
+                        }
 
-                        clientBean.stringProperty('proxyDomain')
-                            .stringProperty('proxyWorkstation');
+                        if (retryBean)
+                            clientBean.beanProperty('retryPolicy', retryBean);
+                    }
 
-                        const retryPolicy = spi.S3.clientConfiguration.retryPolicy;
+                    clientBean.intProperty('maxErrorRetry')
+                        .intProperty('socketTimeout')
+                        .intProperty('connectionTimeout')
+                        .intProperty('requestTimeout')
+                        .intProperty('socketSendBufferSizeHints')
+                        .stringProperty('signerOverride')
+                        .intProperty('connectionTTL')
+                        .intProperty('connectionMaxIdleMillis')
+                        .emptyBeanProperty('dnsResolver')
+                        .intProperty('responseMetadataCacheSize')
+                        .emptyBeanProperty('secureRandom')
+                        .boolProperty('useReaper')
+                        .boolProperty('useGzip')
+                        .boolProperty('preemptiveBasicProxyAuth')
+                        .boolProperty('useTcpKeepAlive');
+
+                    if (clientBean.nonEmpty())
+                        s3Bean.beanProperty('clientConfiguration', clientBean);
+
+                    s3Bean.emptyBeanProperty('checkpointListener');
+
+                    return s3Bean;
+
+                case 'JDBC':
+                    const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi',
+                        'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
+
+                    const id = jdbcBean.valueOf('dataSourceBean');
+                    const dialect = _.get(spi.JDBC, 'dialect');
+
+                    jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect));
+
+                    if (!_.isEmpty(jdbcBean.valueOf('user'))) {
+                        jdbcBean.stringProperty('user')
+                            .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD');
+                    }
 
-                        if (retryPolicy) {
-                            const kind = retryPolicy.kind;
+                    jdbcBean.stringProperty('checkpointTableName')
+                        .stringProperty('keyFieldName')
+                        .stringProperty('keyFieldType')
+                        .stringProperty('valueFieldName')
+                        .stringProperty('valueFieldType')
+                        .stringProperty('expireDateFieldName')
+                        .stringProperty('expireDateFieldType')
+                        .intProperty('numberOfRetries')
+                        .emptyBeanProperty('checkpointListener');
 
-                            const policy = retryPolicy[kind];
+                    return jdbcBean;
 
-                            let retryBean;
+                case 'Custom':
+                    const clsName = _.get(spi, 'Custom.className');
 
-                            switch (kind) {
-                                case 'Default':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: 'DEFAULT_MAX_ERROR_RETRY',
-                                        honorMaxErrorRetryInClientConfig: true
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+                    if (clsName)
+                        return new Bean(clsName, 'checkpointSpiCustom', spi.Cache);
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constantConstructorArgument('maxErrorRetry')
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+                    return null;
 
-                                    break;
+                default:
+                    return null;
+            }
+        }), (checkpointBean) => _.nonNil(checkpointBean));
 
-                                case 'DefaultMaxRetries':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
-                                        honorMaxErrorRetryInClientConfig: false
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+        cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi');
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        return cfg;
+    }
 
-                                    break;
+    // Generate collision group.
+    static clusterCollision(collision, cfg = this.igniteConfigurationBean()) {
+        let colSpi;
+
+        switch (_.get(collision, 'kind')) {
+            case 'JobStealing':
+                colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi',
+                    'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing);
+
+                colSpi.intProperty('activeJobsThreshold')
+                    .intProperty('waitJobsThreshold')
+                    .intProperty('messageExpireTime')
+                    .intProperty('maximumStealingAttempts')
+                    .boolProperty('stealingEnabled')
+                    .emptyBeanProperty('externalCollisionListener')
+                    .mapProperty('stealingAttrs', 'stealingAttributes');
+
+                break;
+            case 'FifoQueue':
+                colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi',
+                    'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue);
+
+                colSpi.intProperty('parallelJobsNumber')
+                    .intProperty('waitingJobsNumber');
+
+                break;
+            case 'PriorityQueue':
+                colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi',
+                    'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue);
+
+                colSpi.intProperty('parallelJobsNumber')
+                    .intProperty('waitingJobsNumber')
+                    .intProperty('priorityAttributeKey')
+                    .intProperty('jobPriorityAttributeKey')
+                    .intProperty('defaultPriority')
+                    .intProperty('starvationIncrement')
+                    .boolProperty('starvationPreventionEnabled');
+
+                break;
+            case 'Custom':
+                if (_.nonNil(_.get(collision, 'Custom.class')))
+                    colSpi = new EmptyBean(collision.Custom.class);
+
+                break;
+            default:
+                return cfg;
+        }
 
-                                case 'DynamoDB':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: 'DYNAMODB_DEFAULT_MAX_ERROR_RETRY',
-                                        honorMaxErrorRetryInClientConfig: true
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+        if (_.nonNil(colSpi))
+            cfg.beanProperty('collisionSpi', colSpi);
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constantConstructorArgument('maxErrorRetry')
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        return cfg;
+    }
 
-                                    break;
+    // Generate communication group.
+    static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi',
+            cluster.communication, clusterDflts.communication);
+
+        commSpi.emptyBeanProperty('listener')
+            .stringProperty('localAddress')
+            .intProperty('localPort')
+            .intProperty('localPortRange')
+            .intProperty('sharedMemoryPort')
+            .intProperty('directBuffer')
+            .intProperty('directSendBuffer')
+            .intProperty('idleConnectionTimeout')
+            .intProperty('connectTimeout')
+            .intProperty('maxConnectTimeout')
+            .intProperty('reconnectCount')
+            .intProperty('socketSendBuffer')
+            .intProperty('socketReceiveBuffer')
+            .intProperty('messageQueueLimit')
+            .intProperty('slowClientQueueLimit')
+            .intProperty('tcpNoDelay')
+            .intProperty('ackSendThreshold')
+            .intProperty('unacknowledgedMessagesBufferSize')
+            .intProperty('socketWriteTimeout')
+            .intProperty('selectorsCount')
+            .emptyBeanProperty('addressResolver');
+
+        if (commSpi.nonEmpty())
+            cfg.beanProperty('communicationSpi', commSpi);
+
+        cfg.intProperty('networkTimeout')
+            .intProperty('networkSendRetryDelay')
+            .intProperty('networkSendRetryCount')
+            .intProperty('discoveryStartupDelay');
+
+        return cfg;
+    }
 
-                                case 'DynamoDBMaxRetries':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', {
-                                        retryCondition: 'DEFAULT_RETRY_CONDITION',
-                                        backoffStrategy: 'DYNAMODB_DEFAULT_BACKOFF_STRATEGY',
-                                        maxErrorRetry: _.get(policy, 'maxErrorRetry') || -1,
-                                        honorMaxErrorRetryInClientConfig: false
-                                    }, clusterDflts.checkpointSpi.S3.clientConfiguration.retryPolicy);
+    // Generate REST access configuration.
+    static clusterConnector(connector, cfg = this.igniteConfigurationBean()) {
+        const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration',
+            'connectorConfiguration', connector, clusterDflts.connector);
+
+        if (connCfg.valueOf('enabled')) {
+            connCfg.pathProperty('jettyPath')
+                .stringProperty('host')
+                .intProperty('port')
+                .intProperty('portRange')
+                .intProperty('idleTimeout')
+                .intProperty('idleQueryCursorTimeout')
+                .intProperty('idleQueryCursorCheckFrequency')
+                .intProperty('receiveBufferSize')
+                .intProperty('sendBufferSize')
+                .intProperty('sendQueueLimit')
+                .intProperty('directBuffer')
+                .intProperty('noDelay')
+                .intProperty('selectorCount')
+                .intProperty('threadPoolSize')
+                .emptyBeanProperty('messageInterceptor')
+                .stringProperty('secretKey');
+
+            if (connCfg.valueOf('sslEnabled')) {
+                connCfg.intProperty('sslClientAuth')
+                    .emptyBeanProperty('sslFactory');
+            }
 
-                                    retryBean.constantConstructorArgument('retryCondition')
-                                        .constantConstructorArgument('backoffStrategy')
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+            if (connCfg.nonEmpty())
+                cfg.beanProperty('connectorConfiguration', connCfg);
+        }
 
-                                    break;
+        return cfg;
+    }
 
-                                case 'Custom':
-                                    retryBean = new Bean('com.amazonaws.retry.RetryPolicy', 'retryPolicy', policy);
+    // Generate deployment group.
+    static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        cfg.enumProperty('deploymentMode')
+            .boolProperty('peerClassLoadingEnabled');
 
-                                    retryBean.beanConstructorArgument('retryCondition', retryBean.valueOf('retryCondition') ? new EmptyBean(retryBean.valueOf('retryCondition')) : null)
-                                        .beanConstructorArgument('backoffStrategy', retryBean.valueOf('backoffStrategy') ? new EmptyBean(retryBean.valueOf('backoffStrategy')) : null)
-                                        .constructorArgument('java.lang.Integer', retryBean.valueOf('maxErrorRetry'))
-                                        .constructorArgument('java.lang.Boolean', retryBean.valueOf('honorMaxErrorRetryInClientConfig'));
+        if (cfg.valueOf('peerClassLoadingEnabled')) {
+            cfg.intProperty('peerClassLoadingMissedResourcesCacheSize')
+                .intProperty('peerClassLoadingThreadPoolSize')
+                .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude',
+                   cluster.peerClassLoadingLocalClassPathExclude);
+        }
 
-                                    break;
+        let deploymentBean = null;
 
-                                default:
-                                    break;
-                            }
+        switch (_.get(cluster, 'deploymentSpi.kind')) {
+            case 'URI':
+                const uriDeployment = cluster.deploymentSpi.URI;
 
-                            if (retryBean)
-                                clientBean.beanProperty('retryPolicy', retryBean);
-                        }
+                deploymentBean = new Bean('org.apache.ignite.spi.deployment.uri.UriDeploymentSpi', 'deploymentSpi', uriDeployment);
 
-                        clientBean.intProperty('maxErrorRetry')
-                            .intProperty('socketTimeout')
-                            .intProperty('connectionTimeout')
-                            .intProperty('requestTimeout')
-                            .intProperty('socketSendBufferSizeHints')
-                            .stringProperty('signerOverride')
-                            .intProperty('connectionTTL')
-                            .intProperty('connectionMaxIdleMillis')
-                            .emptyBeanProperty('dnsResolver')
-                            .intProperty('responseMetadataCacheSize')
-                            .emptyBeanProperty('secureRandom')
-                            .boolProperty('useReaper')
-                            .boolProperty('useGzip')
-                            .boolProperty('preemptiveBasicProxyAuth')
-                            .boolProperty('useTcpKeepAlive');
-
-                        if (clientBean.nonEmpty())
-                            s3Bean.beanProperty('clientConfiguration', clientBean);
-
-                        s3Bean.emptyBeanProperty('checkpointListener');
-
-                        return s3Bean;
-
-                    case 'JDBC':
-                        const jdbcBean = new Bean('org.apache.ignite.spi.checkpoint.jdbc.JdbcCheckpointSpi',
-                            'checkpointSpiJdbc', spi.JDBC, clusterDflts.checkpointSpi.JDBC);
-
-                        const id = jdbcBean.valueOf('dataSourceBean');
-                        const dialect = _.get(spi.JDBC, 'dialect');
-
-                        jdbcBean.dataSource(id, 'dataSource', this.dataSourceBean(id, dialect));
-
-                        if (!_.isEmpty(jdbcBean.valueOf('user'))) {
-                            jdbcBean.stringProperty('user')
-                                .property('pwd', `checkpoint.${jdbcBean.valueOf('dataSourceBean')}.${jdbcBean.valueOf('user')}.jdbc.password`, 'YOUR_PASSWORD');
-                        }
+                const scanners = _.map(uriDeployment.scanners, (scanner) => new EmptyBean(scanner));
 
-                        jdbcBean.stringProperty('checkpointTableName')
-                            .stringProperty('keyFieldName')
-                            .stringProperty('keyFieldType')
-                            .stringProperty('valueFieldName')
-                            .stringProperty('valueFieldType')
-                            .stringProperty('expireDateFieldName')
-                            .stringProperty('expireDateFieldType')
-                            .intProperty('numberOfRetries')
-                            .emptyBeanProperty('checkpointListener');
+                deploymentBean.collectionProperty('uriList', 'uriList', uriDeployment.uriList)
+                    .stringProperty('temporaryDirectoryPath')
+                    .varArgProperty('scanners', 'scanners', scanners,
+                        'org.apache.ignite.spi.deployment.uri.scanners.UriDeploymentScanner')
+                    .emptyBeanProperty('listener')
+                    .boolProperty('checkMd5')
+                    .boolProperty('encodeUri');
 
-                        return jdbcBean;
+                cfg.beanProperty('deploymentSpi', deploymentBean);
 
-                    case 'Custom':
-                        const clsName = _.get(spi, 'Custom.className');
+                break;
 
-                        if (clsName)
-                            return new Bean(clsName, 'checkpointSpiCustom', spi.Cache);
+            case 'Local':
+                deploymentBean = new Bean('org.apache.ignite.spi.deployment.local.LocalDeploymentSpi', 'deploymentSpi', cluster.deploymentSpi.Local);
 
-                        return null;
+                deploymentBean.emptyBeanProperty('listener');
 
-                    default:
-                        return null;
-                }
-            }), (checkpointBean) => _.nonNil(checkpointBean));
+                cfg.beanProperty('deploymentSpi', deploymentBean);
 
-            cfg.arrayProperty('checkpointSpi', 'checkpointSpi', cfgs, 'org.apache.ignite.spi.checkpoint.CheckpointSpi');
+                break;
 
-            return cfg;
-        }
+            case 'Custom':
+                cfg.emptyBeanProperty('deploymentSpi.Custom.className');
 
-        // Generate collision group.
-        static clusterCollision(collision, cfg = this.igniteConfigurationBean()) {
-            let colSpi;
+                break;
 
-            switch (_.get(collision, 'kind')) {
-                case 'JobStealing':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi',
-                        'colSpi', collision.JobStealing, clusterDflts.collision.JobStealing);
+            default:
+                // No-op.
+        }
 
-                    colSpi.intProperty('activeJobsThreshold')
-                        .intProperty('waitJobsThreshold')
-                        .intProperty('messageExpireTime')
-                        .intProperty('maximumStealingAttempts')
-                        .boolProperty('stealingEnabled')
-                        .emptyBeanProperty('externalCollisionListener')
-                        .mapProperty('stealingAttrs', 'stealingAttributes');
+        return cfg;
+    }
 
-                    break;
-                case 'FifoQueue':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi',
-                        'colSpi', collision.FifoQueue, clusterDflts.collision.FifoQueue);
+    // Generate discovery group.
+    static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) {
+        discoSpi.stringProperty('localAddress')
+            .intProperty('localPort')
+            .intProperty('localPortRange')
+            .emptyBeanProperty('addressResolver')
+            .intProperty('socketTimeout')
+            .intProperty('ackTimeout')
+            .intProperty('maxAckTimeout')
+            .intProperty('networkTimeout')
+            .intProperty('joinTimeout')
+            .intProperty('threadPriority')
+            .intProperty('heartbeatFrequency')
+            .intProperty('maxMissedHeartbeats')
+            .intProperty('maxMissedClientHeartbeats')
+            .intProperty('topHistorySize')
+            .emptyBeanProperty('listener')
+            .emptyBeanProperty('dataExchange')
+            .emptyBeanProperty('metricsProvider')
+            .intProperty('reconnectCount')
+            .intProperty('statisticsPrintFrequency')
+            .intProperty('ipFinderCleanFrequency')
+            .emptyBeanProperty('authenticator')
+            .intProperty('forceServerMode')
+            .intProperty('clientReconnectDisabled');
+
+        if (discoSpi.nonEmpty())
+            cfg.beanProperty('discoverySpi', discoSpi);
+
+        return discoSpi;
+    }
 
-                    colSpi.intProperty('parallelJobsNumber')
-                        .intProperty('waitingJobsNumber');
+    // Generate events group.
+    static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const eventStorage = cluster.eventStorage;
 
-                    break;
-                case 'PriorityQueue':
-                    colSpi = new Bean('org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi',
-                        'colSpi', collision.PriorityQueue, clusterDflts.collision.PriorityQueue);
-
-                    colSpi.intProperty('parallelJobsNumber')
-                        .intProperty('waitingJobsNumber')
-                        .intProperty('priorityAttributeKey')
-                        .intProperty('jobPriorityAttributeKey')
-                        .intProperty('defaultPriority')
-                        .intProperty('starvationIncrement')
-                        .boolProperty('starvationPreventionEnabled');
+        let eventStorageBean = null;
 
-                    break;
-                case 'Custom':
-                    if (_.nonNil(_.get(collision, 'Custom.class')))
-                        colSpi = new EmptyBean(collision.Custom.class);
+        switch (_.get(eventStorage, 'kind')) {
+            case 'Memory':
+                eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory);
 
-                    break;
-                default:
-                    return cfg;
-            }
+                eventStorageBean.intProperty('expireAgeMs')
+                    .intProperty('expireCount')
+                    .emptyBeanProperty('filter');
 
-            if (_.nonNil(colSpi))
-                cfg.beanProperty('collisionSpi', colSpi);
+                break;
 
-            return cfg;
-        }
+            case 'Custom':
+                const className = _.get(eventStorage, 'Custom.className');
 
-        // Generate communication group.
-        static clusterCommunication(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const commSpi = new Bean('org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi', 'communicationSpi',
-                cluster.communication, clusterDflts.communication);
+                if (className)
+                    eventStorageBean = new EmptyBean(className);
 
-            commSpi.emptyBeanProperty('listener')
-                .stringProperty('localAddress')
-                .intProperty('localPort')
-                .intProperty('localPortRange')
-                .intProperty('sharedMemoryPort')
-                .intProperty('directBuffer')
-                .intProperty('directSendBuffer')
-                .intProperty('idleConnectionTimeout')
-                .intProperty('connectTimeout')
-                .intProperty('maxConnectTimeout')
-                .intProperty('reconnectCount')
-                .intProperty('socketSendBuffer')
-                .intProperty('socketReceiveBuffer')
-                .intProperty('messageQueueLimit')
-                .intProperty('slowClientQueueLimit')
-                .intProperty('tcpNoDelay')
-                .intProperty('ackSendThreshold')
-                .intProperty('unacknowledgedMessagesBufferSize')
-                .intProperty('socketWriteTimeout')
-                .intProperty('selectorsCount')
-                .emptyBeanProperty('addressResolver');
-
-            if (commSpi.nonEmpty())
-                cfg.beanProperty('communicationSpi', commSpi);
-
-            cfg.intProperty('networkTimeout')
-                .intProperty('networkSendRetryDelay')
-                .intProperty('networkSendRetryCount')
-                .intProperty('discoveryStartupDelay');
+                break;
 
-            return cfg;
+            default:
+                // No-op.
         }
 
-        // Generate REST access configuration.
-        static clusterConnector(connector, cfg = this.igniteConfigurationBean()) {
-            const connCfg = new Bean('org.apache.ignite.configuration.ConnectorConfiguration',
-                'connectorConfiguration', connector, clusterDflts.connector);
-
-            if (connCfg.valueOf('enabled')) {
-                connCfg.pathProperty('jettyPath')
-                    .stringProperty('host')
-                    .intProperty('port')
-                    .intProperty('portRange')
-                    .intProperty('idleTimeout')
-                    .intProperty('idleQueryCursorTimeout')
-                    .intProperty('idleQueryCursorCheckFrequency')
-                    .intProperty('receiveBufferSize')
-                    .intProperty('sendBufferSize')
-                    .intProperty('sendQueueLimit')
-                    .intProperty('directBuffer')
-                    .intProperty('noDelay')
-                    .intProperty('selectorCount')
-                    .intProperty('threadPoolSize')
-                    .emptyBeanProperty('messageInterceptor')
-                    .stringProperty('secretKey');
-
-                if (connCfg.valueOf('sslEnabled')) {
-                    connCfg.intProperty('sslClientAuth')
-                        .emptyBeanProperty('sslFactory');
-                }
-
-                if (connCfg.nonEmpty())
-                    cfg.beanProperty('connectorConfiguration', connCfg);
-            }
+        if (eventStorageBean && eventStorageBean.nonEmpty())
+            cfg.beanProperty('eventStorageSpi', eventStorageBean);
 
-            return cfg;
-        }
+        if (_.nonEmpty(cluster.includeEventTypes))
+            cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes);
 
-        // Generate deployment group.
-        static clusterDeployment(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            cfg.enumProperty('deploymentMode')
-                .boolProperty('peerClassLoadingEnabled');
+        return cfg;
+    }
 
-            if (cfg.valueOf('peerClassLoadingEnabled')) {
-                cfg.intProperty('peerClassLoadingMissedResourcesCacheSize')
-                    .intProperty('peerClassLoadingThreadPoolSize')
-                    .varArgProperty('p2pLocClsPathExcl', 'peerClassLoadingLocalClassPathExclude',
-                       cluster.peerClassLoadingLocalClassPathExclude);
-            }
+    // Generate failover group.
+    static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const spis = [];
 
-            return cfg;
-        }
+        _.forEach(cluster.failoverSpi, (spi) => {
+            let failoverSpi;
 
-        // Generate discovery group.
-        static clusterDiscovery(discovery, cfg = this.igniteConfigurationBean(), discoSpi = this.discoveryConfigurationBean(discovery)) {
-            discoSpi.stringProperty('localAddress')
-                .intProperty('localPort')
-                .intProperty('localPortRange')
-                .emptyBeanProperty('addressResolver')
-                .intProperty('socketTimeout')
-                .intProperty('ackTimeout')
-                .intProperty('maxAckTimeout')
-                .intProperty('networkTimeout')
-                .intProperty('joinTimeout')
-                .intProperty('threadPriority')
-                .intProperty('heartbeatFrequency')
-                .intProperty('maxMissedHeartbeats')
-                .intProperty('maxMissedClientHeartbeats')
-                .intProperty('topHistorySize')
-                .emptyBeanProperty('listener')
-                .emptyBeanProperty('dataExchange')
-                .emptyBeanProperty('metricsProvider')
-                .intProperty('reconnectCount')
-                .intProperty('statisticsPrintFrequency')
-                .intProperty('ipFinderCleanFrequency')
-                .emptyBeanProperty('authenticator')
-                .intProperty('forceServerMode')
-                .intProperty('clientReconnectDisabled');
-
-            if (discoSpi.nonEmpty())
-                cfg.beanProperty('discoverySpi', discoSpi);
-
-            return discoSpi;
-        }
+            switch (_.get(spi, 'kind')) {
+                case 'JobStealing':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi',
+                        'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing);
 
-        // Generate events group.
-        static clusterEvents(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const eventStorage = cluster.eventStorage;
+                    failoverSpi.intProperty('maximumFailoverAttempts');
 
-            let eventStorageBean = null;
+                    break;
+                case 'Never':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi',
+                        'failoverSpi', spi.Never);
 
-            switch (_.get(eventStorage, 'kind')) {
-                case 'Memory':
-                    eventStorageBean = new Bean('org.apache.ignite.spi.eventstorage.memory.MemoryEventStorageSpi', 'eventStorage', eventStorage.Memory, clusterDflts.eventStorage.Memory);
+                    break;
+                case 'Always':
+                    failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi',
+                        'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always);
 
-                    eventStorageBean.intProperty('expireAgeMs')
-                        .intProperty('expireCount')
-                        .emptyBeanProperty('filter');
+                    failoverSpi.intProperty('maximumFailoverAttempts');
 
                     break;
-
                 case 'Custom':
-                    const className = _.get(eventStorage, 'Custom.className');
+                    const className = _.get(spi, 'Custom.class');
 
                     if (className)
-                        eventStorageBean = new EmptyBean(className);
+                        failoverSpi = new EmptyBean(className);
 
                     break;
-
                 default:
                     // No-op.
             }
 
-            if (eventStorageBean && eventStorageBean.nonEmpty())
-                cfg.beanProperty('eventStorageSpi', eventStorageBean);
-
-            if (_.nonEmpty(cluster.includeEventTypes))
-                cfg.eventTypes('evts', 'includeEventTypes', cluster.includeEventTypes);
-
-            return cfg;
-        }
-
-        // Generate failover group.
-        static clusterFailover(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const spis = [];
-
-            _.forEach(cluster.failoverSpi, (spi) => {
-                let failoverSpi;
+            if (failoverSpi)
+                spis.push(failoverSpi);
+        });
 
-                switch (_.get(spi, 'kind')) {
-                    case 'JobStealing':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi',
-                            'failoverSpi', spi.JobStealing, clusterDflts.failoverSpi.JobStealing);
+        if (spis.length)
+            cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi');
 
-                        failoverSpi.intProperty('maximumFailoverAttempts');
-
-                        break;
-                    case 'Never':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.never.NeverFailoverSpi',
-                            'failoverSpi', spi.Never);
-
-                        break;
-                    case 'Always':
-                        failoverSpi = new Bean('org.apache.ignite.spi.failover.always.AlwaysFailoverSpi',
-                            'failoverSpi', spi.Always, clusterDflts.failoverSpi.Always);
+        return cfg;
+    }
 
-                        failoverSpi.intProperty('maximumFailoverAttempts');
+    // Generate load balancing configuration group.
+    static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) {
+        const spis = [];
 
-                        break;
-                    case 'Custom':
-                        const className = _.get(spi, 'Custom.class');
+        _.forEach(cluster.loadBalancingSpi, (spi) => {
+            let loadBalancingSpi;
 
-                        if (className)
-                            failoverSpi = new EmptyBean(className);
+            switch (_.get(spi, 'kind')) {
+                case 'RoundRobin':
+                    loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.roundrobin.RoundRobinLoadBalancingSpi', 'loadBalancingSpiRR', spi.RoundRobin, clusterDflts.loadBalancingSpi.RoundRobin);
 
-                        break;
-                    default:
-                        // No-op.
-                }
+                    loadBalancingSpi.boolProperty('perTask');
 
-                if (failoverSpi)
-                    spis.push(failoverSpi);
-            });
+                    break;
+                case 'Adaptive':
+                    loadBalancingSpi = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveLoadBalancingSpi', 'loadBalancingSpiAdaptive', spi.Adaptive);
 
-            if (spis.length)
-                cfg.arrayProperty('failoverSpi', 'failoverSpi', spis, 'org.apache.ignite.spi.failover.FailoverSpi');
+                    let probeBean;
 
-            return cfg;
-        }
+                    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);
 
-        // Generate load balancing configuration group.
-        static clusterLoadBalancing(cluster, cfg = this.igniteConfigurationBean(cluster)) {
-            const spis = [];
+                            probeBean.boolProperty('useAverage');
 
-            _.forEach(cluster.loadBalancingSpi, (spi) => {
-                let loadBalancingSpi;
+                            break;
+                        case 'CPU':
+                            probeBean = new Bean('org.apache.ignite.spi.loadbalancing.adaptive.AdaptiveCpuLoadProbe', 'cpuProbe', spi.Adaptive.loadProbe.CPU, clusterDflts.loadBalancingSpi.Adaptive

<TRUNCATED>

[33/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 e3f6461..03015e8 100644
--- a/modules/web-console/frontend/views/sql/sql.jade
+++ b/modules/web-console/frontend/views/sql/sql.jade
@@ -14,6 +14,7 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
+include /app/helpers/jade/mixins.jade
 include /app/directives/ui-grid-settings/ui-grid-settings.jade
 
 mixin btn-toolbar(btn, click, tip, focusId)
@@ -56,10 +57,15 @@ mixin notebook-rename
             .input-tip
                 input.form-control(ng-model='notebook.editName' required ignite-on-enter='renameNotebook(notebook.editName)' ignite-on-escape='notebook.edit = false;')
         h1.pull-right
-            a.dropdown-toggle(data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query
+            a.dropdown-toggle(style='margin-right: 20px' data-toggle='dropdown' bs-dropdown='scrollParagraphs' data-placement='bottom-right') Scroll to query
                 span.caret
-            .btn-group(style='margin-top: 2px')
-                +btn-toolbar('fa-plus', 'addParagraph()', 'Add new query')
+            button.btn.btn-default(style='margin-top: 2px' ng-click='addQuery()' ignite-on-click-focus=focusId)
+                i.fa.fa-fw.fa-plus
+                | Add query
+
+            button.btn.btn-default(style='margin-top: 2px' ng-click='addScan()' ignite-on-click-focus=focusId)
+                i.fa.fa-fw.fa-plus
+                | Add scan
 
 mixin notebook-error
     h2 Failed to load notebook
@@ -68,7 +74,7 @@ mixin notebook-error
 
 mixin paragraph-rename
     .col-sm-6(ng-hide='paragraph.edit')
-        i.tipLabel.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"')
+        i.fa(ng-class='paragraphExpanded(paragraph) ? "fa-chevron-circle-down" : "fa-chevron-circle-right"')
         label {{paragraph.name}}
 
         .btn-group(ng-hide='notebook.paragraphs.length > 1')
@@ -85,51 +91,45 @@ mixin paragraph-rename
             input.form-control(id='paragraph-name-{{paragraph.id}}' ng-model='paragraph.editName' required ng-click='$event.stopPropagation();' ignite-on-enter='renameParagraph(paragraph, paragraph.editName)' ignite-on-escape='paragraph.edit = false')
 
 mixin query-settings
-    label.tipLabel Refresh rate:
-        button.btn.btn-default.fa.fa-clock-o.tipLabel(title='Click to show refresh rate dialog' ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
-    label.tipLabel Page size:
-        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-options='item for item in pageSizes' bs-select bs-tooltip data-placement='bottom-right' data-title='Max number of rows to show in query result as one page')
-    label.margin-left-dflt(title='Fetch first page of results only')
-        input(type='checkbox' ng-model='paragraph.firstPageOnly')
-        span Fetch first page only
-    label.margin-left-dflt(title='Execute query locally on selected node.\nNode selection dialog will be shown before query execution.')
-        input(type='checkbox' ng-model='paragraph.localQry')
-        span Local query
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Configure periodical execution of last successfully executed query') Refresh rate:
+        button.btn.btn-default.fa.fa-clock-o.tipLabel(ng-class='{"btn-info": paragraph.rate && paragraph.rate.installed}' bs-popover data-template-url='/sql/paragraph-rate.html' data-placement='left' data-auto-close='1' data-trigger='click') {{rateAsString(paragraph)}}
+
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
+        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
+
+    label.tipLabel(bs-tooltip data-placement='bottom' data-title='Limit query max results to specified number of pages') Max pages:
+        button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.maxPages' bs-select bs-options='item.value as item.label for item in maxPages')
+
+    label.tipLabel(ng-if='nonCollocatedJoinsAvailable(paragraph)' bs-tooltip data-placement='bottom' data-title='Non-collocated joins is a special mode that allow to join data across cluster without collocation.<br/>\
+        Nested joins are not supported for now.<br/>\
+        <b>NOTE</b>: In some cases it may consume more heap memory or may take a long time than collocated joins.' data-trigger='hover')
+        input(type='checkbox' ng-model='paragraph.nonCollocatedJoins')
+        span Allow non-collocated joins
 
 mixin query-actions
-    .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute", true)}}' data-placement='bottom')
-        button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute
-        button.btn.btn-primary.dropdown-toggle(
-            ng-disabled='!actionAvailable(paragraph, true)'
-            bs-dropdown=''
-            data-container='body'
-            data-placement='bottom-right'
-        )
-            span.caret
-        ul.dropdown-menu(role='menu')
-            li #[a(href='javascript:void(0)' ng-click='execute(paragraph)') Execute]
-            li #[a(href='javascript:void(0)' ng-if='nonCollocatedJoinsAvailable(paragraph)' ng-click='execute(paragraph, true)') Execute non collocated joins]
-    .btn-group(bs-tooltip='' data-title='{{actionTooltip(paragraph, "execute scan", false)}}' data-placement='bottom')
-        button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)') Scan
-        button.btn.btn-primary.dropdown-toggle(
-            ng-disabled='!actionAvailable(paragraph, false)'
-            bs-dropdown=''
-            data-container='body'
-            data-placement='bottom-right'
-        )
-            span.caret
-        ul.dropdown-menu(role='menu')
-            li #[a(href='javascript:void(0)' ng-click='scan(paragraph)') Scan]
-            li #[a(href='javascript:void(0)' ng-click='actionAvailable(paragraph, false) && scanWithFilter(paragraph)') Scan with filter]
+    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph)') Execute
+    button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, true)' ng-click='execute(paragraph, true)') Execute on selected node
+
     a.btn.btn-default(ng-disabled='!actionAvailable(paragraph, true)' ng-click='explain(paragraph)' data-placement='bottom' bs-tooltip='' data-title='{{actionTooltip(paragraph, "explain", true)}}') Explain
 
-mixin query-controls
-    .sql-controls
-        +query-actions()
-        .pull-right
-            +query-settings()
+mixin table-result-heading-query
+    .total.row
+        .col-xs-4
+            +ui-grid-settings
+            label Page: #[b {{paragraph.page}}]
+            label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}]
+            label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}]
+        .col-xs-4
+            div(ng-if='paragraph.qryType === "query"')
+                +result-toolbar
+        .col-xs-4
+            .pull-right
+                .btn-group(ng-disabled='paragraph.loading')
+                    button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export
+                    button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right')
+                        span.caret
 
-mixin table-result
+mixin table-result-heading-scan
     .total.row
         .col-xs-4
             +ui-grid-settings
@@ -137,17 +137,16 @@ mixin table-result
             label.margin-left-dflt Results so far: #[b {{paragraph.rows.length + paragraph.total}}]
             label.margin-left-dflt Duration: #[b {{paragraph.duration | duration}}]
         .col-xs-4
-            +result-toolbar
+            div(ng-if='paragraph.qryType === "query"')
+                +result-toolbar
         .col-xs-4
             .pull-right
-                label(style='margin-right: 10px;')
-                    input(type='checkbox' ng-model='paragraph.systemColumns' ng-change='toggleSystemColumns(paragraph)' ng-disabled='paragraph.disabledSystemColumns')
-                    span Show _KEY, _VAL columns
                 .btn-group(ng-disabled='paragraph.loading')
                     button.btn.btn-primary(ng-click='exportCsv(paragraph)' bs-tooltip data-title='{{actionTooltip(paragraph, "export", false)}}') Export
                     button.btn.btn-primary.dropdown-toggle(id='export-item-dropdown' data-toggle='dropdown' data-container='body' bs-dropdown='exportDropdown' data-placement='bottom-right')
                         span.caret
-    
+
+mixin table-result-body
     .grid(ui-grid='paragraph.gridOptions' ui-grid-resize-columns ui-grid-exporter)
 
 mixin chart-result
@@ -166,12 +165,99 @@ mixin chart-result
                 +result-toolbar
         label.margin-top-dflt Charts do not support #[b Explain] and #[b Scan] query
 
+mixin paragraph-scan
+    .panel-heading(bs-collapse-toggle)
+        .row
+            +paragraph-rename
+    .panel-collapse(role='tabpanel' bs-collapse-target)
+        .col-sm-12.sql-controls
+            .col-sm-3
+                +dropdown-required('Cache:', 'paragraph.cacheName', '"cache"', 'true', 'false', 'Choose cache', 'caches')
+            .col-sm-3
+                +text-enabled('Filter:', 'paragraph.filter', '"filter"', true, false, 'Enter filter')
+                    label.btn.btn-default.ignite-form-field__btn(ng-click='paragraph.caseSensitive = !paragraph.caseSensitive')
+                        input(type='checkbox' ng-model='paragraph.caseSensitive')
+                        span(bs-tooltip data-title='Select this checkbox for case sensitive search') Cs
+            label.tipLabel(bs-tooltip data-placement='bottom' data-title='Max number of rows to show in query result as one page') Page size:
+                button.btn.btn-default.select-toggle.tipLabel(ng-model='paragraph.pageSize' bs-select bs-options='item for item in pageSizes')
+
+        .col-sm-12.sql-controls
+            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph)')
+                | Scan
+            button.btn.btn-primary(ng-disabled='!actionAvailable(paragraph, false)' ng-click='scan(paragraph, true)')
+                | Scan on selected node
+
+        .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
+            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .empty(ng-switch-when='empty') Result set is empty
+            .table(ng-switch-when='table')
+                +table-result-heading-scan
+                +table-result-body
+            .footer.clearfix()
+                .pull-left
+                    | Showing results for scan of #[b{{ paragraph.queryArgs.cacheName | defaultName }}]
+                    span(ng-if='paragraph.queryArgs.filter') &nbsp; with filter: #[b {{ paragraph.queryArgs.filter }}]
+                    span(ng-if='paragraph.queryArgs.localNid') &nbsp; on node: #[b {{ paragraph.queryArgs.localNid | limitTo:8 }}]
+
+                -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
+
+                .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
+                    i.fa.fa-chevron-circle-right
+                    a Next
+
+mixin paragraph-query
+    .row.panel-heading(bs-collapse-toggle)
+        +paragraph-rename
+    .panel-collapse(role='tabpanel' bs-collapse-target)
+        .col-sm-12
+            .col-xs-8.col-sm-9(style='border-right: 1px solid #eee')
+                .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' +
+                'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}'
+                ng-model='paragraph.query')
+            .col-xs-4.col-sm-3
+                div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches')
+                    lable.labelField.labelFormField Caches:
+                    i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}')
+                    .input-tip
+                        input.form-control(type='text' st-search='label' placeholder='Filter caches...')
+                    table.links
+                        tbody.scrollable-y(style='max-height: 15em; display: block;')
+                            tr(ng-repeat='cache in displayedCaches track by cache.name')
+                                td(style='width: 100%')
+                                    input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName')
+                                    label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind-html='cache.label')
+                .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0')
+                    label Wrong caches filter
+                .empty-caches(ng-show='caches.length == 0')
+                    label No caches
+        .col-sm-12.sql-controls
+            +query-actions
+
+            .pull-right
+                +query-settings
+        .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
+            .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
+            .empty(ng-switch-when='empty') Result set is empty
+            .table(ng-switch-when='table')
+                +table-result-heading-query
+                +table-result-body
+            .chart(ng-switch-when='chart')
+                +chart-result
+            .footer.clearfix
+                a.pull-left(ng-click='showResultQuery(paragraph)') Show query
+
+                -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
+
+                .pull-right(ng-show='#{nextVisibleCondition}' ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
+                    i.fa.fa-chevron-circle-right
+                    a Next
+
 .row(ng-controller='sqlController')
     .docs-content
         .row(ng-if='notebook' bs-affix style='margin-bottom: 20px;')
             +notebook-rename
 
-        ignite-information(data-title='With SQL notebook you can' style='margin-top: 0; margin-bottom: 30px')
+        ignite-information(data-title='With query notebook you can' style='margin-top: 0; margin-bottom: 30px')
             ul
                 li Create any number of queries
                 li Execute and explain SQL queries
@@ -184,46 +270,9 @@ mixin chart-result
         div(ng-if='notebook' ignite-loading='sqlLoading' ignite-loading-text='{{ loadingText }}' ignite-loading-position='top')
             .docs-body.paragraphs
                 .panel-group(bs-collapse ng-model='notebook.expandedParagraphs' data-allow-multiple='true' data-start-collapsed='false')
-                    .panel.panel-default(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}')
-                        .panel-heading(bs-collapse-toggle)
-                            .row
-                                +paragraph-rename
-                        .panel-collapse(role='tabpanel' bs-collapse-target)
-                            .col-sm-12
-                                .col-xs-8.col-sm-9(style='border-right: 1px solid #eee')
-                                    .sql-editor(ignite-ace='{onLoad: aceInit(paragraph), theme: "chrome", mode: "sql", require: ["ace/ext/language_tools"],' +
-                                    'advanced: {enableSnippets: false, enableBasicAutocompletion: true, enableLiveAutocompletion: true}}'
-                                    ng-model='paragraph.query')
-                                .col-xs-4.col-sm-3
-                                    div(ng-show='caches.length > 0' style='padding: 5px 10px' st-table='displayedCaches' st-safe-src='caches')
-                                        lable.labelField.labelFormField Caches:
-                                        i.fa.fa-database.tipField(title='Click to show cache types metadata dialog' bs-popover data-template-url='/sql/cache-metadata.html' data-placement='bottom' data-trigger='click' data-container='#{{ paragraph.id }}')
-                                        .input-tip
-                                            input.form-control(type='text' st-search='label' placeholder='Filter caches...')
-                                        table.links
-                                            tbody.scrollable-y(style='max-height: 15em; display: block;')
-                                                tr(ng-repeat='cache in displayedCaches track by cache.name')
-                                                    td(style='width: 100%')
-                                                        input.labelField(id='cache_{{ [paragraph.id, $index].join("_") }}' type='radio' value='{{cache.name}}' ng-model='paragraph.cacheName')
-                                                        label(for='cache_{{ [paragraph.id, $index].join("_") }} ' ng-bind='cache.label')
-                                    .empty-caches(ng-show='displayedCaches.length == 0 && caches.length != 0')
-                                        label Wrong caches filter
-                                    .empty-caches(ng-show='caches.length == 0')
-                                        label No caches
-                            .col-sm-12
-                                +query-controls
-                            .col-sm-12.sql-result(ng-if='paragraph.queryExecuted()' ng-switch='paragraph.resultType()')
-                                .error(ng-switch-when='error') Error: {{paragraph.errMsg}}
-                                .empty(ng-switch-when='empty') Result set is empty
-                                .table(ng-switch-when='table')
-                                    +table-result
-                                .chart(ng-switch-when='chart')
-                                    +chart-result
-                                .footer.clearfix
-                                    a.pull-left(ng-click='showResultQuery(paragraph)') Show query
-
-                                    -var nextVisibleCondition = 'paragraph.resultType() != "error" && paragraph.queryId && paragraph.nonRefresh() && (paragraph.table() || paragraph.chart() && !paragraph.scanExplain())'
-
-                                    .pull-right(ng-show=nextVisibleCondition ng-class='{disabled: paragraph.loading}' ng-click='!paragraph.loading && nextPage(paragraph)')
-                                        i.fa.fa-chevron-circle-right
-                                        a Next
+
+                    .panel-paragraph(ng-repeat='paragraph in notebook.paragraphs' id='{{paragraph.id}}' ng-form='form_{{paragraph.id}}')
+                        .panel.panel-default(ng-if='paragraph.qryType === "scan"')
+                            +paragraph-scan
+                        .panel.panel-default(ng-if='paragraph.qryType === "query"')
+                            +paragraph-query

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/templates/alert.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/alert.jade b/modules/web-console/frontend/views/templates/alert.jade
index 182ba99..d30d2fd 100644
--- a/modules/web-console/frontend/views/templates/alert.jade
+++ b/modules/web-console/frontend/views/templates/alert.jade
@@ -16,6 +16,6 @@
 
 .alert(ng-show='type' ng-class='[type ? "alert-" + type : null]')
     button.close(type='button', ng-if='dismissable', ng-click='$hide()') &times;
-    i.alert-icon.fa(ng-if='icon' ng-class='[icon]')
+    i.alert-icon(ng-if='icon' ng-class='[icon]')
     span.alert-title(ng-bind-html='title')
     span.alert-content(ng-bind-html='content')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/templates/select.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/templates/select.jade b/modules/web-console/frontend/views/templates/select.jade
index 5b6cc01..aa6a2ef 100644
--- a/modules/web-console/frontend/views/templates/select.jade
+++ b/modules/web-console/frontend/views/templates/select.jade
@@ -23,4 +23,4 @@ ul.select.dropdown-menu(tabindex='-1' ng-show='$isVisible()' role='select')
         hr(ng-if='match.value == undefined' style='margin: 5px 0')
         a(id='li-dropdown-item-{{$index}}'  role='menuitem' tabindex='-1' ng-class='{active: $isActive($index)}' ng-click='$select($index, $event)' bs-tooltip='widthIsSufficient && !widthIsSufficient("li-dropdown-item-{{$index}}", $index, match.label) ? match.label : ""' data-placement='right auto')
             i(class='{{$iconCheckmark}}' ng-if='$isActive($index)' ng-class='{active: $isActive($index)}')
-            span(ng-bind='match.label')
+            span(ng-bind-html='match.label')


[27/50] [abbrv] ignite git commit: ignite-4450 Need release locks for failing nodes during exchange processing.

Posted by vo...@apache.org.
ignite-4450 Need release locks for failing nodes during exchange processing.


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

Branch: refs/heads/ignite-2.0
Commit: 41dddb87da53bbe72594b0b4bb9e2a396a57b986
Parents: c864fe4
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 11:26:18 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 11:26:18 2016 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |   6 +-
 .../CacheLockReleaseNodeLeaveTest.java          | 135 +++++++++++++++++++
 2 files changed, 139 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/41dddb87/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4f34401..2cfc0d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -805,8 +805,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             }
             catch (IgniteFutureTimeoutCheckedException ignored) {
                 if (dumpedObjects < DUMP_PENDING_OBJECTS_THRESHOLD) {
-                    U.warn(log, "Failed to wait for locks release future. " +
-                        "Dumping pending objects that might be the cause: " + cctx.localNodeId());
+                    U.warn(log, "Failed to wait for locks release future. Dumping pending objects that might be the " +
+                        "cause [topVer=" + topologyVersion() + ", nodeId=" + cctx.localNodeId() + "]: ");
 
                     U.warn(log, "Locked keys:");
 
@@ -1556,6 +1556,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         if (isDone() || !enterBusy())
             return;
 
+        cctx.mvcc().removeExplicitNodeLocks(node.id(), topologyVersion());
+
         try {
             onDiscoveryEvent(new IgniteRunnable() {
                 @Override public void run() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/41dddb87/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
index e84fd3f..969b991 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheLockReleaseNodeLeaveTest.java
@@ -20,9 +20,13 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
@@ -158,4 +162,135 @@ public class CacheLockReleaseNodeLeaveTest extends GridCommonAbstractTest {
 
         fut2.get(5, SECONDS);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        Lock lock = ignite1.cache(null).lock("key");
+        lock.lock();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        lock = ignite2.cache(null).lock("key");
+        lock.lock();
+        lock.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockRelease3() throws Exception {
+        startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        awaitPartitionMapExchange();
+
+        Lock lock = ignite1.cache(null).lock("key");
+        lock.lock();
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        lock = ignite2.cache(null).lock("key");
+        lock.lock();
+        lock.unlock();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTxLockRelease2() throws Exception {
+        final Ignite ignite0 = startGrid(0);
+
+        Ignite ignite1 = startGrid(1);
+
+        IgniteCache cache = ignite1.cache(null);
+        ignite1.transactions().txStart(PESSIMISTIC, REPEATABLE_READ);
+        cache.get(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        final AffinityTopologyVersion topVer = new AffinityTopologyVersion(2, 0);
+
+        // Wait when affinity change exchange start.
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                AffinityTopologyVersion topVer0 =
+                    ((IgniteKernal)ignite0).context().cache().context().exchange().topologyVersion();
+
+                return topVer.compareTo(topVer0) < 0;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
+        assertFalse(fut.isDone());
+
+        ignite1.close();
+
+        fut.get(10_000);
+
+        Ignite ignite2 = ignite(2);
+
+        cache = ignite2.cache(null);
+
+        try (Transaction tx = ignite2.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.get(1);
+
+            tx.commit();
+        }
+    }
 }


[23/50] [abbrv] ignite git commit: ignite-4379: fixed issue after merge with latest DML changes

Posted by vo...@apache.org.
ignite-4379: fixed issue after merge with latest DML changes


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

Branch: refs/heads/ignite-2.0
Commit: e8fd1657de7847922feec3d3e58250ad116e4e47
Parents: 800579c
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 15:27:04 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 15:27:04 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/h2/IgniteH2Indexing.java      | 2 --
 1 file changed, 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e8fd1657/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c541185..9037943 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -801,8 +801,6 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt);
 
         if (!p.isQuery()) {
-            GridH2QueryContext.clearThreadLocal();
-
             SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
 
             if (params != null)


[43/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
Web console beta-7.


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

Branch: refs/heads/ignite-2.0
Commit: 8e7c852b5ff4a3837f2f33f3d9adc0c2318dbdf8
Parents: 1e2469f
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Dec 23 16:34:10 2016 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Dec 23 16:34:11 2016 +0700

----------------------------------------------------------------------
 modules/web-console/backend/app/agent.js        |   15 +
 modules/web-console/backend/app/browser.js      |   13 +
 modules/web-console/backend/app/mongo.js        |   24 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   12 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   52 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |    5 +
 .../generator/ConfigurationGenerator.js         | 2795 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2318 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |    2 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  497 ++-
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  137 -
 .../defaults/cluster.platform.provider.js       |   49 -
 .../generator/defaults/cluster.provider.js      |  293 --
 .../generator/defaults/igfs.provider.js         |   68 -
 .../configuration/generator/generator-common.js |  625 ---
 .../configuration/generator/generator-java.js   | 3617 ------------------
 .../generator/generator-optional.js             |   25 -
 .../configuration/generator/generator-spring.js | 2111 ----------
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  211 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/node-filter.jade       |    2 +-
 .../states/configuration/caches/query.jade      |    3 +
 .../states/configuration/caches/store.jade      |    4 +-
 .../configuration/clusters/checkpoint.jade      |   11 +-
 .../configuration/clusters/checkpoint/fs.jade   |    8 +-
 .../configuration/clusters/checkpoint/jdbc.jade |    8 +-
 .../configuration/clusters/checkpoint/s3.jade   |   25 +-
 .../clusters/collision/custom.jade              |    2 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/deployment.jade      |  129 +-
 .../states/configuration/clusters/events.jade   |    4 +-
 .../states/configuration/clusters/failover.jade |    4 +-
 .../clusters/general/discovery/zookeeper.jade   |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    2 +-
 .../configuration/clusters/load-balancing.jade  |   23 +-
 .../configuration/clusters/logger/custom.jade   |    2 +-
 .../states/configuration/clusters/ssl.jade      |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |  103 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   42 +-
 .../frontend/controllers/domains-controller.js  |   32 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../gulpfile.babel.js/webpack/common.js         |   17 +-
 .../webpack/environments/development.js         |   14 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  178 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   23 +-
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  111 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../views/configuration/domains-import.jade     |    5 +-
 .../frontend/views/configuration/summary.jade   |   25 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  235 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 112 files changed, 5577 insertions(+), 11296 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 f74a3f2..791ea50 100644
--- a/modules/web-console/backend/app/agent.js
+++ b/modules/web-console/backend/app/agent.js
@@ -314,6 +314,21 @@ 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.
+         * @returns {Promise}
+         */
+        queryResetDetailMetrics(demo, nids) {
+            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.VisorCacheResetQueryDetailMetricsTask')
+                .addParam('p3', 'java.lang.Void');
+
+            return this.executeRest(cmd);
+        }
+
+        /**
+         * @param {Boolean} demo Is need run command on demo node.
          * @param {String} cacheName Cache name.
          * @returns {Promise}
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 2710829..499d84d 100644
--- a/modules/web-console/backend/app/browser.js
+++ b/modules/web-console/backend/app/browser.js
@@ -162,6 +162,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:reset:metrics', (nids, cb) => {
+                    agentMgr.findAgent(accountId())
+                        .then((agent) => agent.queryResetDetailMetrics(demo, nids))
+                        .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/8e7c852b/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 0f38eb2..58ab119 100644
--- a/modules/web-console/backend/app/mongo.js
+++ b/modules/web-console/backend/app/mongo.js
@@ -247,6 +247,7 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
         longQueryWarningTimeout: Number,
         sqlFunctionClasses: [String],
         snapshotableIndex: Boolean,
+        queryDetailMetricsSize: Number,
         statisticsEnabled: Boolean,
         managementEnabled: Boolean,
         readFromBackup: Boolean,
@@ -823,7 +824,24 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
             Custom: {
                 className: String
             }
-        }]
+        }],
+        deploymentSpi: {
+            kind: {type: String, enum: ['URI', 'Local', 'Custom']},
+            URI: {
+                uriList: [String],
+                temporaryDirectoryPath: String,
+                scanners: [String],
+                listener: String,
+                checkMd5: Boolean,
+                encodeUri: Boolean
+            },
+            Local: {
+                listener: String
+            },
+            Custom: {
+                className: String
+            }
+        }
     });
 
     ClusterSchema.index({name: 1, space: 1}, {unique: true});
@@ -843,13 +861,15 @@ module.exports.factory = function(passportMongo, settings, pluginMongo, mongoose
             result: {type: String, enum: ['none', 'table', 'bar', 'pie', 'line', 'area']},
             pageSize: Number,
             timeLineSpan: String,
+            maxPages: Number,
             hideSystemColumns: Boolean,
             cacheName: String,
             chartsOptions: {barChart: {stacked: Boolean}, areaChart: {style: String}},
             rate: {
                 value: Number,
                 unit: Number
-            }
+            },
+            qryType: String
         }]
     });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/routes/demo.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/demo.js b/modules/web-console/backend/routes/demo.js
index ad4be6e..3f4166d 100644
--- a/modules/web-console/backend/routes/demo.js
+++ b/modules/web-console/backend/routes/demo.js
@@ -39,20 +39,17 @@ module.exports.factory = (_, express, settings, mongo, spacesService, errors) =>
         router.post('/reset', (req, res) => {
             spacesService.spaces(req.user._id, true)
                 .then((spaces) => {
-                    if (spaces.length) {
-                        const spaceIds = spaces.map((space) => space._id);
-
-                        return Promise.all([
-                            mongo.Cluster.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.Cache.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(),
-                            mongo.Igfs.remove({space: {$in: spaceIds}}).exec()
-                        ]).then(() => spaces[0]);
-                    }
+                    const spaceIds = _.map(spaces, '_id');
+
+                    return spacesService.cleanUp(spaceIds)
+                        .then(() => mongo.Space.remove({_id: {$in: _.tail(spaceIds)}}).exec())
+                        .then(() => _.head(spaces));
                 })
                 .catch((err) => {
                     if (err instanceof errors.MissingResourceException)
                         return spacesService.createDemoSpace(req.user._id);
+
+                    throw err;
                 })
                 .then((space) => {
                     return Promise.all(_.map(clusters, (cluster) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/routes/profile.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/routes/profile.js b/modules/web-console/backend/routes/profile.js
index 4d01cda..1d6fccb 100644
--- a/modules/web-console/backend/routes/profile.js
+++ b/modules/web-console/backend/routes/profile.js
@@ -45,7 +45,7 @@ module.exports.factory = function(_, express, mongo, usersService) {
 
             usersService.save(req.body)
                 .then((user) => {
-                    const becomeUsed = req.session.viewedUser && user.admin;
+                    const becomeUsed = req.session.viewedUser && req.user.admin;
 
                     if (becomeUsed) {
                         req.session.viewedUser = user;
@@ -64,6 +64,7 @@ module.exports.factory = function(_, express, mongo, usersService) {
                         });
                     });
                 })
+                .then(() => usersService.get(req.user, req.session.viewedUser))
                 .then(res.api.ok)
                 .catch(res.api.error);
         });

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/notebooks.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/notebooks.js b/modules/web-console/backend/services/notebooks.js
index 8846d8e..9aa2c38 100644
--- a/modules/web-console/backend/services/notebooks.js
+++ b/modules/web-console/backend/services/notebooks.js
@@ -34,12 +34,14 @@ module.exports = {
 module.exports.factory = (_, mongo, spacesService, errors) => {
     /**
      * Convert remove status operation to own presentation.
+     *
      * @param {RemoveResult} result - The results of remove operation.
      */
     const convertRemoveStatus = ({result}) => ({rowsAffected: result.n});
 
     /**
-     * Update existing notebook
+     * Update existing notebook.
+     *
      * @param {Object} notebook - The notebook for updating
      * @returns {Promise.<mongo.ObjectId>} that resolves cache id
      */
@@ -53,6 +55,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
 
     /**
      * Create new notebook.
+     *
      * @param {Object} notebook - The notebook for creation.
      * @returns {Promise.<mongo.ObjectId>} that resolves cache id.
      */
@@ -67,6 +70,7 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
     class NotebooksService {
         /**
          * Create or update Notebook.
+         *
          * @param {Object} notebook - The Notebook
          * @returns {Promise.<mongo.ObjectId>} that resolves Notebook id of merge operation.
          */
@@ -78,16 +82,18 @@ module.exports.factory = (_, mongo, spacesService, errors) => {
         }
 
         /**
-         * Get caches by spaces.
+         * Get notebooks by spaces.
+         *
          * @param {mongo.ObjectId|String} spaceIds - The spaces ids that own caches.
-         * @returns {Promise.<mongo.Cache[]>} - contains requested caches.
+         * @returns {Promise.<mongo.Notebook[]>} - contains requested caches.
          */
         static listBySpaces(spaceIds) {
             return mongo.Notebook.find({space: {$in: spaceIds}}).sort('name').lean().exec();
         }
 
         /**
-         * Remove Notebook.
+         * Remove notebook.
+         *
          * @param {mongo.ObjectId|String} notebookId - The Notebook id for remove.
          * @returns {Promise.<{rowsAffected}>} - The number of affected rows.
          */

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/sessions.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/sessions.js b/modules/web-console/backend/services/sessions.js
index ff0e303..7f62a60 100644
--- a/modules/web-console/backend/services/sessions.js
+++ b/modules/web-console/backend/services/sessions.js
@@ -38,11 +38,11 @@ module.exports.factory = (_, mongo, errors) => {
          * @param {mongo.ObjectId|String} viewedUserId - id of user to become.
          */
         static become(session, viewedUserId) {
+            if (!session.req.user.admin)
+                return Promise.reject(new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.'));
+
             return mongo.Account.findById(viewedUserId).lean().exec()
                 .then((viewedUser) => {
-                    if (!session.req.user.admin)
-                        throw new errors.IllegalAccessError('Became this user is not permitted. Only administrators can perform this actions.');
-
                     viewedUser.token = session.req.user.token;
 
                     session.viewedUser = viewedUser;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/backend/services/spaces.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/services/spaces.js b/modules/web-console/backend/services/spaces.js
index 863d57c..85f346e 100644
--- a/modules/web-console/backend/services/spaces.js
+++ b/modules/web-console/backend/services/spaces.js
@@ -68,6 +68,21 @@ module.exports.factory = (mongo, errors) => {
         static createDemoSpace(userId) {
             return new mongo.Space({name: 'Demo space', owner: userId, demo: true}).save();
         }
+
+        /**
+         * Clean up spaces.
+         *
+         * @param {mongo.ObjectId|String} spaceIds - The space ids for clean up.
+         * @returns {Promise.<>}
+         */
+        static cleanUp(spaceIds) {
+            return Promise.all([
+                mongo.Cluster.remove({space: {$in: spaceIds}}).exec(),
+                mongo.Cache.remove({space: {$in: spaceIds}}).exec(),
+                mongo.DomainModel.remove({space: {$in: spaceIds}}).exec(),
+                mongo.Igfs.remove({space: {$in: spaceIds}}).exec()
+            ]);
+        }
     }
 
     return SpacesService;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/app.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.js b/modules/web-console/frontend/app/app.js
index 3510743..4ecd9b5 100644
--- a/modules/web-console/frontend/app/app.js
+++ b/modules/web-console/frontend/app/app.js
@@ -99,11 +99,6 @@ import domainsValidation from './filters/domainsValidation.filter';
 import duration from './filters/duration.filter';
 import hasPojo from './filters/hasPojo.filter';
 
-// Generators
-import $generatorOptional from './modules/configuration/generator/generator-optional';
-
-window.$generatorOptional = $generatorOptional;
-
 // Controllers
 import admin from 'controllers/admin-controller';
 import caches from 'controllers/caches-controller';

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/controllers/reset-password.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/controllers/reset-password.controller.js b/modules/web-console/frontend/app/controllers/reset-password.controller.js
index da0c37b..f84a876 100644
--- a/modules/web-console/frontend/app/controllers/reset-password.controller.js
+++ b/modules/web-console/frontend/app/controllers/reset-password.controller.js
@@ -21,10 +21,10 @@ export default ['resetPassword', [
     ($scope, $modal, $http, $state, Messages, Focus) => {
         if ($state.params.token) {
             $http.post('/api/v1/password/validate/token', {token: $state.params.token})
-                .success((res) => {
-                    $scope.email = res.email;
-                    $scope.token = res.token;
-                    $scope.error = res.error;
+                .then(({data}) => {
+                    $scope.email = data.email;
+                    $scope.token = data.token;
+                    $scope.error = data.error;
 
                     if ($scope.token && !$scope.error)
                         Focus.move('user_password');
@@ -34,16 +34,16 @@ export default ['resetPassword', [
         // Try to reset user password for provided token.
         $scope.resetPassword = (reset_info) => {
             $http.post('/api/v1/password/reset', reset_info)
-                .success(() => {
+                .then(() => {
                     $state.go('signin');
 
                     Messages.showInfo('Password successfully changed');
                 })
-                .error((err, state) => {
+                .catch(({data, state}) => {
                     if (state === 503)
                         $state.go('signin');
 
-                    Messages.showError(err);
+                    Messages.showError(data);
                 });
         };
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/data/event-groups.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/event-groups.json b/modules/web-console/frontend/app/data/event-groups.json
new file mode 100644
index 0000000..8d0c878
--- /dev/null
+++ b/modules/web-console/frontend/app/data/event-groups.json
@@ -0,0 +1,169 @@
+[
+  {
+    "label": "EVTS_CHECKPOINT",
+    "value": "EVTS_CHECKPOINT",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CHECKPOINT_SAVED",
+      "EVT_CHECKPOINT_LOADED",
+      "EVT_CHECKPOINT_REMOVED"
+    ]
+  },
+  {
+    "label": "EVTS_DEPLOYMENT",
+    "value": "EVTS_DEPLOYMENT",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CLASS_DEPLOYED",
+      "EVT_CLASS_UNDEPLOYED",
+      "EVT_CLASS_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOYED",
+      "EVT_TASK_UNDEPLOYED",
+      "EVT_TASK_DEPLOY_FAILED"
+    ]
+  },
+  {
+    "label": "EVTS_ERROR",
+    "value": "EVTS_ERROR",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_JOB_TIMEDOUT",
+      "EVT_JOB_FAILED",
+      "EVT_JOB_FAILED_OVER",
+      "EVT_JOB_REJECTED",
+      "EVT_JOB_CANCELLED",
+      "EVT_TASK_TIMEDOUT",
+      "EVT_TASK_FAILED",
+      "EVT_CLASS_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOY_FAILED",
+      "EVT_TASK_DEPLOYED",
+      "EVT_TASK_UNDEPLOYED",
+      "EVT_CACHE_REBALANCE_STARTED",
+      "EVT_CACHE_REBALANCE_STOPPED"
+    ]
+  },
+  {
+    "label": "EVTS_DISCOVERY",
+    "value": "EVTS_DISCOVERY",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_NODE_JOINED",
+      "EVT_NODE_LEFT",
+      "EVT_NODE_FAILED",
+      "EVT_NODE_SEGMENTED",
+      "EVT_CLIENT_NODE_DISCONNECTED",
+      "EVT_CLIENT_NODE_RECONNECTED"
+    ]
+  },
+  {
+    "label": "EVTS_JOB_EXECUTION",
+    "value": "EVTS_JOB_EXECUTION",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_JOB_MAPPED",
+      "EVT_JOB_RESULTED",
+      "EVT_JOB_FAILED_OVER",
+      "EVT_JOB_STARTED",
+      "EVT_JOB_FINISHED",
+      "EVT_JOB_TIMEDOUT",
+      "EVT_JOB_REJECTED",
+      "EVT_JOB_FAILED",
+      "EVT_JOB_QUEUED",
+      "EVT_JOB_CANCELLED"
+    ]
+  },
+  {
+    "label": "EVTS_TASK_EXECUTION",
+    "value": "EVTS_TASK_EXECUTION",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_TASK_STARTED",
+      "EVT_TASK_FINISHED",
+      "EVT_TASK_FAILED",
+      "EVT_TASK_TIMEDOUT",
+      "EVT_TASK_SESSION_ATTR_SET",
+      "EVT_TASK_REDUCED"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE",
+    "value": "EVTS_CACHE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_ENTRY_CREATED",
+      "EVT_CACHE_ENTRY_DESTROYED",
+      "EVT_CACHE_OBJECT_PUT",
+      "EVT_CACHE_OBJECT_READ",
+      "EVT_CACHE_OBJECT_REMOVED",
+      "EVT_CACHE_OBJECT_LOCKED",
+      "EVT_CACHE_OBJECT_UNLOCKED",
+      "EVT_CACHE_OBJECT_SWAPPED",
+      "EVT_CACHE_OBJECT_UNSWAPPED",
+      "EVT_CACHE_OBJECT_EXPIRED"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_REBALANCE",
+    "value": "EVTS_CACHE_REBALANCE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_REBALANCE_STARTED",
+      "EVT_CACHE_REBALANCE_STOPPED",
+      "EVT_CACHE_REBALANCE_PART_LOADED",
+      "EVT_CACHE_REBALANCE_PART_UNLOADED",
+      "EVT_CACHE_REBALANCE_OBJECT_LOADED",
+      "EVT_CACHE_REBALANCE_OBJECT_UNLOADED",
+      "EVT_CACHE_REBALANCE_PART_DATA_LOST"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_LIFECYCLE",
+    "value": "EVTS_CACHE_LIFECYCLE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_STARTED",
+      "EVT_CACHE_STOPPED",
+      "EVT_CACHE_NODES_LEFT"
+    ]
+  },
+  {
+    "label": "EVTS_CACHE_QUERY",
+    "value": "EVTS_CACHE_QUERY",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_CACHE_QUERY_EXECUTED",
+      "EVT_CACHE_QUERY_OBJECT_READ"
+    ]
+  },
+  {
+    "label": "EVTS_SWAPSPACE",
+    "value": "EVTS_SWAPSPACE",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_SWAP_SPACE_CLEARED",
+      "EVT_SWAP_SPACE_DATA_REMOVED",
+      "EVT_SWAP_SPACE_DATA_READ",
+      "EVT_SWAP_SPACE_DATA_STORED",
+      "EVT_SWAP_SPACE_DATA_EVICTED"
+    ]
+  },
+  {
+    "label": "EVTS_IGFS",
+    "value": "EVTS_IGFS",
+    "class": "org.apache.ignite.events.EventType",
+    "events": [
+      "EVT_IGFS_FILE_CREATED",
+      "EVT_IGFS_FILE_RENAMED",
+      "EVT_IGFS_FILE_DELETED",
+      "EVT_IGFS_FILE_OPENED_READ",
+      "EVT_IGFS_FILE_OPENED_WRITE",
+      "EVT_IGFS_FILE_CLOSED_WRITE",
+      "EVT_IGFS_FILE_CLOSED_READ",
+      "EVT_IGFS_FILE_PURGED",
+      "EVT_IGFS_META_UPDATED",
+      "EVT_IGFS_DIR_CREATED",
+      "EVT_IGFS_DIR_RENAMED",
+      "EVT_IGFS_DIR_DELETED"
+    ]
+  }
+]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/data/event-types.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/data/event-types.json b/modules/web-console/frontend/app/data/event-types.json
deleted file mode 100644
index 8d0c878..0000000
--- a/modules/web-console/frontend/app/data/event-types.json
+++ /dev/null
@@ -1,169 +0,0 @@
-[
-  {
-    "label": "EVTS_CHECKPOINT",
-    "value": "EVTS_CHECKPOINT",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CHECKPOINT_SAVED",
-      "EVT_CHECKPOINT_LOADED",
-      "EVT_CHECKPOINT_REMOVED"
-    ]
-  },
-  {
-    "label": "EVTS_DEPLOYMENT",
-    "value": "EVTS_DEPLOYMENT",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CLASS_DEPLOYED",
-      "EVT_CLASS_UNDEPLOYED",
-      "EVT_CLASS_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOYED",
-      "EVT_TASK_UNDEPLOYED",
-      "EVT_TASK_DEPLOY_FAILED"
-    ]
-  },
-  {
-    "label": "EVTS_ERROR",
-    "value": "EVTS_ERROR",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_JOB_TIMEDOUT",
-      "EVT_JOB_FAILED",
-      "EVT_JOB_FAILED_OVER",
-      "EVT_JOB_REJECTED",
-      "EVT_JOB_CANCELLED",
-      "EVT_TASK_TIMEDOUT",
-      "EVT_TASK_FAILED",
-      "EVT_CLASS_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOY_FAILED",
-      "EVT_TASK_DEPLOYED",
-      "EVT_TASK_UNDEPLOYED",
-      "EVT_CACHE_REBALANCE_STARTED",
-      "EVT_CACHE_REBALANCE_STOPPED"
-    ]
-  },
-  {
-    "label": "EVTS_DISCOVERY",
-    "value": "EVTS_DISCOVERY",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_NODE_JOINED",
-      "EVT_NODE_LEFT",
-      "EVT_NODE_FAILED",
-      "EVT_NODE_SEGMENTED",
-      "EVT_CLIENT_NODE_DISCONNECTED",
-      "EVT_CLIENT_NODE_RECONNECTED"
-    ]
-  },
-  {
-    "label": "EVTS_JOB_EXECUTION",
-    "value": "EVTS_JOB_EXECUTION",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_JOB_MAPPED",
-      "EVT_JOB_RESULTED",
-      "EVT_JOB_FAILED_OVER",
-      "EVT_JOB_STARTED",
-      "EVT_JOB_FINISHED",
-      "EVT_JOB_TIMEDOUT",
-      "EVT_JOB_REJECTED",
-      "EVT_JOB_FAILED",
-      "EVT_JOB_QUEUED",
-      "EVT_JOB_CANCELLED"
-    ]
-  },
-  {
-    "label": "EVTS_TASK_EXECUTION",
-    "value": "EVTS_TASK_EXECUTION",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_TASK_STARTED",
-      "EVT_TASK_FINISHED",
-      "EVT_TASK_FAILED",
-      "EVT_TASK_TIMEDOUT",
-      "EVT_TASK_SESSION_ATTR_SET",
-      "EVT_TASK_REDUCED"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE",
-    "value": "EVTS_CACHE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_ENTRY_CREATED",
-      "EVT_CACHE_ENTRY_DESTROYED",
-      "EVT_CACHE_OBJECT_PUT",
-      "EVT_CACHE_OBJECT_READ",
-      "EVT_CACHE_OBJECT_REMOVED",
-      "EVT_CACHE_OBJECT_LOCKED",
-      "EVT_CACHE_OBJECT_UNLOCKED",
-      "EVT_CACHE_OBJECT_SWAPPED",
-      "EVT_CACHE_OBJECT_UNSWAPPED",
-      "EVT_CACHE_OBJECT_EXPIRED"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_REBALANCE",
-    "value": "EVTS_CACHE_REBALANCE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_REBALANCE_STARTED",
-      "EVT_CACHE_REBALANCE_STOPPED",
-      "EVT_CACHE_REBALANCE_PART_LOADED",
-      "EVT_CACHE_REBALANCE_PART_UNLOADED",
-      "EVT_CACHE_REBALANCE_OBJECT_LOADED",
-      "EVT_CACHE_REBALANCE_OBJECT_UNLOADED",
-      "EVT_CACHE_REBALANCE_PART_DATA_LOST"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_LIFECYCLE",
-    "value": "EVTS_CACHE_LIFECYCLE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_STARTED",
-      "EVT_CACHE_STOPPED",
-      "EVT_CACHE_NODES_LEFT"
-    ]
-  },
-  {
-    "label": "EVTS_CACHE_QUERY",
-    "value": "EVTS_CACHE_QUERY",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_CACHE_QUERY_EXECUTED",
-      "EVT_CACHE_QUERY_OBJECT_READ"
-    ]
-  },
-  {
-    "label": "EVTS_SWAPSPACE",
-    "value": "EVTS_SWAPSPACE",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_SWAP_SPACE_CLEARED",
-      "EVT_SWAP_SPACE_DATA_REMOVED",
-      "EVT_SWAP_SPACE_DATA_READ",
-      "EVT_SWAP_SPACE_DATA_STORED",
-      "EVT_SWAP_SPACE_DATA_EVICTED"
-    ]
-  },
-  {
-    "label": "EVTS_IGFS",
-    "value": "EVTS_IGFS",
-    "class": "org.apache.ignite.events.EventType",
-    "events": [
-      "EVT_IGFS_FILE_CREATED",
-      "EVT_IGFS_FILE_RENAMED",
-      "EVT_IGFS_FILE_DELETED",
-      "EVT_IGFS_FILE_OPENED_READ",
-      "EVT_IGFS_FILE_OPENED_WRITE",
-      "EVT_IGFS_FILE_CLOSED_WRITE",
-      "EVT_IGFS_FILE_CLOSED_READ",
-      "EVT_IGFS_FILE_PURGED",
-      "EVT_IGFS_META_UPDATED",
-      "EVT_IGFS_DIR_CREATED",
-      "EVT_IGFS_DIR_RENAMED",
-      "EVT_IGFS_DIR_DELETED"
-    ]
-  }
-]

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 acf2bc8..7d2bed0 100644
--- a/modules/web-console/frontend/app/data/pom-dependencies.json
+++ b/modules/web-console/frontend/app/data/pom-dependencies.json
@@ -10,11 +10,11 @@
     "HadoopIgfsJcl": {"artifactId": "ignite-hadoop"},
     "SLF4J": {"artifactId": "ignite-slf4j"},
 
-    "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"},
+    "Generic": {"groupId": "com.mchange", "artifactId": "c3p0", "version": "0.9.5.2"},
+    "MySQL": {"groupId": "mysql", "artifactId": "mysql-connector-java", "version": "5.1.40"},
+    "PostgreSQL": {"groupId": "org.postgresql", "artifactId": "postgresql", "version": "9.4.1212.jre7"},
     "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"}
+    "Oracle": {"groupId": "com.oracle.jdbc", "artifactId": "ojdbc7", "version": "12.1.0.2", "jar": "ojdbc7.jar"},
+    "DB2": {"groupId": "ibm", "artifactId": "jdbc", "version": "4.21.29", "jar": "db2jcc4.jar"},
+    "SQLServer": {"groupId": "microsoft", "artifactId": "jdbc", "version": "4.2", "jar": "sqljdbc41.jar"}
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
index 32feaf3..de335ae 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.controller.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-export default ['$scope', 'GeneratorDocker', function($scope, docker) {
+export default ['$scope', 'IgniteDockerGenerator', function($scope, docker) {
     const ctrl = this;
 
     // Watchers definition.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
index 3b0e7b8..3a24cfb 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
+++ b/modules/web-console/frontend/app/directives/ui-ace-docker/ui-ace-docker.jade
@@ -20,7 +20,7 @@ mixin hard-link(ref, txt)
 .panel-details-noborder
     .details-row
         p
-            +hard-link('https://docs.docker.com/reference/builder', 'Docker')
+            +hard-link('https://docs.docker.com/engine/reference/builder/', 'Docker')
             | &nbsp;file is a text file with instructions to create Docker image.<br/>
             | To build image you have to store following Docker file with your Ignite XML configuration to the same directory.<br>
             | Also you could use predefined&nbsp;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
index 4e11874..61bf086 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-pojos/ui-ace-pojos.controller.js
@@ -30,7 +30,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
     const updatePojos = () => {
         delete ctrl.pojos;
 
-        if (!ctrl.cluster || !ctrl.cluster.caches)
+        if (_.isNil(ctrl.cluster) || _.isEmpty(ctrl.cluster.caches))
             return;
 
         ctrl.pojos = generator.pojos(ctrl.cluster.caches, ctrl.useConstructor, ctrl.includeKeyFields);
@@ -46,7 +46,7 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
         const classes = ctrl.classes = [];
 
         _.forEach(ctrl.pojos, (pojo) => {
-            if (pojo.keyType && JavaTypes.nonBuiltInClass(pojo.keyType))
+            if (_.nonNil(pojo.keyClass))
                 classes.push(pojo.keyType);
 
             classes.push(pojo.valueType);
@@ -55,17 +55,17 @@ export default ['$scope', 'JavaTypes', 'JavaTransformer', function($scope, JavaT
 
     // Update pojos class.
     const updateClass = (value) => {
-        if (!value || !ctrl.pojos.length)
+        if (_.isEmpty(value))
             return;
 
-        const keyType = ctrl.pojos[0].keyType;
+        const pojo = value[0];
 
-        ctrl.class = ctrl.class || (JavaTypes.nonBuiltInClass(keyType) ? keyType : null) || ctrl.pojos[0].valueType;
+        ctrl.class = ctrl.class || (pojo.keyClass ? pojo.keyType : pojo.valueType);
     };
 
     // Update pojos data.
     const updatePojosData = (value) => {
-        if (!value)
+        if (_.isNil(value))
             return;
 
         _.forEach(ctrl.pojos, (pojo) => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
index 2bf78c3..477cf20 100644
--- a/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
+++ b/modules/web-console/frontend/app/directives/ui-ace-pom/ui-ace-pom.controller.js
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom, Version) {
+export default ['$scope', 'IgniteMavenGenerator', 'IgniteVersion', function($scope, maven, Version) {
     const ctrl = this;
 
     // Watchers definition.
@@ -25,7 +25,7 @@ export default ['$scope', 'GeneratorPom', 'IgniteVersion', function($scope, pom,
         if (!value)
             return;
 
-        ctrl.data = pom.generate($scope.cluster, Version.productVersion().ignite).asString();
+        ctrl.data = maven.generate($scope.cluster, Version.productVersion().ignite).asString();
     };
 
     // Setup watchers.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 298db52..33af6d1 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
@@ -28,7 +28,7 @@ mixin ignite-form-field-dropdown(label, model, name, disabled, required, multipl
             data-ng-disabled=disabled && '#{disabled}' || '!#{options}.length'
 
             bs-select
-            bs-options='item.value as item.label for item in #{options}' 
+            bs-options='item.value as item.label for item in #{options}'
 
             data-multiple=multiple ? '1' : false
             data-container='body > .wrapper'
@@ -41,7 +41,8 @@ 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.icon-help(bs-tooltip='' data-title=tip)
+            if tip
+                i.tipField.icon-help(bs-tooltip='' data-title=tip)
 
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 d48343c..58b0dcd 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,8 @@ 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.icon-help(bs-tooltip='' data-title=tip)
+            if 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/8e7c852b/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 136d23b..1f93d3b 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
@@ -30,13 +30,30 @@ mixin ignite-form-field-input(name, model, disabled, required, placeholder)
         data-ignite-form-panel-field=''
     )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {})
 
+mixin ignite-form-field-url-input(name, model, disabled, required, placeholder)
+    input.form-control(
+        id='{{ #{name} }}Input'
+        name='{{ #{name} }}'
+        placeholder=placeholder
+        type='url'
+
+        data-ng-model=model
+
+        data-ng-required=required && '#{required}'
+        data-ng-disabled=disabled && '#{disabled}'
+        data-ng-focus='tableReset()'
+
+        data-ignite-form-panel-field=''
+    )&attributes(attributes ? attributes.attributes ? attributes.attributes : attributes: {})
+
 mixin ignite-form-field-text(label, model, name, disabled, required, placeholder, tip)
     -var errLbl = label.substring(0, label.length - 1)
 
     .ignite-form-field
         +ignite-form-field__label(label, name, required)
         .ignite-form-field__control
-            i.tipField.icon-help(bs-tooltip='' data-title=tip)
+            if tip
+                i.tipField.icon-help(bs-tooltip='' data-title=tip)
             
             if block
                 block

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 92af1b0..6ca41f6 100644
--- a/modules/web-console/frontend/app/helpers/jade/mixins.jade
+++ b/modules/web-console/frontend/app/helpers/jade/mixins.jade
@@ -183,6 +183,14 @@ mixin text-enabled(lbl, model, name, enabled, required, placeholder, tip)
         if  block
             block
 
+//- Mixin for text field with autofocus.
+mixin text-enabled-autofocus(lbl, model, name, enabled, required, placeholder, tip)
+    +ignite-form-field-text(lbl, model, name, enabledToDisabled(enabled), required, placeholder, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for text field.
 mixin text(lbl, model, name, required, placeholder, tip)
     +ignite-form-field-text(lbl, model, name, false, required, placeholder, tip)
@@ -221,12 +229,28 @@ mixin dropdown-required-empty(lbl, model, name, enabled, required, placeholder,
         if  block
             block
 
+//- Mixin for required dropdown field with autofocus.
+mixin dropdown-required-empty-autofocus(lbl, model, name, enabled, required, placeholder, placeholderEmpty, options, tip)
+    +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, placeholderEmpty, options, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for required dropdown field.
 mixin dropdown-required(lbl, model, name, enabled, required, placeholder, options, tip)
     +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip)
         if  block
             block
 
+//- Mixin for required dropdown field with autofocus.
+mixin dropdown-required-autofocus(lbl, model, name, enabled, required, placeholder, options, tip)
+    +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), required, false, placeholder, '', options, tip)(
+        data-ignite-form-field-input-autofocus='true'
+    )
+        if  block
+            block
+
 //- Mixin for dropdown field.
 mixin dropdown(lbl, model, name, enabled, placeholder, options, tip)
     +ignite-form-field-dropdown(lbl, model, name, enabledToDisabled(enabled), false, false, placeholder, '', options, tip)
@@ -324,6 +348,28 @@ mixin table-java-package-field(name, model, items, valid, save, newItem)
                 ignite-on-escape=onEscape
             )
 
+//- Mixin for table java package field.
+mixin table-url-field(name, model, items, valid, save, newItem)
+    -var resetOnEnter = newItem ? '(stopblur = true) && (group.add = [{}])' : '(field.edit = false)'
+    -var onEnter = valid + ' && (' + save + '); ' + valid + ' && ' + resetOnEnter + ';'
+
+    -var onEscape = newItem ? 'group.add = []' : 'field.edit = false'
+
+    -var resetOnBlur = newItem ? '!stopblur && (group.add = [])' : 'field.edit = false'
+    -var onBlur = valid + ' && ( ' + save + '); ' + resetOnBlur + ';'
+
+    div(ignite-on-focus-out=onBlur)
+        if block
+            block
+
+        .input-tip
+            +ignite-form-field-url-input(name, model, false, 'true', 'Enter URL')(
+                data-ignite-unique=items
+                data-ignite-form-field-input-autofocus='true'
+
+                ignite-on-enter=onEnter
+                ignite-on-escape=onEscape
+            )
 
 //- Mixin for table address field.
 mixin table-address-field(name, model, items, valid, save, newItem, portRange)
@@ -393,17 +439,17 @@ mixin table-save-button(valid, save, newItem)
     )
 
 //- Mixin for table remove button.
-mixin table-remove-conditional-button(items, show, tip)
+mixin table-remove-conditional-button(items, show, tip, row)
     i.tipField.fa.fa-remove(
         ng-hide='!#{show} || field.edit'
         bs-tooltip
         data-title=tip
-        ng-click='#{items}.splice(#{items}.indexOf(model), 1)'
+        ng-click='#{items}.splice(#{items}.indexOf(#{row}), 1)'
     )
 
 //- Mixin for table remove button.
 mixin table-remove-button(items, tip)
-    +table-remove-conditional-button(items, 'true', tip)
+    +table-remove-conditional-button(items, 'true', tip, 'model')
 
 //- Mixin for cache mode.
 mixin cacheMode(lbl, model, name, placeholder)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/Demo/Demo.module.js b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
index 83d55ed..a3700ca 100644
--- a/modules/web-console/frontend/app/modules/Demo/Demo.module.js
+++ b/modules/web-console/frontend/app/modules/Demo/Demo.module.js
@@ -41,11 +41,11 @@ angular
             url: '/demo/reset',
             controller: ['$state', '$http', 'IgniteMessages', ($state, $http, Messages) => {
                 $http.post('/api/v1/demo/reset')
-                    .success(() => $state.go('base.configuration.clusters'))
-                    .error((err) => {
+                    .then(() => $state.go('base.configuration.clusters'))
+                    .catch((res) => {
                         $state.go('base.configuration.clusters');
 
-                        Messages.showError(err);
+                        Messages.showError(res);
                     });
             }],
             metaTags: {}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js b/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
deleted file mode 100644
index 61f3188..0000000
--- a/modules/web-console/frontend/app/modules/configuration/EventGroups.provider.js
+++ /dev/null
@@ -1,30 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-// Events groups.
-import GROUPS from 'app/data/event-types.json';
-
-export default ['igniteEventGroups', function() {
-    const groups = GROUPS;
-
-    this.push = (data) => groups.push(data);
-
-    this.$get = [() => {
-        return groups;
-    }];
-}];
-

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/Version.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/Version.service.js b/modules/web-console/frontend/app/modules/configuration/Version.service.js
index 06efdda..f0e9c4c 100644
--- a/modules/web-console/frontend/app/modules/configuration/Version.service.js
+++ b/modules/web-console/frontend/app/modules/configuration/Version.service.js
@@ -22,7 +22,7 @@ const VERSION_MATCHER = /(\d+)\.(\d+)\.(\d+)([-.]([^0123456789][^-]+)(-SNAPSHOT)
 
 const numberComparator = (a, b) => a > b ? 1 : a < b ? -1 : 0;
 
-export default class Version {
+export default class IgniteVersion {
     /**
      * Tries to parse product version from it's string representation.
      *
@@ -70,7 +70,7 @@ export default class Version {
         if (res !== 0)
             return res;
 
-        return numberComparator(pa.revTs, pb.maintenance);
+        return numberComparator(pa.revTs, pb.revTs);
     }
 
     /**
@@ -79,7 +79,7 @@ export default class Version {
      */
     productVersion() {
         return {
-            ignite: '1.7.0'
+            ignite: '1.8.0'
         };
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/configuration.module.js b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
index 27f7bef..4288ff7 100644
--- a/modules/web-console/frontend/app/modules/configuration/configuration.module.js
+++ b/modules/web-console/frontend/app/modules/configuration/configuration.module.js
@@ -17,26 +17,28 @@
 
 import angular from 'angular';
 
-import igniteEventGroups from './EventGroups.provider';
+
 import igniteSidebar from './Sidebar.provider';
-import Version from './Version.service';
+import IgniteVersion from './Version.service';
 
-import clusterDefaults from './generator/defaults/cluster.provider';
-import clusterPlatformDefaults from './generator/defaults/cluster.platform.provider';
-import cacheDefaults from './generator/defaults/cache.provider';
-import cachePlatformDefaults from './generator/defaults/cache.platform.provider';
-import igfsDefaults from './generator/defaults/igfs.provider';
+import IgniteClusterDefaults from './generator/defaults/Cluster.service';
+import IgniteClusterPlatformDefaults from './generator/defaults/Cluster.platform.service';
+import IgniteCacheDefaults from './generator/defaults/Cache.service';
+import IgniteCachePlatformDefaults from './generator/defaults/Cache.platform.service';
+import IgniteIGFSDefaults from './generator/defaults/IGFS.service';
+import IgniteEventGroups from './generator/defaults/Event-groups.service';
 
-import ConfigurationGenerator from './generator/ConfigurationGenerator';
-import PlatformGenerator from './generator/PlatformGenerator';
+import IgniteConfigurationGenerator from './generator/ConfigurationGenerator';
+import IgnitePlatformGenerator from './generator/PlatformGenerator';
 
-import SpringTransformer from './generator/SpringTransformer.service';
-import JavaTransformer from './generator/JavaTransformer.service';
+import IgniteSpringTransformer from './generator/SpringTransformer.service';
+import IgniteJavaTransformer from './generator/JavaTransformer.service';
 import SharpTransformer from './generator/SharpTransformer.service';
-import GeneratorDocker from './generator/Docker.service';
-import GeneratorPom from './generator/Pom.service';
-import GeneratorProperties from './generator/Properties.service';
-import GeneratorReadme from './generator/Readme.service';
+import IgniteDockerGenerator from './generator/Docker.service';
+import IgniteMavenGenerator from './generator/Maven.service';
+import IgniteGeneratorProperties from './generator/Properties.service';
+import IgniteReadmeGenerator from './generator/Readme.service';
+import IgniteCustomGenerator from './generator/Custom.service';
 
 import igniteSidebarDirective from './sidebar.directive';
 
@@ -45,21 +47,22 @@ angular
 .module('ignite-console.configuration', [
 
 ])
-.provider('igniteClusterDefaults', clusterDefaults)
-.provider('igniteClusterPlatformDefaults', clusterPlatformDefaults)
-.provider('igniteCacheDefaults', cacheDefaults)
-.provider('igniteCachePlatformDefaults', cachePlatformDefaults)
-.provider('igniteIgfsDefaults', igfsDefaults)
-.provider(...igniteEventGroups)
 .provider(...igniteSidebar)
 .directive(...igniteSidebarDirective)
-.service('IgniteVersion', Version)
-.service('IgniteConfigurationGenerator', ConfigurationGenerator)
-.service('IgnitePlatformGenerator', PlatformGenerator)
-.service('SpringTransformer', SpringTransformer)
-.service('JavaTransformer', JavaTransformer)
+.service('IgniteConfigurationGenerator', IgniteConfigurationGenerator)
+.service('IgnitePlatformGenerator', IgnitePlatformGenerator)
+.service('SpringTransformer', IgniteSpringTransformer)
+.service('JavaTransformer', IgniteJavaTransformer)
 .service('IgniteSharpTransformer', SharpTransformer)
-.service('IgnitePropertiesGenerator', GeneratorProperties)
-.service('IgniteReadmeGenerator', GeneratorReadme)
-.service(...GeneratorDocker)
-.service(...GeneratorPom);
+.service('IgniteVersion', IgniteVersion)
+.service('IgniteEventGroups', IgniteEventGroups)
+.service('IgniteClusterDefaults', IgniteClusterDefaults)
+.service('IgniteClusterPlatformDefaults', IgniteClusterPlatformDefaults)
+.service('IgniteCacheDefaults', IgniteCacheDefaults)
+.service('IgniteCachePlatformDefaults', IgniteCachePlatformDefaults)
+.service('IgniteIGFSDefaults', IgniteIGFSDefaults)
+.service('IgnitePropertiesGenerator', IgniteGeneratorProperties)
+.service('IgniteReadmeGenerator', IgniteReadmeGenerator)
+.service('IgniteDockerGenerator', IgniteDockerGenerator)
+.service('IgniteMavenGenerator', IgniteMavenGenerator)
+.service('IgniteCustomGenerator', IgniteCustomGenerator);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
index 6244a53..f5afe59 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/AbstractTransformer.js
@@ -17,7 +17,24 @@
 
 import StringBuilder from './StringBuilder';
 
+import IgniteConfigurationGenerator from './ConfigurationGenerator';
+import IgniteEventGroups from './defaults/Event-groups.service';
+
+import IgniteClusterDefaults from './defaults/Cluster.service';
+import IgniteCacheDefaults from './defaults/Cache.service';
+import IgniteIGFSDefaults from './defaults/IGFS.service';
+
+import JavaTypes from '../../../services/JavaTypes.service';
+
+const clusterDflts = new IgniteClusterDefaults();
+const cacheDflts = new IgniteCacheDefaults();
+const igfsDflts = new IgniteIGFSDefaults();
+
 export default class AbstractTransformer {
+    static generator = IgniteConfigurationGenerator;
+    static javaTypes = new JavaTypes(clusterDflts, cacheDflts, igfsDflts);
+    static eventGroups = new IgniteEventGroups();
+
     // Append comment with time stamp.
     static mainComment(sb, ...lines) {
         lines.push(sb.generatedBy());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 2750626..ca19342 100644
--- a/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
+++ b/modules/web-console/frontend/app/modules/configuration/generator/Beans.js
@@ -17,6 +17,11 @@
 
 import _ from 'lodash';
 
+_.mixin({
+    nonNil: _.negate(_.isNil),
+    nonEmpty: _.negate(_.isEmpty)
+});
+
 export class EmptyBean {
     /**
      * @param {String} clsName


[13/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
IGNITE-3220 I/O bottleneck on server/client cluster configuration
Communications optimizations:
- possibility to open separate in/out connections
- possibility to have multiple connections between nodes
- implemented NIO sessions balancing between NIO threads
- reduced amount of work and blocking calls in NIO threads
Other:
- implemented StripedExecutor for cache messages handling
- added 'io test' messages for IO performance testing

(cherry picked from commit 10ade28)


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

Branch: refs/heads/ignite-2.0
Commit: 05dd08b993e2d7f88176c051463b178431714f85
Parents: 57eb47f
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 12:28:47 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 16 14:01:53 2016 +0300

----------------------------------------------------------------------
 .../ignite/examples/ExampleNodeStartup.java     |    2 +-
 .../examples/datagrid/CachePutGetExample.java   |    2 +-
 .../scalar/examples/ScalarJvmCloudExample.scala |    2 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../rest/protocols/tcp/MockNioSession.java      |   25 +-
 .../apache/ignite/IgniteSystemProperties.java   |    3 +
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |    6 +-
 .../configuration/IgniteConfiguration.java      |   50 +-
 .../internal/GridEventConsumeHandler.java       |    2 +-
 .../ignite/internal/GridJobContextImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |    9 +
 .../ignite/internal/GridKernalContextImpl.java  |   16 +-
 .../internal/GridPerformanceSuggestions.java    |    2 +-
 .../org/apache/ignite/internal/GridTopic.java   |    5 +-
 .../ignite/internal/IgniteInternalFuture.java   |   11 +
 .../apache/ignite/internal/IgniteKernal.java    |   85 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   32 +-
 .../GridClientConnectionManagerAdapter.java     |    1 +
 .../client/router/impl/GridTcpRouterImpl.java   |    1 +
 .../managers/communication/GridIoManager.java   |  207 ++-
 .../managers/communication/GridIoMessage.java   |   13 +
 .../communication/GridIoMessageFactory.java     |   12 +-
 .../communication/IgniteIoTestMessage.java      |  235 +++
 .../processors/cache/GridCacheAdapter.java      |   26 +-
 .../processors/cache/GridCacheMessage.java      |    7 +
 .../processors/cache/GridCacheUtils.java        |   35 +
 .../processors/cache/IgniteCacheProxy.java      |    8 +
 .../distributed/GridDistributedLockRequest.java |    5 +
 .../GridDistributedTxFinishResponse.java        |    6 +
 .../GridDistributedUnlockRequest.java           |    5 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 +-
 .../distributed/dht/GridDhtLockResponse.java    |    9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    5 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |    5 +
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    5 +
 .../atomic/GridNearAtomicFullUpdateRequest.java |    5 +
 .../GridNearAtomicSingleUpdateRequest.java      |    5 +
 .../distributed/near/GridNearGetRequest.java    |    5 +
 .../local/atomic/GridLocalAtomicCache.java      |    3 +
 .../query/GridCacheDistributedQueryManager.java |    2 +-
 .../cache/query/GridCacheQueryRequest.java      |    6 +-
 .../transactions/IgniteTxLocalAdapter.java      |    8 +-
 .../datastreamer/DataStreamProcessor.java       |   22 +-
 .../internal/processors/igfs/IgfsContext.java   |    4 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 +-
 .../internal/processors/odbc/OdbcProcessor.java |    1 +
 .../platform/compute/PlatformCompute.java       |    6 +
 .../tcp/GridTcpMemcachedNioListener.java        |   15 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |    2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    1 +
 .../service/GridServiceProcessor.java           |    6 +-
 .../ignite/internal/util/IgniteUtils.java       |   62 +-
 .../ignite/internal/util/StripedExecutor.java   |  667 +++++++++
 .../util/future/GridFinishedFuture.java         |   24 +
 .../internal/util/future/GridFutureAdapter.java |   15 +-
 .../util/future/GridFutureChainListener.java    |   30 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    2 +-
 .../nio/GridAbstractCommunicationClient.java    |   12 +-
 .../util/nio/GridCommunicationClient.java       |    9 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   15 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   10 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   17 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   16 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   10 +-
 .../internal/util/nio/GridNioFilterChain.java   |   14 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    4 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  124 +-
 .../ignite/internal/util/nio/GridNioServer.java | 1404 +++++++++++++++---
 .../internal/util/nio/GridNioSession.java       |   25 +-
 .../internal/util/nio/GridNioSessionImpl.java   |   65 +-
 .../ignite/internal/util/nio/GridNioWorker.java |   48 +
 .../util/nio/GridSelectorNioSessionImpl.java    |  221 ++-
 .../util/nio/GridShmemCommunicationClient.java  |    7 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   55 +-
 .../internal/util/nio/SessionWriteRequest.java  |   85 ++
 .../internal/util/nio/ssl/GridNioSslFilter.java |   10 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    4 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 1340 ++++++++++++-----
 .../tcp/TcpCommunicationSpiMBean.java           |   40 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   14 +-
 .../ignite/stream/socket/SocketStreamer.java    |    1 +
 .../ignite/thread/IgniteThreadFactory.java      |    8 +-
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 ...unicationBalanceMultipleConnectionsTest.java |   28 +
 .../IgniteCommunicationBalanceTest.java         |  339 +++++
 .../communication/IgniteIoTestMessagesTest.java |   95 ++
 .../IgniteVariousConnectionNumberTest.java      |  166 +++
 .../cache/CrossCacheTxRandomOperationsTest.java |   30 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |    6 +-
 ...eAtomicMessageRecovery10ConnectionsTest.java |   28 +
 ...cMessageRecoveryNoPairedConnectionsTest.java |   47 +
 ...acheConnectionRecovery10ConnectionsTest.java |   35 +
 .../distributed/IgniteCacheCreatePutTest.java   |    2 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |   24 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |   17 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 .../TxDeadlockDetectionNoHangsTest.java         |    2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   29 +-
 .../GridServiceProcessorProxySelfTest.java      |    2 +-
 .../util/future/GridFutureAdapterSelfTest.java  |  122 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   32 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   18 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   28 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   82 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    5 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   23 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    9 +-
 ...tionSpiRecoveryFailureDetectionSelfTest.java |    1 +
 ...ationSpiRecoveryNoPairedConnectionsTest.java |   28 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   67 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    9 +-
 .../junits/GridTestKernalContext.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   17 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |    2 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    4 +-
 .../HadoopExternalCommunication.java            |    5 +-
 .../communication/HadoopIpcToNioAdapter.java    |    2 +-
 .../communication/HadoopMarshallerFilter.java   |    6 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    8 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    6 +-
 .../yardstick/cache/CacheEntryEventProbe.java   |    2 +-
 .../yardstick/cache/IgniteIoTestBenchmark.java  |   73 +
 .../io/IgniteIoTestAbstractBenchmark.java       |   61 +
 .../io/IgniteIoTestSendAllBenchmark.java        |   32 +
 .../io/IgniteIoTestSendRandomBenchmark.java     |   35 +
 134 files changed, 5935 insertions(+), 998 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
index ad12297..dd8a72b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
@@ -33,4 +33,4 @@ public class ExampleNodeStartup {
     public static void main(String[] args) throws IgniteException {
         Ignition.start("examples/config/example-ignite.xml");
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
index 82a76b8..b9bae5b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
@@ -105,4 +105,4 @@ public class CachePutGetExample {
         for (Map.Entry<Integer, String> e : vals.entrySet())
             System.out.println("Got entry [key=" + e.getKey() + ", val=" + e.getValue() + ']');
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
index 1014726..814bb2e 100644
--- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
+++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
@@ -50,7 +50,7 @@ object ScalarJvmCloudExample {
             val pool = Executors.newFixedThreadPool(NODES.size)
 
             // Concurrently startup all nodes.
-            NODES.foreach(name => pool.submit(new Runnable {
+            NODES.foreach(name => pool.execute(new Runnable {
                 @impl def run() {
                     // All defaults.
                     val cfg = new IgniteConfiguration

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
index 0f56c73..c03c06e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
@@ -111,6 +111,8 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
 
         Map<String, Object> map = client.getBulk("getKey1", "getKey2");
 
+        info("Map: " + map);
+
         Assert.assertEquals(2, map.size());
 
         Assert.assertEquals("getVal1", map.get("getKey1"));
@@ -443,4 +445,4 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
             return res;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
index c82c73e..9bc4e7f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.rest.protocols.tcp;
 
 import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFinishedFuture;
 import org.apache.ignite.internal.util.nio.GridNioFuture;
@@ -111,6 +112,11 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
     }
 
     /** {@inheritDoc} */
+    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public GridNioFuture<Object> resumeReads() {
         return null;
     }
@@ -131,12 +137,27 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void systemMessage(Object msg) {
+        // No-op.
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/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 d0c0d5e..9650a31 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -477,6 +477,9 @@ public final class IgniteSystemProperties {
     @Deprecated
     public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES";
 
+    /** */
+    public static final String IGNITE_IO_BALANCE_PERIOD = "IGNITE_IO_BALANCE_PERIOD";
+
     /**
      * When set to {@code true} fields are written by BinaryMarshaller in sorted order. Otherwise
      * the natural order is used.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
index 7494e37..d3f381e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
@@ -153,14 +153,14 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
                 buf.add(iter.next());
 
                 if (buf.size() == batchSize) {
-                    exec.submit(new Worker(c, buf, args));
+                    exec.execute(new Worker(c, buf, args));
 
                     buf = new ArrayList<>(batchSize);
                 }
             }
 
             if (!buf.isEmpty())
-                exec.submit(new Worker(c, buf, args));
+                exec.execute(new Worker(c, buf, args));
         }
         catch (RejectedExecutionException ignored) {
             // Because of custom RejectedExecutionHandler.
@@ -330,4 +330,4 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 75145a3..dcd8a80 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -146,7 +146,7 @@ public class IgniteConfiguration {
     public static final int AVAILABLE_PROC_CNT = Runtime.getRuntime().availableProcessors();
 
     /** Default core size of public thread pool. */
-    public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2;
+    public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT);
 
     /** Default keep alive time for public thread pool. */
     @Deprecated
@@ -236,6 +236,12 @@ public class IgniteConfiguration {
     /** Async Callback pool size. */
     private int callbackPoolSize = DFLT_PUBLIC_THREAD_CNT;
 
+    /**
+     * Use striped pool for internal requests processing when possible
+     * (e.g. cache requests per-partition striping).
+     */
+    private int stripedPoolSize = DFLT_PUBLIC_THREAD_CNT;
+
     /** System pool size. */
     private int sysPoolSize = DFLT_SYSTEM_CORE_THREAD_CNT;
 
@@ -553,6 +559,7 @@ public class IgniteConfiguration {
         sndRetryDelay = cfg.getNetworkSendRetryDelay();
         sslCtxFactory = cfg.getSslContextFactory();
         storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
+        stripedPoolSize = cfg.getStripedPoolSize();
         svcCfgs = cfg.getServiceConfiguration();
         sysPoolSize = cfg.getSystemThreadPoolSize();
         timeSrvPortBase = cfg.getTimeServerPortBase();
@@ -712,6 +719,47 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Returns striped pool size that should be used for cache requests
+     * processing.
+     * <p>
+     * If set to non-positive value then requests get processed in system pool.
+     * <p>
+     * Striped pool is better for typical cache operations.
+     *
+     * @return Positive value if striped pool should be initialized
+     *      with configured number of threads (stripes) and used for requests processing
+     *      or non-positive value to process requests in system pool.
+     *
+     * @see #getPublicThreadPoolSize()
+     * @see #getSystemThreadPoolSize()
+     */
+    public int getStripedPoolSize() {
+        return stripedPoolSize;
+    }
+
+    /**
+     * Sets striped pool size that should be used for cache requests
+     * processing.
+     * <p>
+     * If set to non-positive value then requests get processed in system pool.
+     * <p>
+     * Striped pool is better for typical cache operations.
+     *
+     * @param stripedPoolSize Positive value if striped pool should be initialized
+     *      with passed in number of threads (stripes) and used for requests processing
+     *      or non-positive value to process requests in system pool.
+     * @return {@code this} for chaining.
+     *
+     * @see #getPublicThreadPoolSize()
+     * @see #getSystemThreadPoolSize()
+     */
+    public IgniteConfiguration setStripedPoolSize(int stripedPoolSize) {
+        this.stripedPoolSize = stripedPoolSize;
+
+        return this;
+    }
+
+    /**
      * Should return a thread pool size to be used in grid.
      * This executor service will be in charge of processing {@link ComputeJob GridJobs}
      * and user messages sent to node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 715f8a5..68d34ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -181,7 +181,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
                         notificationQueue.add(new T3<>(nodeId, routineId, evt));
 
                         if (!notificationInProgress) {
-                            ctx.getSystemExecutorService().submit(new Runnable() {
+                            ctx.getSystemExecutorService().execute(new Runnable() {
                                 @Override public void run() {
                                     if (!ctx.continuous().lockStopping())
                                         return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
index 804d228..dbfa0b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
@@ -217,7 +217,7 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
 
                                 assert execSvc != null;
 
-                                execSvc.submit(new Runnable() {
+                                execSvc.execute(new Runnable() {
                                     @Override public void run() {
                                         callcc0();
                                     }
@@ -300,4 +300,4 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
     @Override public String toString() {
         return S.toString(GridJobContextImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index ae29223..927944f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.plugin.PluginNotFoundException;
 import org.apache.ignite.plugin.PluginProvider;
@@ -511,6 +512,14 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public ExecutorService getSystemExecutorService();
 
     /**
+     * Executor service that is in charge of processing internal system messages
+     * in stripes (dedicated threads).
+     *
+     * @return Thread pool implementation to be used in grid for internal system messages.
+     */
+    public StripedExecutor getStripedExecutorService();
+
+    /**
      * Executor service that is in charge of processing internal and Visor
      * {@link org.apache.ignite.compute.ComputeJob GridJobs}.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 94c6448..a2ad1b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -83,6 +83,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -300,6 +301,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    protected StripedExecutor stripedExecSvc;
+
+    /** */
+    @GridToStringExclude
     private ExecutorService p2pExecSvc;
 
     /** */
@@ -381,6 +386,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      * @param marshCachePool Marshaller cache pool.
      * @param execSvc Public executor service.
      * @param sysExecSvc System executor service.
+     * @param stripedExecSvc Striped executor.
      * @param p2pExecSvc P2P executor service.
      * @param mgmtExecSvc Management executor service.
      * @param igfsExecSvc IGFS executor service.
@@ -400,6 +406,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService marshCachePool,
         ExecutorService execSvc,
         ExecutorService sysExecSvc,
+        StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
@@ -407,7 +414,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
         IgniteStripedThreadPoolExecutor callbackExecSvc,
-        List<PluginProvider> plugins) throws IgniteCheckedException {
+        List<PluginProvider> plugins
+    ) throws IgniteCheckedException {
         assert grid != null;
         assert cfg != null;
         assert gw != null;
@@ -419,6 +427,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.marshCachePool = marshCachePool;
         this.execSvc = execSvc;
         this.sysExecSvc = sysExecSvc;
+        this.stripedExecSvc = stripedExecSvc;
         this.p2pExecSvc = p2pExecSvc;
         this.mgmtExecSvc = mgmtExecSvc;
         this.igfsExecSvc = igfsExecSvc;
@@ -948,6 +957,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public StripedExecutor getStripedExecutorService() {
+        return stripedExecSvc;
+    }
+
+    /** {@inheritDoc} */
     @Override public ExecutorService getManagementExecutorService() {
         return mgmtExecSvc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
index b040a97..5e8e520 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
@@ -89,4 +89,4 @@ public class GridPerformanceSuggestions {
     @Override public String toString() {
         return S.toString(GridPerformanceSuggestions.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index b5608db..24ddcd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -97,7 +97,10 @@ public enum GridTopic {
     TOPIC_QUERY,
 
     /** */
-    TOPIC_TX;
+    TOPIC_TX,
+
+    /** */
+    TOPIC_IO_TEST;
 
     /** Enum values. */
     private static final GridTopic[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
index b80a755..789556d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal;
 
+import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
@@ -133,6 +134,16 @@ public interface IgniteInternalFuture<R> {
     public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb);
 
     /**
+     * Make a chained future to convert result of this future (when complete) into a new format.
+     * It is guaranteed that done callback will be called only ONCE.
+     *
+     * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result.
+     * @param exec Executor to run callback.
+     * @return Chained future that finishes after this future completes and done callback is called.
+     */
+    public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb, Executor exec);
+
+    /**
      * @return Error value if future has already been completed with error.
      */
     public Throwable error();

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/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 57aab00..7935e06 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
@@ -61,10 +61,10 @@ import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
 import org.apache.ignite.IgniteQueue;
-import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteServices;
@@ -115,7 +115,6 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
-import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
@@ -139,6 +138,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -175,6 +175,7 @@ import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 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;
@@ -182,7 +183,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_OPTIMIZED_MARSHALL
 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;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.IgniteSystemProperties.snapshot;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
@@ -199,7 +199,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS;
@@ -208,11 +207,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JMX_PORT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_ARGS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_PID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LANG_RUNTIME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
@@ -663,6 +663,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param utilityCachePool Utility cache pool.
      * @param execSvc Executor service.
      * @param sysExecSvc System executor service.
+     * @param stripedExecSvc Striped executor.
      * @param p2pExecSvc P2P executor service.
      * @param mgmtExecSvc Management executor service.
      * @param igfsExecSvc IGFS executor service.
@@ -673,11 +674,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings({"CatchGenericClass", "unchecked"})
-    public void start(final IgniteConfiguration cfg,
+    public void start(
+        final IgniteConfiguration cfg,
         ExecutorService utilityCachePool,
         ExecutorService marshCachePool,
         final ExecutorService execSvc,
         final ExecutorService sysExecSvc,
+        final StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
@@ -685,7 +688,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
         IgniteStripedThreadPoolExecutor callbackExecSvc,
-        GridAbsClosure errHnd)
+        GridAbsClosure errHnd
+    )
         throws IgniteCheckedException
     {
         gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
@@ -785,6 +789,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 marshCachePool,
                 execSvc,
                 sysExecSvc,
+                stripedExecSvc,
                 p2pExecSvc,
                 mgmtExecSvc,
                 igfsExecSvc,
@@ -792,7 +797,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 affExecSvc,
                 idxExecSvc,
                 callbackExecSvc,
-                plugins);
+                plugins
+            );
 
             cfg.getMarshaller().setContext(ctx.marshallerContext());
 
@@ -986,24 +992,51 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             starveTask = ctx.timeout().schedule(new Runnable() {
                 /** Last completed task count. */
-                private long lastCompletedCnt;
+                private long lastCompletedCntPub;
+
+                /** Last completed task count. */
+                private long lastCompletedCntSys;
 
                 @Override public void run() {
-                    if (!(execSvc instanceof ThreadPoolExecutor))
-                        return;
+                    if (execSvc instanceof ThreadPoolExecutor) {
+                        ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+                        lastCompletedCntPub = checkPoolStarvation(exec, lastCompletedCntPub, "public");
+                    }
 
-                    ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+                    if (sysExecSvc instanceof ThreadPoolExecutor) {
+                        ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
 
+                        lastCompletedCntSys = checkPoolStarvation(exec, lastCompletedCntSys, "system");
+                    }
+
+                    if (stripedExecSvc != null)
+                        stripedExecSvc.checkStarvation();
+                }
+
+                /**
+                 * @param exec Thread pool executor to check.
+                 * @param lastCompletedCnt Last completed tasks count.
+                 * @param pool Pool name for message.
+                 * @return Current completed tasks count.
+                 */
+                private long checkPoolStarvation(
+                    ThreadPoolExecutor exec,
+                    long lastCompletedCnt,
+                    String pool
+                ) {
                     long completedCnt = exec.getCompletedTaskCount();
 
                     // If all threads are active and no task has completed since last time and there is
                     // at least one waiting request, then it is possible starvation.
                     if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
                         !exec.getQueue().isEmpty())
-                        LT.warn(log, "Possible thread pool starvation detected (no task completed in last " +
-                            interval + "ms, is executorService pool size large enough?)");
+                        LT.warn(
+                            log,
+                            "Possible thread pool starvation detected (no task completed in last " +
+                                interval + "ms, is " + pool + " thread pool size large enough?)");
 
-                    lastCompletedCnt = completedCnt;
+                    return completedCnt;
                 }
             }, interval, interval);
         }
@@ -1128,6 +1161,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             }, longOpDumpTimeout, longOpDumpTimeout);
         }
 
+        ctx.performance().add("Disable assertions (remove '-ea' from JVM options)", !U.assertionsEnabled());
+
         ctx.performance().logSuggestions(log, gridName);
 
         U.quietAndInfo(log, "To start Console Management & Monitoring run ignitevisorcmd.{sh|bat}");
@@ -3509,6 +3544,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param node Node.
+     * @param payload Message payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+        return ctx.io().sendIoTest(node, payload, procFromNioThread);
+    }
+
+    /**
+     * @param nodes Nodes.
+     * @param payload Message payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+        return ctx.io().sendIoTest(nodes, payload, procFromNioThread);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index b3a9eec..f32a753 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
 import org.apache.ignite.internal.util.typedef.CA;
 import org.apache.ignite.internal.util.typedef.F;
@@ -1459,6 +1460,9 @@ public class IgnitionEx {
         /** System executor service. */
         private ThreadPoolExecutor sysExecSvc;
 
+        /** */
+        private StripedExecutor stripedExecSvc;
+
         /** Management executor service. */
         private ThreadPoolExecutor mgmtExecSvc;
 
@@ -1652,8 +1656,6 @@ public class IgnitionEx {
 
             execSvc.allowCoreThreadTimeOut(true);
 
-            // Note that since we use 'LinkedBlockingQueue', number of
-            // maximum threads has no effect.
             validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
 
             sysExecSvc = new IgniteThreadPoolExecutor(
@@ -1666,6 +1668,9 @@ public class IgnitionEx {
 
             sysExecSvc.allowCoreThreadTimeOut(true);
 
+            if (cfg.getStripedPoolSize() > 0)
+                stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getGridName(), "sys", log);
+
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as management pool may
@@ -1791,13 +1796,26 @@ public class IgnitionEx {
                 // Init here to make grid available to lifecycle listeners.
                 grid = grid0;
 
-                grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc,
-                    igfsExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
+                grid0.start(
+                    myCfg,
+                    utilityCacheExecSvc,
+                    marshCacheExecSvc,
+                    execSvc,
+                    sysExecSvc,
+                    stripedExecSvc,
+                    p2pExecSvc,
+                    mgmtExecSvc,
+                    igfsExecSvc,
+                    restExecSvc,
+                    affExecSvc,
+                    idxExecSvc,
+                    callbackExecSvc,
                     new CA() {
                         @Override public void apply() {
                             startLatch.countDown();
                         }
-                    });
+                    }
+                );
 
                 state = STARTED;
 
@@ -2415,6 +2433,10 @@ public class IgnitionEx {
 
             sysExecSvc = null;
 
+            U.shutdownNow(getClass(), stripedExecSvc, log);
+
+            stripedExecSvc = null;
+
             U.shutdownNow(getClass(), mgmtExecSvc, log);
 
             mgmtExecSvc = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 6ea7c22..12baee0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -200,6 +200,7 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
                     .socketSendBufferSize(0)
                     .idleTimeout(Long.MAX_VALUE)
                     .gridName(routerClient ? "routerClient" : "gridClient")
+                    .serverName("tcp-client")
                     .daemon(cfg.isDaemon())
                     .build();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
index 06a4929..3566830 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
@@ -258,6 +258,7 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif
                 .logger(log)
                 .selectorCount(Runtime.getRuntime().availableProcessors())
                 .gridName(gridName)
+                .serverName("router")
                 .tcpNoDelay(tcpNoDelay)
                 .directBuffer(false)
                 .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 3df29cf..7ef7bc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -26,15 +26,17 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -44,6 +46,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteDeploymentCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.direct.DirectMessageReader;
 import org.apache.ignite.internal.direct.DirectMessageWriter;
 import org.apache.ignite.internal.managers.GridManagerAdapter;
@@ -55,6 +58,8 @@ import org.apache.ignite.internal.processors.pool.PoolProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -83,6 +88,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER;
+import static org.apache.ignite.internal.GridTopic.TOPIC_IO_TEST;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
@@ -176,6 +182,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /** Stopping flag. */
     private boolean stopping;
 
+    /** */
+    private final AtomicReference<ConcurrentHashMap<Long, IoTestFuture>> ioTestMap = new AtomicReference<>();
+
+    /** */
+    private final AtomicLong ioTestId = new AtomicLong();
+
     /**
      * @param ctx Grid kernal context.
      */
@@ -297,6 +309,114 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         if (log.isDebugEnabled())
             log.debug(startInfo());
+
+        addMessageListener(GridTopic.TOPIC_IO_TEST, new GridMessageListener() {
+            @Override public void onMessage(UUID nodeId, Object msg) {
+                ClusterNode node = ctx.discovery().node(nodeId);
+
+                if (node == null)
+                    return;
+
+                IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg;
+
+                if (msg0.request()) {
+                    IgniteIoTestMessage res = new IgniteIoTestMessage(msg0.id(), false, null);
+
+                    res.flags(msg0.flags());
+
+                    try {
+                        send(node, GridTopic.TOPIC_IO_TEST, res, GridIoPolicy.SYSTEM_POOL);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send IO test response [msg=" + msg0 + "]", e);
+                    }
+                }
+                else {
+                    IoTestFuture fut = ioTestMap().get(msg0.id());
+
+                    if (fut == null)
+                        U.warn(log, "Failed to find IO test future [msg=" + msg0 + ']');
+                    else
+                        fut.onResponse();
+                }
+            }
+        });
+    }
+
+    /**
+     * @param nodes Nodes.
+     * @param payload Payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+        long id = ioTestId.getAndIncrement();
+
+        IoTestFuture fut = new IoTestFuture(id, nodes.size());
+
+        IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+        msg.processFromNioThread(procFromNioThread);
+
+        ioTestMap().put(id, fut);
+
+        for (int i = 0; i < nodes.size(); i++) {
+            ClusterNode node = nodes.get(i);
+
+            try {
+                send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+            }
+            catch (IgniteCheckedException e) {
+                ioTestMap().remove(msg.id());
+
+                return new GridFinishedFuture(e);
+            }
+        }
+
+        return fut;
+    }
+
+    /**
+     * @param node Node.
+     * @param payload Payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+        long id = ioTestId.getAndIncrement();
+
+        IoTestFuture fut = new IoTestFuture(id, 1);
+
+        IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+        msg.processFromNioThread(procFromNioThread);
+
+        ioTestMap().put(id, fut);
+
+        try {
+            send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            ioTestMap().remove(msg.id());
+
+            return new GridFinishedFuture(e);
+        }
+
+        return fut;
+    }
+
+    /**
+     * @return IO test futures map.
+     */
+    private ConcurrentHashMap<Long, IoTestFuture> ioTestMap() {
+        ConcurrentHashMap<Long, IoTestFuture> map = ioTestMap.get();
+
+        if (map == null) {
+            if (!ioTestMap.compareAndSet(null, map = new ConcurrentHashMap<>()))
+                map = ioTestMap.get();
+        }
+
+        return map;
     }
 
     /** {@inheritDoc} */
@@ -514,16 +634,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 return;
             }
 
-            // Check discovery.
-            ClusterNode node = ctx.discovery().node(nodeId);
-
-            if (node == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Ignoring message from dead node [senderId=" + nodeId + ", msg=" + msg + ']');
-
-                return; // We can't receive messages from non-discovered ones.
-            }
-
             if (msg.topic() == null) {
                 int topicOrd = msg.topicOrdinal();
 
@@ -678,8 +788,31 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                     msgC.run();
                 }
             }
+
+            @Override public String toString() {
+                return "Message closure [msg=" + msg + ']';
+            }
         };
 
+        if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) {
+            IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message();
+
+            if (msg0.processFromNioThread()) {
+                c.run();
+
+                return;
+            }
+        }
+
+        if (ctx.config().getStripedPoolSize() > 0 &&
+            plc == GridIoPolicy.SYSTEM_POOL &&
+            msg.partition() != Integer.MIN_VALUE
+            ) {
+            ctx.getStripedExecutorService().execute(msg.partition(), c);
+
+            return;
+        }
+
         try {
             pools.poolForPolicy(plc).execute(c);
         }
@@ -2460,4 +2593,56 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             return S.toString(DelayedMessage.class, this, super.toString());
         }
     }
+
+    /**
+     *
+     */
+    private class IoTestFuture extends GridFutureAdapter<Object> {
+        /** */
+        private final long id;
+
+        /** */
+        private int cntr;
+
+        /**
+         * @param id ID.
+         * @param cntr Counter.
+         */
+        IoTestFuture(long id, int cntr) {
+            assert cntr > 0 : cntr;
+
+            this.id = id;
+            this.cntr = cntr;
+        }
+
+        /**
+         *
+         */
+        void onResponse() {
+            boolean complete;
+
+            synchronized (this) {
+                complete = --cntr == 0;
+            }
+
+            if (complete)
+                onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+            if (super.onDone(res, err)) {
+                ioTestMap().remove(id);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(IoTestFuture.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index b28ced2..b1a26e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.communication;
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -321,6 +322,18 @@ public class GridIoMessage implements Message {
         return 7;
     }
 
+    /**
+     * Get single partition for this message (if applicable).
+     *
+     * @return Partition ID.
+     */
+    public int partition() {
+        if (msg instanceof GridCacheMessage)
+            return ((GridCacheMessage)msg).partition();
+        else
+            return Integer.MIN_VALUE;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridIoMessage.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 504e683..b1fe910 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -171,6 +171,16 @@ public class GridIoMessageFactory implements MessageFactory {
         Message msg = null;
 
         switch (type) {
+            case -44:
+                msg = new TcpCommunicationSpi.HandshakeMessage2();
+
+                break;
+
+            case -43:
+                msg = new IgniteIoTestMessage();
+
+                break;
+
             case -42:
                 msg = new HadoopDirectShuffleMessage();
 
@@ -816,7 +826,7 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
-            // [-3..119] [124..127] [-36]- this
+            // [-3..119] [124..127] [-36..-44]- this
             // [120..123] - DR
             // [-4..-22, -30..-35] - SQL
             default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
new file mode 100644
index 0000000..77aaa09
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+import java.nio.ByteBuffer;
+
+/**
+ *
+ */
+public class IgniteIoTestMessage implements Message {
+    /** */
+    private static byte FLAG_PROC_FROM_NIO = 1;
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long id;
+
+    /** */
+    private byte flags;
+
+    /** */
+    private boolean req;
+
+    /** */
+    private byte payload[];
+
+    /**
+     *
+     */
+    IgniteIoTestMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param id Message ID.
+     * @param req Request flag.
+     * @param payload Payload.
+     */
+    IgniteIoTestMessage(long id, boolean req, byte[] payload) {
+        this.id = id;
+        this.req = req;
+        this.payload = payload;
+    }
+
+    /**
+     * @return {@code True} if message should be processed from NIO thread
+     * (otherwise message is submitted to system pool).
+     */
+    boolean processFromNioThread() {
+        return isFlag(FLAG_PROC_FROM_NIO);
+    }
+
+    /**
+     * @param procFromNioThread {@code True} if message should be processed from NIO thread.
+     */
+    void processFromNioThread(boolean procFromNioThread) {
+        setFlag(procFromNioThread, FLAG_PROC_FROM_NIO);
+    }
+
+    /**
+     * @param flags Flags.
+     */
+    public void flags(byte flags) {
+        this.flags = flags;
+    }
+
+    /**
+     * @return Flags.
+     */
+    public byte flags() {
+        return flags;
+    }
+
+    /**
+     * Sets flag mask.
+     *
+     * @param flag Set or clear.
+     * @param mask Mask.
+     */
+    private void setFlag(boolean flag, int mask) {
+        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+    }
+
+    /**
+     * Reads flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
+     */
+    private boolean isFlag(int mask) {
+        return (flags & mask) != 0;
+    }
+
+    /**
+     * @return {@code true} if this is request.
+     */
+    public boolean request() {
+        return req;
+    }
+
+    /**
+     * @return ID.
+     */
+    public long id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("id", id))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeByteArray("payload", payload))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeBoolean("req", req))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                id = reader.readLong("id");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                payload = reader.readByteArray("payload");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                req = reader.readBoolean("req");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(IgniteIoTestMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -43;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteIoTestMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2e24e67..a8d9f1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -288,6 +288,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** Asynchronous operations limit semaphore. */
     private Semaphore asyncOpsSem;
 
+    /** */
+    protected volatile boolean asyncToggled;
+
     /** {@inheritDoc} */
     @Override public String name() {
         return cacheCfg.getName();
@@ -364,6 +367,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Toggles async flag if someone calls {@code withAsync()}
+     * on proxy and since that we have to properly handle all cache
+     * operations (sync and async) to put them in proper sequence.
+     *
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-4393
+     */
+    void toggleAsync() {
+        if (!asyncToggled)
+            asyncToggled = true;
+    }
+
+    /**
      * Prints memory stats.
      */
     public void printMemoryStats() {
@@ -1134,7 +1149,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 execSvc = Executors.newFixedThreadPool(jobs.size() - 1);
 
                 for (int i = 1; i < jobs.size(); i++)
-                    execSvc.submit(jobs.get(i));
+                    execSvc.execute(jobs.get(i));
             }
 
             try {
@@ -2534,6 +2549,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Put future.
      */
     public IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+        A.notNull(key, "key", val, "val");
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2554,8 +2571,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     public IgniteInternalFuture<Boolean> putAsync0(final K key, final V val,
         @Nullable final CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         if (keyCheck)
             validateCacheKey(key);
 
@@ -4592,6 +4607,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Failed future if waiting was interrupted.
      */
     @Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
+        if (!asyncToggled)
+            return null;
+
         try {
             if (asyncOpsSem != null)
                 asyncOpsSem.acquire();
@@ -4610,7 +4628,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Releases asynchronous operations permit, if limited.
      */
     protected void asyncOpRelease() {
-        if (asyncOpsSem != null)
+        if (asyncOpsSem != null && asyncToggled)
             asyncOpsSem.release();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 71f99d3..0646d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -129,6 +129,13 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * @return Partition ID this message is targeted to or {@code -1} if it cannot be determined.
+     */
+    public int partition() {
+        return -1;
+    }
+
+    /**
      * If class loading error occurred during unmarshalling and {@link #ignoreClassErrors()} is
      * set to {@code true}, then the error will be passed into this method.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 90e428c..3178203 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -110,6 +110,41 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.REA
  * Cache utility methods.
  */
 public class GridCacheUtils {
+    /** Cheat cache ID for debugging and benchmarking purposes. */
+    public static final int cheatCacheId;
+
+    /*
+     *
+     */
+    static {
+        String cheatCache = System.getProperty("CHEAT_CACHE");
+
+        if (cheatCache != null) {
+            cheatCacheId = cheatCache.hashCode();
+
+            if (cheatCacheId == 0)
+                throw new RuntimeException();
+
+            System.out.println(">>> Cheat cache ID [id=" + cheatCacheId + ", name=" + cheatCache + ']');
+        }
+        else
+            cheatCacheId = 0;
+    }
+
+    /**
+     * Quickly checks if passed in cache ID is a "cheat cache ID" set by -DCHEAT_CACHE=user_cache_name
+     * and resolved in static block above.
+     *
+     * FOR DEBUGGING AND TESTING PURPOSES!
+     *
+     * @param id Cache ID to check.
+     * @return {@code True} if this is cheat cache ID.
+     */
+    @Deprecated
+    public static boolean cheatCache(int id) {
+        return cheatCacheId != 0 && id == cheatCacheId;
+    }
+
     /**  Hadoop syste cache name. */
     public static final String SYS_CACHE_HADOOP_MR = "ignite-hadoop-mr-sys-cache";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f87fa1d..b9e6e82 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -334,6 +334,14 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withAsync() {
+        if (delegate instanceof GridCacheAdapter)
+            ((GridCacheAdapter)delegate).toggleAsync();
+
+        return super.withAsync();
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withSkipStore() {
         return skipStore();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index 9639a9a..a671296 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -310,6 +310,11 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         return keys;
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /**
      * @return Max lock wait time.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index 109d665..c5cf332 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,6 +86,12 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        // TODO https://issues.apache.org/jira/browse/IGNITE-4371
+        return Integer.MIN_VALUE;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
index df6acdd..5d70ec1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
@@ -89,6 +89,11 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
         partIds.add(key.partition());
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /** {@inheritDoc}
      * @param ctx*/
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 35e6267..519d0fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -218,7 +218,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     @Override public void onKernalStart() throws IgniteCheckedException {
         super.onKernalStart();
 
-        preldr.onKernalStart();
+        if (preldr != null)
+            preldr.onKernalStart();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
index 1e92b54..63e3309 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectCollection;
@@ -31,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
@@ -57,7 +57,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
     /** Invalid partitions. */
     @GridToStringInclude
     @GridDirectCollection(int.class)
-    private Collection<Integer> invalidParts = new GridLeanSet<>();
+    private Collection<Integer> invalidParts;
 
     /** Preload entries. */
     @GridDirectCollection(GridCacheEntryInfo.class)
@@ -127,6 +127,9 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
      * @param part Invalid partition.
      */
     public void addInvalidPartition(int part) {
+        if (invalidParts == null)
+            invalidParts = new HashSet<>();
+
         invalidParts.add(part);
     }
 
@@ -134,7 +137,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
      * @return Invalid partitions.
      */
     public Collection<Integer> invalidPartitions() {
-        return invalidParts;
+        return invalidParts == null ? Collections.<Integer>emptySet() : invalidParts;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 940c74e..0e60ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -613,8 +613,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAsync0(
             key,
             val,
@@ -814,6 +812,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     @SuppressWarnings("unchecked")
     protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
+        if (!asyncToggled)
+            return op.apply();
+
         IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
index a03d948..0af7cf5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
@@ -235,6 +235,11 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
+    /** {@inheritDoc} */
     @Override public int partitionId(int idx) {
         assert idx == 0 : idx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index f2fbb0e..1854e52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -487,6 +487,11 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
+    /** {@inheritDoc} */
     @Override public long conflictExpireTime(int idx) {
         if (conflictExpireTimes != null) {
             assert idx >= 0 && idx < conflictExpireTimes.size();


[04/50] [abbrv] ignite git commit: IGNITE-4277: Hadoop: implemented "partially raw" comparator. This closes #1345.

Posted by vo...@apache.org.
IGNITE-4277: Hadoop: implemented "partially raw" comparator. This closes #1345.


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

Branch: refs/heads/ignite-2.0
Commit: c1ddf21fd627c76a8b7e0d81ad43480b1f1e204d
Parents: 30b869d
Author: devozerov <vo...@gridgain.com>
Authored: Thu Dec 15 11:58:28 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:41 2016 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    |   1 +
 .../processors/hadoop/HadoopJobProperty.java    |   6 +-
 .../processors/hadoop/HadoopTaskContext.java    |   8 ++
 .../io/PartiallyOffheapRawComparatorEx.java     |  33 +++++
 .../hadoop/io/PartiallyRawComparator.java       |  33 +++++
 .../org/apache/ignite/hadoop/io/RawMemory.java  |  86 ++++++++++++
 .../hadoop/io/TextPartiallyRawComparator.java   | 115 ++++++++++++++++
 .../apache/ignite/hadoop/io/package-info.java   |  22 ++++
 ...DelegatingPartiallyOffheapRawComparator.java |  54 ++++++++
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |  21 +++
 .../processors/hadoop/io/OffheapRawMemory.java  | 131 +++++++++++++++++++
 .../shuffle/collections/HadoopSkipList.java     |  14 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |   7 +
 .../collections/HadoopAbstractMapTest.java      |   6 +
 14 files changed, 535 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index f6c2fa9..81c1405 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -372,6 +372,7 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
             // We use "contains" instead of "equals" to handle subclasses properly.
             if (clsName.contains("org.apache.ignite.hadoop.fs.v1.IgniteHadoopFileSystem") ||
                 clsName.contains("org.apache.ignite.hadoop.fs.v2.IgniteHadoopFileSystem") ||
+                clsName.contains("org.apache.ignite.hadoop.io.TextPartialRawComparator") ||
                 clsName.contains("org.apache.ignite.hadoop.mapreduce.IgniteHadoopClientProtocolProvider"))
                 return true;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 9e1dede..4122eef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -64,6 +64,11 @@ public enum HadoopJobProperty {
     JOB_SHARED_CLASSLOADER("ignite.job.shared.classloader"),
 
     /**
+     * Fully qualified name of partially-raw comparator which should be used on sorting phase.
+     */
+    JOB_PARTIAL_RAW_COMPARATOR("ignite.job.partial.raw.comparator"),
+
+    /**
      * Size in bytes of single memory page which will be allocated for data structures in shuffle.
      * <p>
      * By default is {@code 32 * 1024}.
@@ -112,7 +117,6 @@ public enum HadoopJobProperty {
      */
     SHUFFLE_JOB_THROTTLE("ignite.shuffle.job.throttle");
 
-
     /** Property name. */
     private final String propName;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
index ecb9f26..dddd017 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskContext.java
@@ -22,6 +22,7 @@ import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
 
 /**
  * Task context.
@@ -157,6 +158,13 @@ public abstract class HadoopTaskContext {
     public abstract Comparator<Object> sortComparator();
 
     /**
+     * Get semi-raw sorting comparator.
+     *
+     * @return Semi-raw sorting comparator.
+     */
+    public abstract PartiallyOffheapRawComparatorEx<Object> partialRawSortComparator();
+
+    /**
      * Gets comparator for grouping on combine or reduce operation.
      *
      * @return Comparator.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/io/PartiallyOffheapRawComparatorEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/io/PartiallyOffheapRawComparatorEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/io/PartiallyOffheapRawComparatorEx.java
new file mode 100644
index 0000000..157609e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/io/PartiallyOffheapRawComparatorEx.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.io;
+
+/**
+ * Special version of raw comparator allowing direct access to the underlying memory.
+ */
+public interface PartiallyOffheapRawComparatorEx<T> {
+    /**
+     * Perform compare.
+     *
+     * @param val1 First value.
+     * @param val2Ptr Pointer to the second value data.
+     * @param val2Len Length of the second value data.
+     * @return Result.
+     */
+    int compare(T val1, long val2Ptr, int val2Len);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/PartiallyRawComparator.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/PartiallyRawComparator.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/PartiallyRawComparator.java
new file mode 100644
index 0000000..b9a4505
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/PartiallyRawComparator.java
@@ -0,0 +1,33 @@
+/*
+ * 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.hadoop.io;
+
+/**
+ * Partially raw comparator. Compares one deserialized value with serialized value.
+ */
+public interface PartiallyRawComparator<T> {
+    /**
+     * Do compare.
+     *
+     * @param val1 First value (deserialized).
+     * @param val2Buf Second value (serialized).
+     * @return A negative integer, zero, or a positive integer as this object is less than, equal to, or greater
+     *     than the specified object.
+     */
+    int compare(T val1, RawMemory val2Buf);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/RawMemory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/RawMemory.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/RawMemory.java
new file mode 100644
index 0000000..8dcaf83
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/RawMemory.java
@@ -0,0 +1,86 @@
+/*
+ * 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.hadoop.io;
+
+/**
+ * Memory abstraction for raw comparison.
+ */
+public interface RawMemory {
+    /**
+     * Get byte value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    byte get(int idx);
+
+    /**
+     * Get short value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    short getShort(int idx);
+
+    /**
+     * Get char value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    char getChar(int idx);
+
+    /**
+     * Get int value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    int getInt(int idx);
+
+    /**
+     * Get long value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    long getLong(int idx);
+
+    /**
+     * Get float value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    float getFloat(int idx);
+
+    /**
+     * Get double value at the given index.
+     *
+     * @param idx Index.
+     * @return Value.
+     */
+    double getDouble(int idx);
+
+    /**
+     * Get length.
+     *
+     * @return Length.
+     */
+    int length();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
new file mode 100644
index 0000000..a2bc3d4
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/TextPartiallyRawComparator.java
@@ -0,0 +1,115 @@
+/*
+ * 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.hadoop.io;
+
+import com.google.common.primitives.Longs;
+import com.google.common.primitives.UnsignedBytes;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.WritableUtils;
+import org.apache.ignite.internal.processors.hadoop.io.OffheapRawMemory;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
+import org.apache.ignite.internal.util.GridUnsafe;
+
+/**
+ * Partial raw comparator for {@link Text} data type.
+ * <p>
+ * Implementation is borrowed from {@code org.apache.hadoop.io.FastByteComparisons} and adopted to Ignite
+ * infrastructure.
+ */
+public class TextPartiallyRawComparator implements PartiallyRawComparator<Text>, PartiallyOffheapRawComparatorEx<Text> {
+    /** {@inheritDoc} */
+    @Override public int compare(Text val1, RawMemory val2Buf) {
+        if (val2Buf instanceof OffheapRawMemory) {
+            OffheapRawMemory val2Buf0 = (OffheapRawMemory)val2Buf;
+
+            return compare(val1, val2Buf0.pointer(), val2Buf0.length());
+        }
+        else
+            throw new UnsupportedOperationException("Text can be compared only with offheap memory.");
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(Text val1, long val2Ptr, int val2Len) {
+        int len2 = WritableUtils.decodeVIntSize(GridUnsafe.getByte(val2Ptr));
+
+        return compareBytes(val1.getBytes(), val1.getLength(), val2Ptr + len2, val2Len - len2);
+    }
+
+    /**
+     * Internal comparison routine.
+     *
+     * @param buf1 Bytes 1.
+     * @param len1 Length 1.
+     * @param ptr2 Pointer 2.
+     * @param len2 Length 2.
+     * @return Result.
+     */
+    @SuppressWarnings("SuspiciousNameCombination")
+    private static int compareBytes(byte[] buf1, int len1, long ptr2, int len2) {
+        int minLength = Math.min(len1, len2);
+
+        int minWords = minLength / Longs.BYTES;
+
+        for (int i = 0; i < minWords * Longs.BYTES; i += Longs.BYTES) {
+            long lw = GridUnsafe.getLong(buf1, GridUnsafe.BYTE_ARR_OFF + i);
+            long rw = GridUnsafe.getLong(ptr2 + i);
+
+            long diff = lw ^ rw;
+
+            if (diff != 0) {
+                if (GridUnsafe.BIG_ENDIAN)
+                    return (lw + Long.MIN_VALUE) < (rw + Long.MIN_VALUE) ? -1 : 1;
+
+                // Use binary search
+                int n = 0;
+                int y;
+                int x = (int) diff;
+
+                if (x == 0) {
+                    x = (int) (diff >>> 32);
+
+                    n = 32;
+                }
+
+                y = x << 16;
+
+                if (y == 0)
+                    n += 16;
+                else
+                    x = y;
+
+                y = x << 8;
+
+                if (y == 0)
+                    n += 8;
+
+                return (int) (((lw >>> n) & 0xFFL) - ((rw >>> n) & 0xFFL));
+            }
+        }
+
+        // The epilogue to cover the last (minLength % 8) elements.
+        for (int i = minWords * Longs.BYTES; i < minLength; i++) {
+            int res = UnsignedBytes.compare(buf1[i], GridUnsafe.getByte(ptr2 + i));
+
+            if (res != 0)
+                return res;
+        }
+
+        return len1 - len2;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/package-info.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/package-info.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/package-info.java
new file mode 100644
index 0000000..0d1f7b9
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/io/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * <!-- Package description. -->
+ * Contains <b>Hadoop Accelerator</b> API for input-output operations.
+ */
+package org.apache.ignite.hadoop.io;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2DelegatingPartiallyOffheapRawComparator.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2DelegatingPartiallyOffheapRawComparator.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2DelegatingPartiallyOffheapRawComparator.java
new file mode 100644
index 0000000..e6d369e
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2DelegatingPartiallyOffheapRawComparator.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.impl.v2;
+
+import org.apache.ignite.hadoop.io.PartiallyRawComparator;
+import org.apache.ignite.internal.processors.hadoop.io.OffheapRawMemory;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
+
+/**
+ * Delegating partial raw comparator.
+ */
+public class HadoopV2DelegatingPartiallyOffheapRawComparator<T> implements PartiallyOffheapRawComparatorEx<T> {
+    /** Target comparator. */
+    private final PartiallyRawComparator<T> target;
+
+    /** Memory. */
+    private OffheapRawMemory mem;
+
+    /**
+     * Constructor.
+     *
+     * @param target Target.
+     */
+    public HadoopV2DelegatingPartiallyOffheapRawComparator(PartiallyRawComparator<T> target) {
+        assert target != null;
+
+        this.target = target;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int compare(T val1, long val2Ptr, int val2Len) {
+        if (mem == null)
+            mem = new OffheapRawMemory(val2Ptr, val2Len);
+        else
+            mem.update(val2Ptr, val2Len);
+
+        return target.compare(val1, mem);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index d444f2b..42bbec5 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -38,13 +38,16 @@ import org.apache.hadoop.mapreduce.JobSubmissionFiles;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.TaskType;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.hadoop.io.PartiallyRawComparator;
 import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopCommonUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopExternalSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
 import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopSplitWrapper;
@@ -62,6 +65,7 @@ import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1MapTask;
 import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1Partitioner;
 import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1ReduceTask;
 import org.apache.ignite.internal.processors.hadoop.impl.v1.HadoopV1SetupTask;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
 import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
@@ -421,11 +425,28 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public Comparator<Object> sortComparator() {
         return (Comparator<Object>)jobCtx.getSortComparator();
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public PartiallyOffheapRawComparatorEx<Object> partialRawSortComparator() {
+        Class cls = jobCtx.getJobConf().getClass(HadoopJobProperty.JOB_PARTIAL_RAW_COMPARATOR.propertyName(), null);
+
+        if (cls == null)
+            return null;
+
+        Object res = ReflectionUtils.newInstance(cls, jobConf());
+
+        if (res instanceof PartiallyOffheapRawComparatorEx)
+            return (PartiallyOffheapRawComparatorEx)res;
+        else
+            return new HadoopV2DelegatingPartiallyOffheapRawComparator<>((PartiallyRawComparator)res);
+    }
+
+    /** {@inheritDoc} */
     @Override public Comparator<Object> groupComparator() {
         Comparator<?> res;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/io/OffheapRawMemory.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/io/OffheapRawMemory.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/io/OffheapRawMemory.java
new file mode 100644
index 0000000..564f92c
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/io/OffheapRawMemory.java
@@ -0,0 +1,131 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.io;
+
+import org.apache.ignite.hadoop.io.RawMemory;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Offheap-based memory.
+ */
+public class OffheapRawMemory implements RawMemory {
+    /** Pointer. */
+    private long ptr;
+
+    /** Length. */
+    private int len;
+
+    /**
+     * Constructor.
+     *
+     * @param ptr Pointer.
+     * @param len Length.
+     */
+    public OffheapRawMemory(long ptr, int len) {
+        update(ptr, len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte get(int idx) {
+        ensure(idx, 1);
+
+        return GridUnsafe.getByte(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short getShort(int idx) {
+        ensure(idx, 2);
+
+        return GridUnsafe.getShort(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public char getChar(int idx) {
+        ensure(idx, 2);
+
+        return GridUnsafe.getChar(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getInt(int idx) {
+        ensure(idx, 4);
+
+        return GridUnsafe.getInt(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getLong(int idx) {
+        ensure(idx, 8);
+
+        return GridUnsafe.getLong(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public float getFloat(int idx) {
+        ensure(idx, 4);
+
+        return GridUnsafe.getFloat(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public double getDouble(int idx) {
+        ensure(idx, 8);
+
+        return GridUnsafe.getDouble(ptr + idx);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int length() {
+        return len;
+    }
+
+    /**
+     * @return Raw pointer.
+     */
+    public long pointer() {
+        return ptr;
+    }
+
+    /**
+     * Update pointer and length.
+     *
+     * @param ptr Pointer.
+     * @param len Length.
+     */
+    public void update(long ptr, int len) {
+        this.ptr = ptr;
+        this.len = len;
+    }
+
+    /**
+     * Ensure that the given number of bytes are available for read. Throw an exception otherwise.
+     *
+     * @param idx Index.
+     * @param cnt Count.
+     */
+    private void ensure(int idx, int cnt) {
+        if (idx < 0 || idx + cnt - 1 >= len)
+            throw new IndexOutOfBoundsException("Illegal index [len=" + len + ", idx=" + idx + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(OffheapRawMemory.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
index 7db88bc..f300a18 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopSkipList.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
 import org.apache.ignite.internal.util.GridLongList;
 import org.apache.ignite.internal.util.GridRandom;
 import org.apache.ignite.internal.util.offheap.unsafe.GridUnsafeMemory;
@@ -280,6 +281,9 @@ public class HadoopSkipList extends HadoopMultimapBase {
         private final Comparator<Object> cmp;
 
         /** */
+        private final PartiallyOffheapRawComparatorEx<Object> partialRawCmp;
+
+        /** */
         private final Random rnd = new GridRandom();
 
         /** */
@@ -298,6 +302,7 @@ public class HadoopSkipList extends HadoopMultimapBase {
             keyReader = new Reader(keySer);
 
             cmp = ctx.sortComparator();
+            partialRawCmp = ctx.partialRawSortComparator();
         }
 
         /** {@inheritDoc} */
@@ -475,7 +480,14 @@ public class HadoopSkipList extends HadoopMultimapBase {
         private int cmp(Object key, long meta) {
             assert meta != 0;
 
-            return cmp.compare(key, keyReader.readKey(meta));
+            if (partialRawCmp != null) {
+                long keyPtr = key(meta);
+                int keySize = keySize(keyPtr);
+
+                return partialRawCmp.compare(key, keyPtr + 4, keySize);
+            }
+            else
+                return cmp.compare(key, keyReader.readKey(meta));
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
index 0cc9564..a016506 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
@@ -41,8 +41,10 @@ import org.apache.hadoop.util.ToolRunner;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.configuration.HadoopConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.hadoop.io.TextPartiallyRawComparator;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
 
 import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.createJobInfo;
 
@@ -161,6 +163,11 @@ public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
         jobConf.set("mapred.min.split.size", String.valueOf(splitSize));
         jobConf.set("mapred.max.split.size", String.valueOf(splitSize));
 
+        jobConf.setBoolean(HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT.propertyName(), true);
+
+        jobConf.set(HadoopJobProperty.JOB_PARTIAL_RAW_COMPARATOR.propertyName(),
+            TextPartiallyRawComparator.class.getName());
+
         Job job = setupConfig(jobConf);
 
         HadoopJobId jobId = new HadoopJobId(UUID.randomUUID(), 1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1ddf21f/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
index 9d1fd4f..1f8978d 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/shuffle/collections/HadoopAbstractMapTest.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounter;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopWritableSerialization;
+import org.apache.ignite.internal.processors.hadoop.io.PartiallyOffheapRawComparatorEx;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.Nullable;
 
@@ -84,6 +85,11 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public PartiallyOffheapRawComparatorEx<Object> partialRawSortComparator() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public Comparator<Object> groupComparator() {
             return ComparableComparator.getInstance();


[26/50] [abbrv] ignite git commit: GridTopic.TOPIC_HADOOP_MSG: new topic should be added at the end for compatibility with previous releases.

Posted by vo...@apache.org.
GridTopic.TOPIC_HADOOP_MSG: new topic should be added at the end for compatibility with previous releases.


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

Branch: refs/heads/ignite-2.0
Commit: 8372e69dd01972a01dbd59689475f2d45c2b6d94
Parents: ce46c10
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 10:30:05 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 10:30:05 2016 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/GridTopic.java  | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8372e69d/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index 24ddcd2..2962540 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -91,16 +91,16 @@ public enum GridTopic {
     TOPIC_HADOOP,
 
     /** */
-    TOPIC_HADOOP_MSG,
-
-    /** */
     TOPIC_QUERY,
 
     /** */
     TOPIC_TX,
 
     /** */
-    TOPIC_IO_TEST;
+    TOPIC_IO_TEST,
+
+    /** */
+    TOPIC_HADOOP_MSG;
 
     /** Enum values. */
     private static final GridTopic[] VALS = values();


[11/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c8e2e0b..bc1f173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -37,6 +37,7 @@ import java.nio.channels.WritableByteChannel;
 import java.nio.channels.spi.SelectorProvider;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Deque;
 import java.util.Iterator;
 import java.util.List;
@@ -44,6 +45,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -63,6 +66,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -86,11 +90,14 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPER
  *
  */
 public class GridNioServer<T> {
+    /** */
+    public static final String IGNITE_IO_BALANCE_RANDOM_BALANCE = "IGNITE_IO_BALANCE_RANDOM_BALANCER";
+
     /** Default session write timeout. */
     public static final int DFLT_SES_WRITE_TIMEOUT = 5000;
 
     /** Default send queue limit. */
-    public static final int DFLT_SEND_QUEUE_LIMIT = 1024;
+    public static final int DFLT_SEND_QUEUE_LIMIT = 0;
 
     /** Time, which server will wait before retry operation. */
     private static final long ERR_WAIT_TIME = 2000;
@@ -122,6 +129,9 @@ public class GridNioServer<T> {
         }
     }
 
+    /** Defines how many times selector should do {@code selectNow()} before doing {@code select(long)}. */
+    private long selectorSpins;
+
     /** Accept worker thread. */
     @GridToStringExclude
     private final IgniteThread acceptThread;
@@ -145,9 +155,13 @@ public class GridNioServer<T> {
     /** Flag indicating if this server should use direct buffers. */
     private final boolean directBuf;
 
-    /** Index to select which thread will serve next socket channel. Using round-robin balancing. */
+    /** Index to select which thread will serve next incoming socket channel. Using round-robin balancing. */
+    @GridToStringExclude
+    private int readBalanceIdx;
+
+    /** Index to select which thread will serve next out socket channel. Using round-robin balancing. */
     @GridToStringExclude
-    private int balanceIdx;
+    private int writeBalanceIdx = 1;
 
     /** Tcp no delay flag. */
     private final boolean tcpNoDelay;
@@ -204,12 +218,25 @@ public class GridNioServer<T> {
     /** Optional listener to monitor outbound message queue size. */
     private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
+    /** */
+    private final AtomicLong readerMoveCnt = new AtomicLong();
+
+    /** */
+    private final AtomicLong writerMoveCnt = new AtomicLong();
+
+    /** */
+    private final IgniteRunnable balancer;
+
     /**
      * @param addr Address.
      * @param port Port.
      * @param log Log.
      * @param selectorCnt Count of selectors and selecting threads.
      * @param gridName Grid name.
+     * @param srvName Logical server name for threads identification.
+     * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+     *      falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     *      Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
      * @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
      * @param directBuf Direct buffer flag.
      * @param order Byte order.
@@ -223,6 +250,7 @@ public class GridNioServer<T> {
      * @param writerFactory Writer factory.
      * @param skipRecoveryPred Skip recovery predicate.
      * @param msgQueueLsnr Message queue size listener.
+     * @param balancing NIO sessions balancing flag.
      * @param filters Filters for this server.
      * @throws IgniteCheckedException If failed.
      */
@@ -232,6 +260,8 @@ public class GridNioServer<T> {
         IgniteLogger log,
         int selectorCnt,
         @Nullable String gridName,
+        @Nullable String srvName,
+        long selectorSpins,
         boolean tcpNoDelay,
         boolean directBuf,
         ByteOrder order,
@@ -245,6 +275,7 @@ public class GridNioServer<T> {
         GridNioMessageWriterFactory writerFactory,
         IgnitePredicate<Message> skipRecoveryPred,
         IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
+        boolean balancing,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         if (port != -1)
@@ -268,6 +299,7 @@ public class GridNioServer<T> {
         this.sockSndBuf = sockSndBuf;
         this.sndQueueLimit = sndQueueLimit;
         this.msgQueueLsnr = msgQueueLsnr;
+        this.selectorSpins = selectorSpins;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -299,9 +331,16 @@ public class GridNioServer<T> {
         clientThreads = new IgniteThread[selectorCnt];
 
         for (int i = 0; i < selectorCnt; i++) {
+            String threadName;
+
+            if (srvName == null)
+                threadName = "grid-nio-worker-" + i;
+            else
+                threadName = "grid-nio-worker-" + srvName + "-" + i;
+
             AbstractNioClientWorker worker = directMode ?
-                new DirectNioClientWorker(i, gridName, "grid-nio-worker-" + i, log) :
-                new ByteBufferNioClientWorker(i, gridName, "grid-nio-worker-" + i, log);
+                new DirectNioClientWorker(i, gridName, threadName, log) :
+                new ByteBufferNioClientWorker(i, gridName, threadName, log);
 
             clientWorkers.add(worker);
 
@@ -315,6 +354,32 @@ public class GridNioServer<T> {
         this.writerFactory = writerFactory;
 
         this.skipRecoveryPred = skipRecoveryPred != null ? skipRecoveryPred : F.<Message>alwaysFalse();
+
+        long balancePeriod = IgniteSystemProperties.getLong(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, 5000);
+
+        IgniteRunnable balancer0 = null;
+
+        if (balancing && balancePeriod > 0) {
+            boolean rndBalance = IgniteSystemProperties.getBoolean(IGNITE_IO_BALANCE_RANDOM_BALANCE, false);
+
+            balancer0 = rndBalance ? new RandomBalancer() : new SizeBasedBalancer(balancePeriod);
+        }
+
+        this.balancer = balancer0;
+    }
+
+    /**
+     * @return Number of reader sessions move.
+     */
+    public long readerMoveCount() {
+        return readerMoveCnt.get();
+    }
+
+    /**
+     * @return Number of reader writer move.
+     */
+    public long writerMoveCount() {
+        return writerMoveCnt.get();
     }
 
     /**
@@ -377,6 +442,13 @@ public class GridNioServer<T> {
     }
 
     /**
+     * @return Selector spins.
+     */
+    public long selectorSpins() {
+        return selectorSpins;
+    }
+
+    /**
      * @param ses Session to close.
      * @return Future for operation.
      */
@@ -390,7 +462,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<Boolean> fut = new NioOperationFuture<>(impl, NioOperation.CLOSE);
 
-        clientWorkers.get(impl.selectorIndex()).offer(fut);
+        impl.offerStateChange(fut);
 
         return fut;
     }
@@ -398,61 +470,91 @@ public class GridNioServer<T> {
     /**
      * @param ses Session.
      * @param msg Message.
+     * @param createFut {@code True} if future should be created.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg) {
-        assert ses instanceof GridSelectorNioSessionImpl;
+    GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg, boolean createFut) throws IgniteCheckedException {
+        assert ses instanceof GridSelectorNioSessionImpl : ses;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
+        if (createFut) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
 
-        send0(impl, fut, false);
+            send0(impl, fut, false);
 
-        return fut;
+            return fut;
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, true);
+
+            send0(impl, req, false);
+
+            return null;
+        }
     }
 
     /**
      * @param ses Session.
      * @param msg Message.
+     * @param createFut {@code True} if future should be created.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, Message msg) {
+    GridNioFuture<?> send(GridNioSession ses, Message msg, boolean createFut) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-            skipRecoveryPred.apply(msg));
+        if (createFut) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+                skipRecoveryPred.apply(msg));
 
-        send0(impl, fut, false);
+            send0(impl, fut, false);
 
-        return fut;
+            return fut;
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, skipRecoveryPred.apply(msg));
+
+            send0(impl, req, false);
+
+            return null;
+        }
     }
 
     /**
      * @param ses Session.
-     * @param fut Future.
+     * @param req Request.
      * @param sys System message flag.
+     * @throws IgniteCheckedException If session was closed.
      */
-    private void send0(GridSelectorNioSessionImpl ses, NioOperationFuture<?> fut, boolean sys) {
+    private void send0(GridSelectorNioSessionImpl ses, SessionWriteRequest req, boolean sys) throws IgniteCheckedException {
         assert ses != null;
-        assert fut != null;
+        assert req != null;
 
-        int msgCnt = sys ? ses.offerSystemFuture(fut) : ses.offerFuture(fut);
+        int msgCnt = sys ? ses.offerSystemFuture(req) : ses.offerFuture(req);
 
         IgniteInClosure<IgniteException> ackC;
 
         if (!sys && (ackC = ses.removeMeta(ACK_CLOSURE.ordinal())) != null)
-            fut.ackClosure(ackC);
+            req.ackClosure(ackC);
 
         if (ses.closed()) {
-            if (ses.removeFuture(fut))
-                fut.connectionClosed();
+            if (ses.removeFuture(req)) {
+                IOException err = new IOException("Failed to send message (connection was closed): " + ses);
+
+                req.onError(err);
+
+                if (!(req instanceof GridNioFuture))
+                    throw new IgniteCheckedException(err);
+            }
+        }
+        else if (!ses.procWrite.get() && ses.procWrite.compareAndSet(false, true)) {
+            AbstractNioClientWorker worker = (AbstractNioClientWorker)ses.worker();
+
+            if (worker != null)
+                worker.offer((SessionChangeRequest)req);
         }
-        else if (msgCnt == 1)
-            // Change from 0 to 1 means that worker thread should be waken up.
-            clientWorkers.get(ses.selectorIndex()).offer(fut);
 
         if (msgQueueLsnr != null)
             msgQueueLsnr.apply(ses, msgCnt);
@@ -463,10 +565,10 @@ public class GridNioServer<T> {
      *
      * @param ses Session.
      * @param msg Message.
-     * @return Future.
+     * @throws IgniteCheckedException If session was closed.
      */
-    public GridNioFuture<?> sendSystem(GridNioSession ses, Message msg) {
-        return sendSystem(ses, msg, null);
+    public void sendSystem(GridNioSession ses, Message msg) throws IgniteCheckedException {
+        sendSystem(ses, msg, null);
     }
 
     /**
@@ -475,27 +577,30 @@ public class GridNioServer<T> {
      * @param ses Session.
      * @param msg Message.
      * @param lsnr Future listener notified from the session thread.
-     * @return Future.
+     * @throws IgniteCheckedException If session was closed.
      */
-    public GridNioFuture<?> sendSystem(GridNioSession ses,
+    public void sendSystem(GridNioSession ses,
         Message msg,
-        @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) {
+        @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-            skipRecoveryPred.apply(msg));
-
         if (lsnr != null) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+                skipRecoveryPred.apply(msg));
+
             fut.listen(lsnr);
 
             assert !fut.isDone();
-        }
 
-        send0(impl, fut, true);
+            send0(impl, fut, true);
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
 
-        return fut;
+            send0(impl, req, true);
+        }
     }
 
     /**
@@ -504,37 +609,69 @@ public class GridNioServer<T> {
     public void resend(GridNioSession ses) {
         assert ses instanceof GridSelectorNioSessionImpl;
 
-        GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+        GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
-        if (recoveryDesc != null && !recoveryDesc.messagesFutures().isEmpty()) {
-            Deque<GridNioFuture<?>> futs = recoveryDesc.messagesFutures();
+        if (recoveryDesc != null && !recoveryDesc.messagesRequests().isEmpty()) {
+            Deque<SessionWriteRequest> futs = recoveryDesc.messagesRequests();
 
             if (log.isDebugEnabled())
                 log.debug("Resend messages [rmtNode=" + recoveryDesc.node().id() + ", msgCnt=" + futs.size() + ']');
 
             GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
 
-            GridNioFuture<?> fut0 = futs.iterator().next();
+            SessionWriteRequest fut0 = futs.iterator().next();
 
-            for (GridNioFuture<?> fut : futs) {
+            for (SessionWriteRequest fut : futs) {
                 fut.messageThread(true);
 
-                ((NioOperationFuture)fut).resetSession(ses0);
+                fut.resetSession(ses0);
             }
 
             ses0.resend(futs);
 
             // Wake up worker.
-            clientWorkers.get(ses0.selectorIndex()).offer(((NioOperationFuture)fut0));
+            ses0.offerStateChange((GridNioServer.SessionChangeRequest)fut0);
         }
     }
 
     /**
+     * @return Sessions.
+     */
+    public Collection<? extends GridNioSession> sessions() {
+        return sessions;
+    }
+
+    /**
+     * @return Workers.
+     */
+    public List<AbstractNioClientWorker> workers() {
+        return clientWorkers;
+    }
+
+    /**
+     * @param ses Session.
+     * @param from Move from index.
+     * @param to Move to index.
+     */
+    private void moveSession(GridNioSession ses, int from, int to) {
+        assert from >= 0 && from < clientWorkers.size() : from;
+        assert to >= 0 && to < clientWorkers.size() : to;
+        assert from != to;
+
+        GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+        SessionMoveFuture fut = new SessionMoveFuture(ses0, to);
+
+        if (!ses0.offerMove(clientWorkers.get(from), fut))
+            fut.onDone(false);
+    }
+
+    /**
      * @param ses Session.
      * @param op Operation.
      * @return Future for operation.
      */
-    GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
+    private GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
         assert ses instanceof GridSelectorNioSessionImpl;
         assert op == NioOperation.PAUSE_READ || op == NioOperation.RESUME_READ;
 
@@ -546,7 +683,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, op);
 
-        clientWorkers.get(impl.selectorIndex()).offer(fut);
+        impl.offerStateChange(fut);
 
         return fut;
     }
@@ -555,6 +692,9 @@ public class GridNioServer<T> {
      *
      */
     public void dumpStats() {
+        U.warn(log, "NIO server statistics [readerSesBalanceCnt=" + readerMoveCnt.get() +
+            ", writerSesBalanceCnt=" + writerMoveCnt.get() + ']');
+
         for (int i = 0; i < clientWorkers.size(); i++)
             clientWorkers.get(i).offer(new NioOperationFuture<Void>(null, NioOperation.DUMP_STATS));
     }
@@ -675,12 +815,35 @@ public class GridNioServer<T> {
      * @param req Request to balance.
      */
     private synchronized void offerBalanced(NioOperationFuture req) {
-        clientWorkers.get(balanceIdx).offer(req);
+        assert req.operation() == NioOperation.REGISTER : req;
+        assert req.socketChannel() != null : req;
+
+        int workers = clientWorkers.size();
+
+        int balanceIdx;
+
+        if (workers > 1) {
+            if (req.accepted()) {
+                balanceIdx = readBalanceIdx;
+
+                readBalanceIdx += 2;
+
+                if (readBalanceIdx >= workers)
+                    readBalanceIdx = 0;
+            }
+            else {
+                balanceIdx = writeBalanceIdx;
 
-        balanceIdx++;
+                writeBalanceIdx += 2;
 
-        if (balanceIdx == clientWorkers.size())
+                if (writeBalanceIdx >= workers)
+                    writeBalanceIdx = 1;
+            }
+        }
+        else
             balanceIdx = 0;
+
+        clientWorkers.get(balanceIdx).offer(req);
     }
 
     /** {@inheritDoc} */
@@ -792,21 +955,30 @@ public class GridNioServer<T> {
 
             while (true) {
                 ByteBuffer buf = ses.removeMeta(BUF_META_KEY);
-                NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
                 // Check if there were any pending data from previous writes.
                 if (buf == null) {
                     assert req == null;
 
-                    req = (NioOperationFuture<?>)ses.pollFuture();
+                    req = ses.pollFuture();
 
                     if (req == null) {
-                        key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                        if (ses.procWrite.get()) {
+                            ses.procWrite.set(false);
+
+                            if (ses.writeQueue().isEmpty()) {
+                                if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                            }
+                            else
+                                ses.procWrite.set(true);
+                        }
 
                         break;
                     }
 
-                    buf = req.message();
+                    buf = (ByteBuffer)req.message();
                 }
 
                 if (!skipWrite) {
@@ -841,10 +1013,15 @@ public class GridNioServer<T> {
                     // Message was successfully written.
                     assert req != null;
 
-                    req.onDone();
+                    req.onMessageWritten();
                 }
             }
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ByteBufferNioClientWorker.class, this, super.toString());
+        }
     }
 
     /**
@@ -909,6 +1086,7 @@ public class GridNioServer<T> {
                 metricsLsnr.onBytesReceived(cnt);
 
             ses.bytesReceived(cnt);
+            onRead(cnt);
 
             readBuf.flip();
 
@@ -921,6 +1099,12 @@ public class GridNioServer<T> {
                     readBuf.compact();
                 else
                     readBuf.clear();
+
+                if (ses.hasSystemMessage() && !ses.procWrite.get()) {
+                    ses.procWrite.set(true);
+
+                    registerWrite(ses);
+                }
             }
             catch (IgniteCheckedException e) {
                 close(ses, e);
@@ -993,16 +1177,29 @@ public class GridNioServer<T> {
                 if (ses.meta(WRITE_BUF_LIMIT) != null)
                     buf.limit((int)ses.meta(WRITE_BUF_LIMIT));
 
-                NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
                 while (true) {
                     if (req == null) {
-                        req = (NioOperationFuture<?>)ses.pollFuture();
+                        req = systemMessage(ses);
 
-                        if (req == null && buf.position() == 0) {
-                            key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                        if (req == null) {
+                            req = ses.pollFuture();
 
-                            break;
+                            if (req == null && buf.position() == 0) {
+                                if (ses.procWrite.get()) {
+                                    ses.procWrite.set(false);
+
+                                    if (ses.writeQueue().isEmpty()) {
+                                        if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                            key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                                    }
+                                    else
+                                        ses.procWrite.set(true);
+                                }
+
+                                break;
+                            }
                         }
                     }
 
@@ -1010,7 +1207,7 @@ public class GridNioServer<T> {
                     boolean finished = false;
 
                     if (req != null) {
-                        msg = req.directMessage();
+                        msg = (Message)req.message();
 
                         assert msg != null;
 
@@ -1025,14 +1222,17 @@ public class GridNioServer<T> {
 
                     // Fill up as many messages as possible to write buffer.
                     while (finished) {
-                        req.onDone();
+                        req.onMessageWritten();
 
-                        req = (NioOperationFuture<?>)ses.pollFuture();
+                        req = systemMessage(ses);
+
+                        if (req == null)
+                            req = ses.pollFuture();
 
                         if (req == null)
                             break;
 
-                        msg = req.directMessage();
+                        msg = (Message)req.message();
 
                         assert msg != null;
 
@@ -1129,13 +1329,31 @@ public class GridNioServer<T> {
                 ses.bytesSent(cnt);
 
                 if (!buf.hasRemaining())
-                    queue.remove(buf);
+                    queue.poll();
                 else
                     break;
             }
         }
 
         /**
+         * @param ses Session.
+         * @return System message request.
+         */
+        private SessionWriteRequest systemMessage(GridSelectorNioSessionImpl ses) {
+            if (ses.hasSystemMessage()) {
+                Object msg = ses.systemMessage();
+
+                SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
+
+                assert !ses.hasSystemMessage();
+
+                return req;
+            }
+
+            return null;
+        }
+
+        /**
          * Processes write-ready event on the key.
          *
          * @param key Key that is ready to be written.
@@ -1147,7 +1365,7 @@ public class GridNioServer<T> {
 
             GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
             ByteBuffer buf = ses.writeBuffer();
-            NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+            SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
             MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
 
@@ -1161,12 +1379,25 @@ public class GridNioServer<T> {
             }
 
             if (req == null) {
-                req = (NioOperationFuture<?>)ses.pollFuture();
+                req = systemMessage(ses);
 
-                if (req == null && buf.position() == 0) {
-                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                if (req == null) {
+                    req = ses.pollFuture();
 
-                    return;
+                    if (req == null && buf.position() == 0) {
+                        if (ses.procWrite.get()) {
+                            ses.procWrite.set(false);
+
+                            if (ses.writeQueue().isEmpty()) {
+                                if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                            }
+                            else
+                                ses.procWrite.set(true);
+                        }
+
+                        return;
+                    }
                 }
             }
 
@@ -1174,9 +1405,9 @@ public class GridNioServer<T> {
             boolean finished = false;
 
             if (req != null) {
-                msg = req.directMessage();
+                msg = (Message)req.message();
 
-                assert msg != null;
+                assert msg != null : req;
 
                 if (writer != null)
                     writer.setCurrentWriteClass(msg.getClass());
@@ -1189,14 +1420,17 @@ public class GridNioServer<T> {
 
             // Fill up as many messages as possible to write buffer.
             while (finished) {
-                req.onDone();
+                req.onMessageWritten();
 
-                req = (NioOperationFuture<?>)ses.pollFuture();
+                req = systemMessage(ses);
+
+                if (req == null)
+                    req = ses.pollFuture();
 
                 if (req == null)
                     break;
 
-                msg = req.directMessage();
+                msg = (Message)req.message();
 
                 assert msg != null;
 
@@ -1223,6 +1457,7 @@ public class GridNioServer<T> {
                     metricsLsnr.onBytesSent(cnt);
 
                 ses.bytesSent(cnt);
+                onWrite(cnt);
             }
             else {
                 // For test purposes only (skipWrite is set to true in tests only).
@@ -1242,14 +1477,19 @@ public class GridNioServer<T> {
             else
                 buf.clear();
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DirectNioClientWorker.class, this, super.toString());
+        }
     }
 
     /**
      * Thread performing only read operations from the channel.
      */
-    private abstract class AbstractNioClientWorker extends GridWorker {
+    private abstract class AbstractNioClientWorker extends GridWorker implements GridNioWorker {
         /** Queue of change requests on this selector. */
-        private final ConcurrentLinkedQueue<NioOperationFuture> changeReqs = new ConcurrentLinkedQueue<>();
+        private final ConcurrentLinkedQueue<SessionChangeRequest> changeReqs = new ConcurrentLinkedQueue<>();
 
         /** Selector to select read events. */
         private Selector selector;
@@ -1260,6 +1500,25 @@ public class GridNioServer<T> {
         /** Worker index. */
         private final int idx;
 
+        /** */
+        private long bytesRcvd;
+
+        /** */
+        private long bytesSent;
+
+        /** */
+        private volatile long bytesRcvd0;
+
+        /** */
+        private volatile long bytesSent0;
+
+        /** Sessions assigned to this worker. */
+        private final GridConcurrentHashSet<GridSelectorNioSessionImpl> workerSessions =
+            new GridConcurrentHashSet<>();
+
+        /** {@code True} if worker has called or is about to call {@code Selector.select()}. */
+        private volatile boolean select;
+
         /**
          * @param idx Index of this worker in server's array.
          * @param gridName Grid name.
@@ -1322,15 +1581,15 @@ public class GridNioServer<T> {
             try {
                 SelectedSelectionKeySet selectedKeySet = new SelectedSelectionKeySet();
 
-                Class<?> selectorImplClass =
+                Class<?> selectorImplCls =
                     Class.forName("sun.nio.ch.SelectorImpl", false, U.gridClassLoader());
 
                 // Ensure the current selector implementation is what we can instrument.
-                if (!selectorImplClass.isAssignableFrom(selector.getClass()))
+                if (!selectorImplCls.isAssignableFrom(selector.getClass()))
                     return;
 
-                Field selectedKeysField = selectorImplClass.getDeclaredField("selectedKeys");
-                Field publicSelectedKeysField = selectorImplClass.getDeclaredField("publicSelectedKeys");
+                Field selectedKeysField = selectorImplCls.getDeclaredField("selectedKeys");
+                Field publicSelectedKeysField = selectorImplCls.getDeclaredField("publicSelectedKeys");
 
                 selectedKeysField.setAccessible(true);
                 publicSelectedKeysField.setAccessible(true);
@@ -1357,48 +1616,126 @@ public class GridNioServer<T> {
          *
          * @param req Change request.
          */
-        private void offer(NioOperationFuture req) {
+        public void offer(SessionChangeRequest req) {
             changeReqs.offer(req);
 
+            if (select)
+                selector.wakeup();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void offer(Collection<SessionChangeRequest> reqs) {
+            for (SessionChangeRequest req : reqs)
+                changeReqs.offer(req);
+
             selector.wakeup();
         }
 
+        /** {@inheritDoc} */
+        @Override public List<SessionChangeRequest> clearSessionRequests(GridNioSession ses) {
+            List<SessionChangeRequest> sesReqs = null;
+
+            for (SessionChangeRequest changeReq : changeReqs) {
+                if (changeReq.session() == ses && !(changeReq instanceof SessionMoveFuture)) {
+                    boolean rmv = changeReqs.remove(changeReq);
+
+                    assert rmv : changeReq;
+
+                    if (sesReqs == null)
+                        sesReqs = new ArrayList<>();
+
+                    sesReqs.add(changeReq);
+                }
+            }
+
+            return sesReqs;
+        }
+
         /**
          * Processes read and write events and registration requests.
          *
          * @throws IgniteCheckedException If IOException occurred or thread was unable to add worker to workers pool.
          */
         @SuppressWarnings("unchecked")
-        private void bodyInternal() throws IgniteCheckedException {
+        private void bodyInternal() throws IgniteCheckedException, InterruptedException {
             try {
                 long lastIdleCheck = U.currentTimeMillis();
 
+                mainLoop:
                 while (!closed && selector.isOpen()) {
-                    NioOperationFuture req;
+                    SessionChangeRequest req0;
 
-                    while ((req = changeReqs.poll()) != null) {
-                        switch (req.operation()) {
+                    while ((req0 = changeReqs.poll()) != null) {
+                        switch (req0.operation()) {
                             case REGISTER: {
-                                register(req);
+                                register((NioOperationFuture)req0);
 
                                 break;
                             }
 
-                            case REQUIRE_WRITE: {
-                                //Just register write key.
-                                SelectionKey key = req.session().key();
+                            case MOVE: {
+                                SessionMoveFuture f = (SessionMoveFuture)req0;
 
-                                if (key.isValid()) {
-                                    key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                                GridSelectorNioSessionImpl ses = f.session();
+
+                                if (idx == f.toIdx) {
+                                    assert f.movedSocketChannel() != null : f;
+
+                                    boolean add = workerSessions.add(ses);
+
+                                    assert add;
 
-                                    // Update timestamp to protected against false write timeout.
-                                    ((GridNioSessionImpl)key.attachment()).bytesSent(0);
+                                    ses.finishMoveSession(this);
+
+                                    if (idx % 2 == 0)
+                                        readerMoveCnt.incrementAndGet();
+                                    else
+                                        writerMoveCnt.incrementAndGet();
+
+                                    SelectionKey key = f.movedSocketChannel().register(selector,
+                                        SelectionKey.OP_READ | SelectionKey.OP_WRITE,
+                                        ses);
+
+                                    ses.key(key);
+
+                                    ses.procWrite.set(true);
+
+                                    f.onDone(true);
+                                }
+                                else {
+                                    assert f.movedSocketChannel() == null : f;
+
+                                    if (workerSessions.remove(ses)) {
+                                        ses.startMoveSession(this);
+
+                                        SelectionKey key = ses.key();
+
+                                        assert key.channel() != null : key;
+
+                                        f.movedSocketChannel((SocketChannel)key.channel());
+
+                                        key.cancel();
+
+                                        clientWorkers.get(f.toIndex()).offer(f);
+                                    }
+                                    else
+                                        f.onDone(false);
                                 }
 
                                 break;
                             }
 
+                            case REQUIRE_WRITE: {
+                                SessionWriteRequest req = (SessionWriteRequest)req0;
+
+                                registerWrite((GridSelectorNioSessionImpl)req.session());
+
+                                break;
+                            }
+
                             case CLOSE: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 if (close(req.session(), null))
                                     req.onDone(true);
                                 else
@@ -1408,6 +1745,8 @@ public class GridNioServer<T> {
                             }
 
                             case PAUSE_READ: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 SelectionKey key = req.session().key();
 
                                 if (key.isValid()) {
@@ -1426,6 +1765,8 @@ public class GridNioServer<T> {
                             }
 
                             case RESUME_READ: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 SelectionKey key = req.session().key();
 
                                 if (key.isValid()) {
@@ -1444,76 +1785,66 @@ public class GridNioServer<T> {
                             }
 
                             case DUMP_STATS: {
-                                StringBuilder sb = new StringBuilder();
+                                NioOperationFuture req = (NioOperationFuture)req0;
 
-                                Set<SelectionKey> keys = selector.keys();
-
-                                sb.append(U.nl())
-                                    .append(">> Selector info [idx=").append(idx)
-                                    .append(", keysCnt=").append(keys.size())
-                                    .append("]").append(U.nl());
-
-                                for (SelectionKey key : keys) {
-                                    GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
-
-                                    MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
-                                    MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
-
-                                    sb.append("    Connection info [")
-                                        .append("rmtAddr=").append(ses.remoteAddress())
-                                        .append(", locAddr=").append(ses.localAddress());
-
-                                    GridNioRecoveryDescriptor desc = ses.recoveryDescriptor();
+                                try {
+                                    dumpStats();
+                                }
+                                finally {
+                                    // Complete the request just in case (none should wait on this future).
+                                    req.onDone(true);
+                                }
+                            }
+                        }
+                    }
 
-                                    if (desc != null) {
-                                        sb.append(", msgsSent=").append(desc.sent())
-                                            .append(", msgsAckedByRmt=").append(desc.acked())
-                                            .append(", msgsRcvd=").append(desc.received())
-                                            .append(", descIdHash=").append(System.identityHashCode(desc));
-                                    }
-                                    else
-                                        sb.append(", recoveryDesc=null");
+                    int res = 0;
 
-                                    sb.append(", bytesRcvd=").append(ses.bytesReceived())
-                                        .append(", bytesSent=").append(ses.bytesSent())
-                                        .append(", opQueueSize=").append(ses.writeQueueSize())
-                                        .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
-                                        .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+                    for (long i = 0; i < selectorSpins && res == 0; i++) {
+                        res = selector.selectNow();
 
-                                    int cnt = 0;
+                        if (res > 0) {
+                            // Walk through the ready keys collection and process network events.
+                            if (selectedKeys == null)
+                                processSelectedKeys(selector.selectedKeys());
+                            else
+                                processSelectedKeysOptimized(selectedKeys.flip());
+                        }
 
-                                    for (GridNioFuture<?> fut : ses.writeQueue()) {
-                                        if (cnt == 0)
-                                            sb.append(",\n opQueue=[").append(fut);
-                                        else
-                                            sb.append(',').append(fut);
+                        if (!changeReqs.isEmpty())
+                            continue mainLoop;
 
-                                        if (++cnt == 5) {
-                                            sb.append(']');
+                        // Just in case we do busy selects.
+                        long now = U.currentTimeMillis();
 
-                                            break;
-                                        }
-                                    }
+                        if (now - lastIdleCheck > 2000) {
+                            lastIdleCheck = now;
 
+                            checkIdle(selector.keys());
+                        }
 
-                                    sb.append("]").append(U.nl());
-                                }
+                        if (isCancelled())
+                            return;
+                    }
 
-                                U.warn(log, sb.toString());
+                    // Falling to blocking select.
+                    select = true;
 
-                                // Complete the request just in case (none should wait on this future).
-                                req.onDone(true);
-                            }
+                    try {
+                        if (!changeReqs.isEmpty())
+                            continue;
+
+                        // Wake up every 2 seconds to check if closed.
+                        if (selector.select(2000) > 0) {
+                            // Walk through the ready keys collection and process network events.
+                            if (selectedKeys == null)
+                                processSelectedKeys(selector.selectedKeys());
+                            else
+                                processSelectedKeysOptimized(selectedKeys.flip());
                         }
                     }
-
-                    // Wake up every 2 seconds to check if closed.
-                    if (selector.select(2000) > 0) {
-                        // Walk through the ready keys collection and process network events.
-                        if (selectedKeys == null)
-                            processSelectedKeys(selector.selectedKeys());
-                        else
-                            processSelectedKeysOptimized(selectedKeys.flip());
+                    finally {
+                        select = false;
                     }
 
                     long now = U.currentTimeMillis();
@@ -1554,6 +1885,98 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param ses Session.
+         */
+        public final void registerWrite(GridSelectorNioSessionImpl ses) {
+            SelectionKey key = ses.key();
+
+            if (key.isValid()) {
+                if ((key.interestOps() & SelectionKey.OP_WRITE) == 0)
+                    key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+
+                // Update timestamp to protected against false write timeout.
+                ses.bytesSent(0);
+            }
+        }
+
+        /**
+         *
+         */
+        private void dumpStats() {
+            StringBuilder sb = new StringBuilder();
+
+            Set<SelectionKey> keys = selector.keys();
+
+            sb.append(U.nl())
+                .append(">> Selector info [idx=").append(idx)
+                .append(", keysCnt=").append(keys.size())
+                .append(", bytesRcvd=").append(bytesRcvd)
+                .append(", bytesRcvd0=").append(bytesRcvd0)
+                .append(", bytesSent=").append(bytesSent)
+                .append(", bytesSent0=").append(bytesSent0)
+                .append("]").append(U.nl());
+
+            for (SelectionKey key : keys) {
+                GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+                MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
+                MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
+
+                sb.append("    Connection info [")
+                    .append("in=").append(ses.accepted())
+                    .append(", rmtAddr=").append(ses.remoteAddress())
+                    .append(", locAddr=").append(ses.localAddress());
+
+                GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
+
+                if (outDesc != null) {
+                    sb.append(", msgsSent=").append(outDesc.sent())
+                        .append(", msgsAckedByRmt=").append(outDesc.acked())
+                        .append(", descIdHash=").append(System.identityHashCode(outDesc));
+                }
+                else
+                    sb.append(", outRecoveryDesc=null");
+
+                GridNioRecoveryDescriptor inDesc = ses.inRecoveryDescriptor();
+
+                if (inDesc != null) {
+                    sb.append(", msgsRcvd=").append(inDesc.received())
+                        .append(", lastAcked=").append(inDesc.lastAcknowledged())
+                        .append(", descIdHash=").append(System.identityHashCode(inDesc));
+                }
+                else
+                    sb.append(", inRecoveryDesc=null");
+
+                sb.append(", bytesRcvd=").append(ses.bytesReceived())
+                    .append(", bytesRcvd0=").append(ses.bytesReceived0())
+                    .append(", bytesSent=").append(ses.bytesSent())
+                    .append(", bytesSent0=").append(ses.bytesSent0())
+                    .append(", opQueueSize=").append(ses.writeQueueSize())
+                    .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
+                    .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+
+                int cnt = 0;
+
+                for (SessionWriteRequest req : ses.writeQueue()) {
+                    if (cnt == 0)
+                        sb.append(",\n opQueue=[").append(req);
+                    else
+                        sb.append(',').append(req);
+
+                    if (++cnt == 5) {
+                        sb.append(']');
+
+                        break;
+                    }
+                }
+
+                sb.append("]").append(U.nl());
+            }
+
+            U.warn(log, sb.toString());
+        }
+
+        /**
          * Processes keys selected by a selector.
          *
          * @param keys Selected keys.
@@ -1671,7 +2094,9 @@ public class GridNioServer<T> {
 
                     long idleTimeout0 = idleTimeout;
 
-                    if (!opWrite && now - ses.lastReceiveTime() > idleTimeout0 && now - ses.lastSendScheduleTime() > idleTimeout0) {
+                    if (!opWrite &&
+                        now - ses.lastReceiveTime() > idleTimeout0 &&
+                        now - ses.lastSendScheduleTime() > idleTimeout0) {
                         filterChain.onSessionIdleTimeout(ses);
 
                         // Update timestamp to avoid multiple notifications within one timeout interval.
@@ -1715,7 +2140,7 @@ public class GridNioServer<T> {
 
                 final GridSelectorNioSessionImpl ses = new GridSelectorNioSessionImpl(
                     log,
-                    idx,
+                    this,
                     filterChain,
                     (InetSocketAddress)sockCh.getLocalAddress(),
                     (InetSocketAddress)sockCh.getRemoteAddress(),
@@ -1739,6 +2164,7 @@ public class GridNioServer<T> {
                     resend(ses);
 
                 sessions.add(ses);
+                workerSessions.add(ses);
 
                 try {
                     filterChain.onSessionOpened(ses);
@@ -1764,7 +2190,7 @@ public class GridNioServer<T> {
         }
 
         /**
-         * Closes the ses and all associated resources, then notifies the listener.
+         * Closes the session and all associated resources, then notifies the listener.
          *
          * @param ses Session to be closed.
          * @param e Exception to be passed to the listener, if any.
@@ -1781,12 +2207,10 @@ public class GridNioServer<T> {
             }
 
             sessions.remove(ses);
+            workerSessions.remove(ses);
 
             SelectionKey key = ses.key();
 
-            // Shutdown input and output so that remote client will see correct socket close.
-            Socket sock = ((SocketChannel)key.channel()).socket();
-
             if (ses.setClosed()) {
                 ses.onClosed();
 
@@ -1798,6 +2222,9 @@ public class GridNioServer<T> {
                         ((DirectBuffer)ses.readBuffer()).cleaner().clean();
                 }
 
+                // Shutdown input and output so that remote client will see correct socket close.
+                Socket sock = ((SocketChannel)key.channel()).socket();
+
                 try {
                     try {
                         sock.shutdownInput();
@@ -1824,28 +2251,35 @@ public class GridNioServer<T> {
                 ses.removeMeta(BUF_META_KEY);
 
                 // Since ses is in closed state, no write requests will be added.
-                NioOperationFuture<?> fut = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
+
+                GridNioRecoveryDescriptor outRecovery = ses.outRecoveryDescriptor();
+                GridNioRecoveryDescriptor inRecovery = ses.inRecoveryDescriptor();
 
-                GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+                IOException err = new IOException("Failed to send message (connection was closed): " + ses);
 
-                if (recovery != null) {
+                if (outRecovery != null || inRecovery != null) {
                     try {
                         // Poll will update recovery data.
-                        while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null) {
-                            if (fut.skipRecovery())
-                                fut.connectionClosed();
+                        while ((req = ses.pollFuture()) != null) {
+                            if (req.skipRecovery())
+                                req.onError(err);
                         }
                     }
                     finally {
-                        recovery.release();
+                        if (outRecovery != null)
+                            outRecovery.release();
+
+                        if (inRecovery != null && inRecovery != outRecovery)
+                            inRecovery.release();
                     }
                 }
                 else {
-                    if (fut != null)
-                        fut.connectionClosed();
+                    if (req != null)
+                        req.onError(err);
 
-                    while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null)
-                        fut.connectionClosed();
+                    while ((req = ses.pollFuture()) != null)
+                        req.onError(err);
                 }
 
                 try {
@@ -1876,12 +2310,44 @@ public class GridNioServer<T> {
          * @throws IOException If write failed.
          */
         protected abstract void processWrite(SelectionKey key) throws IOException;
-    }
 
-    /**
-     * Gets outbound messages queue size.
-     *
-     * @return Write queue size.
+        /**
+         * @param cnt
+         */
+        final void onRead(int cnt) {
+            bytesRcvd += cnt;
+            bytesRcvd0 += cnt;
+        }
+
+        /**
+         * @param cnt
+         */
+        final void onWrite(int cnt) {
+            bytesSent += cnt;
+            bytesSent0 += cnt;
+        }
+
+        /**
+         *
+         */
+        final void reset0() {
+            bytesSent0 = 0;
+            bytesRcvd0 = 0;
+
+            for (GridSelectorNioSessionImpl ses : workerSessions)
+                ses.reset0();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(AbstractNioClientWorker.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Gets outbound messages queue size.
+     *
+     * @return Write queue size.
      */
     public int outboundMessagesQueueSize() {
         int res = 0;
@@ -1952,6 +2418,9 @@ public class GridNioServer<T> {
                     if (selector.select(2000) > 0)
                         // Walk through the ready keys collection and process date requests.
                         processSelectedKeys(selector.selectedKeys());
+
+                    if (balancer != null)
+                        balancer.run();
                 }
             }
             // Ignore this exception as thread interruption is equal to 'close' call.
@@ -2048,10 +2517,13 @@ public class GridNioServer<T> {
     /**
      * Asynchronous operation that may be requested on selector.
      */
-    private enum NioOperation {
+    enum NioOperation {
         /** Register read key selection. */
         REGISTER,
 
+        /** Move session between workers. */
+        MOVE,
+
         /** Register write key selection. */
         REQUIRE_WRITE,
 
@@ -2069,9 +2541,193 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    private static final class WriteRequestSystemImpl implements SessionWriteRequest, SessionChangeRequest {
+        /** */
+        private final Object msg;
+
+        /** */
+        private final GridNioSession ses;
+
+        /**
+         * @param ses Session.
+         * @param msg Message.
+         */
+        WriteRequestSystemImpl(GridNioSession ses, Object msg) {
+            this.ses = ses;
+            this.msg = msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void messageThread(boolean msgThread) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean messageThread() {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteInClosure<IgniteException> ackClosure() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object message() {
+            return msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resetSession(GridNioSession ses) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioSession session() {
+            return ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NioOperation operation() {
+            return NioOperation.REQUIRE_WRITE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(WriteRequestSystemImpl.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static final class WriteRequestImpl implements SessionWriteRequest, SessionChangeRequest {
+        /** */
+        private GridNioSession ses;
+
+        /** */
+        private final Object msg;
+
+        /** */
+        private boolean msgThread;
+
+        /** */
+        private final boolean skipRecovery;
+
+        /** */
+        private IgniteInClosure<IgniteException> ackC;
+
+        /**
+         * @param ses Session.
+         * @param msg Message.
+         * @param skipRecovery Skip recovery flag.
+         */
+        WriteRequestImpl(GridNioSession ses, Object msg, boolean skipRecovery) {
+            this.ses = ses;
+            this.msg = msg;
+            this.skipRecovery = skipRecovery;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void messageThread(boolean msgThread) {
+            this.msgThread = msgThread;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean messageThread() {
+            return msgThread;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return skipRecovery;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+            ackC = c;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            assert msg instanceof Message;
+
+            ((Message)msg).onAckReceived();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteInClosure<IgniteException> ackClosure() {
+            return ackC;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object message() {
+            return msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resetSession(GridNioSession ses) {
+            this.ses = ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioSession session() {
+            return ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NioOperation operation() {
+            return NioOperation.REQUIRE_WRITE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(WriteRequestImpl.class, this);
+        }
+    }
+
+    /**
      * Class for requesting write and session close operations.
      */
-    private static class NioOperationFuture<R> extends GridNioFutureImpl<R> {
+    private static class NioOperationFuture<R> extends GridNioFutureImpl<R> implements SessionWriteRequest,
+        SessionChangeRequest {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2087,11 +2743,7 @@ public class GridNioServer<T> {
         private NioOperation op;
 
         /** Message. */
-        @GridToStringExclude
-        private ByteBuffer msg;
-
-        /** Direct message. */
-        private Message commMsg;
+        private Object msg;
 
         /** */
         @GridToStringExclude
@@ -2153,8 +2805,7 @@ public class GridNioServer<T> {
          * @param op Requested operation.
          * @param msg Message.
          */
-        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op,
-            ByteBuffer msg) {
+        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op, Object msg) {
             assert ses != null;
             assert op != null;
             assert op != NioOperation.REGISTER;
@@ -2182,51 +2833,36 @@ public class GridNioServer<T> {
 
             this.ses = ses;
             this.op = op;
-            this.commMsg = commMsg;
+            this.msg = commMsg;
             this.skipRecovery = skipRecovery;
         }
 
-        /**
-         * @return Requested change operation.
-         */
-        private NioOperation operation() {
+        /** {@inheritDoc} */
+        public NioOperation operation() {
             return op;
         }
 
-        /**
-         * @return Message.
-         */
-        private ByteBuffer message() {
+        /** {@inheritDoc} */
+        public Object message() {
             return msg;
         }
 
-        /**
-         * @return Direct message.
-         */
-        private Message directMessage() {
-            return commMsg;
-        }
-
-        /**
-         * @param ses New session instance.
-         */
-        private void resetSession(GridSelectorNioSessionImpl ses) {
-            assert commMsg != null;
+        /** {@inheritDoc} */
+        public void resetSession(GridNioSession ses) {
+            assert msg instanceof Message : msg;
 
-            this.ses = ses;
+            this.ses = (GridSelectorNioSessionImpl)ses;
         }
 
         /**
          * @return Socket channel for register request.
          */
-        private SocketChannel socketChannel() {
+        SocketChannel socketChannel() {
             return sockCh;
         }
 
-        /**
-         * @return Session for this change request.
-         */
-        private GridSelectorNioSessionImpl session() {
+        /** {@inheritDoc} */
+        public GridSelectorNioSessionImpl session() {
             return ses;
         }
 
@@ -2244,21 +2880,21 @@ public class GridNioServer<T> {
             return meta;
         }
 
-        /**
-         * Applicable to write futures only. Fails future with corresponding IOException.
-         */
-        private void connectionClosed() {
-            assert op == NioOperation.REQUIRE_WRITE;
-            assert ses != null;
-
-            onDone(new IOException("Failed to send message (connection was closed): " + ses));
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            onDone(e);
         }
 
         /** {@inheritDoc} */
         @Override public void onAckReceived() {
-            assert commMsg != null;
+            assert msg instanceof Message : msg;
 
-            commMsg.onAckReceived();
+            ((Message)msg).onAckReceived();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            onDone();
         }
 
         /** {@inheritDoc} */
@@ -2273,6 +2909,59 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    private static class SessionMoveFuture extends NioOperationFuture<Boolean> {
+        /** */
+        private final int toIdx;
+
+        /** */
+        @GridToStringExclude
+        private SocketChannel movedSockCh;
+
+        /**
+         * @param ses Session.
+         * @param toIdx Target worker index.
+         */
+        SessionMoveFuture(
+            GridSelectorNioSessionImpl ses,
+            int toIdx
+        ) {
+            super(ses, NioOperation.MOVE);
+
+            this.toIdx = toIdx;
+        }
+
+        /**
+         * @return Target worker index.
+         */
+        int toIndex() {
+            return toIdx;
+        }
+
+        /**
+         * @return Moved session socket channel.
+         */
+        SocketChannel movedSocketChannel() {
+            return movedSockCh;
+        }
+
+        /**
+         * @param movedSockCh Moved session socket channel.
+         */
+        void movedSocketChannel(SocketChannel movedSockCh) {
+            assert movedSockCh != null;
+
+            this.movedSockCh = movedSockCh;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(SessionMoveFuture.class, this, super.toString());
+        }
+    }
+
+    /**
      * Filter forwarding messages from chain's head to this server.
      */
     private class HeadFilter extends GridNioFilterAdapter {
@@ -2302,7 +2991,7 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             if (directMode) {
                 boolean sslSys = sslFilter != null && msg instanceof ByteBuffer;
 
@@ -2313,18 +3002,18 @@ public class GridNioServer<T> {
 
                     queue.offer((ByteBuffer)msg);
 
-                    SelectionKey key = ((GridSelectorNioSessionImpl)ses).key();
+                    GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
 
-                    if (key.isValid())
-                        key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                    if (!ses0.procWrite.get() && ses0.procWrite.compareAndSet(false, true))
+                        ses0.worker().registerWrite(ses0);
 
                     return null;
                 }
                 else
-                    return send(ses, (Message)msg);
+                    return send(ses, (Message)msg, fut);
             }
             else
-                return send(ses, (ByteBuffer)msg);
+                return send(ses, (ByteBuffer)msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -2429,6 +3118,15 @@ public class GridNioServer<T> {
         /** Message queue size listener. */
         private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
+        /** Name for threads identification. */
+        private String srvName;
+
+        /** */
+        private long selectorSpins;
+
+        /** NIO sessions balancing flag. */
+        private boolean balancing;
+
         /**
          * Finishes building the instance.
          *
@@ -2442,6 +3140,8 @@ public class GridNioServer<T> {
                 log,
                 selectorCnt,
                 gridName,
+                srvName,
+                selectorSpins,
                 tcpNoDelay,
                 directBuf,
                 byteOrder,
@@ -2455,6 +3155,7 @@ public class GridNioServer<T> {
                 writerFactory,
                 skipRecoveryPred,
                 msgQueueLsnr,
+                balancing,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -2468,6 +3169,16 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param balancing NIO sessions balancing flag.
+         * @return This for chaining.
+         */
+        public Builder<T> balancing(boolean balancing) {
+            this.balancing = balancing;
+
+            return this;
+        }
+
+        /**
          * @param addr Local address.
          * @return This for chaining.
          */
@@ -2519,6 +3230,28 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param srvName Logical server name for threads identification.
+         * @return This for chaining.
+         */
+        public Builder<T> serverName(@Nullable String srvName) {
+            this.srvName = srvName;
+
+            return this;
+        }
+
+        /**
+         * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+         *      falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+         *      Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+         * @return This for chaining.
+         */
+        public Builder<T> selectorSpins(long selectorSpins) {
+            this.selectorSpins = selectorSpins;
+
+            return this;
+        }
+
+        /**
          * @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
          * @return This for chaining.
          */
@@ -2678,4 +3411,225 @@ public class GridNioServer<T> {
             return this;
         }
     }
+
+    /**
+     *
+     */
+    private class SizeBasedBalancer implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private long lastBalance;
+
+        /** */
+        private final long balancePeriod;
+
+        /**
+         * @param balancePeriod Period.
+         */
+        SizeBasedBalancer(long balancePeriod) {
+            this.balancePeriod = balancePeriod;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            long now = U.currentTimeMillis();
+
+            if (lastBalance + balancePeriod < now) {
+                lastBalance = now;
+
+                long maxRcvd0 = -1, minRcvd0 = -1, maxSent0 = -1, minSent0 = -1;
+                int maxRcvdIdx = -1, minRcvdIdx = -1, maxSentIdx = -1, minSentIdx = -1;
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    int sesCnt = worker.workerSessions.size();
+
+                    if (i % 2 == 0) {
+                        // Reader.
+                        long bytesRcvd0 = worker.bytesRcvd0;
+
+                        if ((maxRcvd0 == -1 || bytesRcvd0 > maxRcvd0) && bytesRcvd0 > 0 && sesCnt > 1) {
+                            maxRcvd0 = bytesRcvd0;
+                            maxRcvdIdx = i;
+                        }
+
+                        if (minRcvd0 == -1 || bytesRcvd0 < minRcvd0) {
+                            minRcvd0 = bytesRcvd0;
+                            minRcvdIdx = i;
+                        }
+                    }
+                    else {
+                        // Writer.
+                        long bytesSent0 = worker.bytesSent0;
+
+                        if ((maxSent0 == -1 || bytesSent0 > maxSent0) && bytesSent0 > 0 && sesCnt > 1) {
+                            maxSent0 = bytesSent0;
+                            maxSentIdx = i;
+                        }
+
+                        if (minSent0 == -1 || bytesSent0 < minSent0) {
+                            minSent0 = bytesSent0;
+                            minSentIdx = i;
+                        }
+                    }
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Balancing data [minSent0=" + minSent0 + ", minSentIdx=" + minSentIdx +
+                        ", maxSent0=" + maxSent0 + ", maxSentIdx=" + maxSentIdx +
+                        ", minRcvd0=" + minRcvd0 + ", minRcvdIdx=" + minRcvdIdx +
+                        ", maxRcvd0=" + maxRcvd0 + ", maxRcvdIdx=" + maxRcvdIdx + ']');
+
+                if (maxSent0 != -1 && minSent0 != -1) {
+                    GridSelectorNioSessionImpl ses = null;
+
+                    long sentDiff = maxSent0 - minSent0;
+                    long delta = sentDiff;
+                    double threshold = sentDiff * 0.9;
+
+                    GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+                        clientWorkers.get(maxSentIdx).workerSessions;
+
+                    for (GridSelectorNioSessionImpl ses0 : sessions) {
+                        long bytesSent0 = ses0.bytesSent0();
+
+                        if (bytesSent0 < threshold &&
+                            (ses == null || delta > U.safeAbs(bytesSent0 - sentDiff / 2))) {
+                            ses = ses0;
+                            delta = U.safeAbs(bytesSent0 - sentDiff / 2);
+                        }
+                    }
+
+                    if (ses != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Will move session to less loaded writer [ses=" + ses +
+                                ", from=" + maxSentIdx + ", to=" + minSentIdx + ']');
+
+                        moveSession(ses, maxSentIdx, minSentIdx);
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to find session to move for writers.");
+                    }
+                }
+
+                if (maxRcvd0 != -1 && minRcvd0 != -1) {
+                    GridSelectorNioSessionImpl ses = null;
+
+                    long rcvdDiff = maxRcvd0 - minRcvd0;
+                    long delta = rcvdDiff;
+                    double threshold = rcvdDiff * 0.9;
+
+                    GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+                        clientWorkers.get(maxRcvdIdx).workerSessions;
+
+                    for (GridSelectorNioSessionImpl ses0 : sessions) {
+                        long bytesRcvd0 = ses0.bytesReceived0();
+
+                        if (bytesRcvd0 < threshold &&
+                            (ses == null || delta > U.safeAbs(bytesRcvd0 - rcvdDiff / 2))) {
+                            ses = ses0;
+                            delta = U.safeAbs(bytesRcvd0 - rcvdDiff / 2);
+                        }
+                    }
+
+                    if (ses != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Will move session to less loaded reader [ses=" + ses +
+                                ", from=" + maxRcvdIdx + ", to=" + minRcvdIdx + ']');
+
+                        moveSession(ses, maxRcvdIdx, minRcvdIdx);
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to find session to move for readers.");
+                    }
+                }
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    worker.reset0();
+                }
+            }
+        }
+    }
+
+    /**
+     * For tests only.
+     */
+    @SuppressWarnings("unchecked")
+    private class RandomBalancer implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            int w1 = rnd.nextInt(clientWorkers.size());
+
+            if (clientWorkers.get(w1).workerSessions.isEmpty())
+                return;
+
+            int w2 = rnd.nextInt(clientWorkers.size());
+
+            while (w2 == w1)
+                w2 = rnd.nextInt(clientWorkers.size());
+
+            GridNioSession ses = randomSession(clientWorkers.get(w1));
+
+            if (ses != null) {
+                log.info("Move session [from=" + w1 +
+                    ", to=" + w2 +
+                    ", ses=" + ses + ']');
+
+                moveSession(ses, w1, w2);
+            }
+        }
+
+        /**
+         * @param worker Worker.
+         * @return NIO session.
+         */
+        private GridNioSession randomSession(GridNioServer.AbstractNioClientWorker worker) {
+            Collection<GridNioSession> sessions = worker.workerSessions;
+
+            int size = sessions.size();
+
+            if (size == 0)
+                return null;
+
+            int idx = ThreadLocalRandom.current().nextInt(size);
+
+            Iterator<GridNioSession> it = sessions.iterator();
+
+            int cnt = 0;
+
+            while (it.hasNext()) {
+                GridNioSession ses = it.next();
+
+                if (cnt == idx)
+                    return ses;
+            }
+
+            return null;
+        }
+
+    }
+
+    /**
+     *
+     */
+    interface SessionChangeRequest {
+        GridNioSession session();
+
+        /**
+         * @return Requested change operation.
+         */
+        NioOperation operation();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
index e4a7225..c1b60ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.nio;
 
 import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -105,6 +106,11 @@ public interface GridNioSession {
     public GridNioFuture<?> send(Object msg);
 
     /**
+     * @param msg Message to be sent.
+     */
+    public void sendNoFuture(Object msg) throws IgniteCheckedException;
+
+    /**
      * Gets metadata associated with specified key.
      *
      * @param key Key to look up.
@@ -158,10 +164,25 @@ public interface GridNioSession {
     /**
      * @param recoveryDesc Recovery descriptor.
      */
-    public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+    public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+
+    /**
+     * @param recoveryDesc Recovery descriptor.
+     */
+    public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
 
     /**
      * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
      */
-    @Nullable public GridNioRecoveryDescriptor recoveryDescriptor();
+    @Nullable public GridNioRecoveryDescriptor outRecoveryDescriptor();
+
+    /**
+     * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
+     */
+    @Nullable public GridNioRecoveryDescriptor inRecoveryDescriptor();
+
+    /**
+     * @param msg System message to send.
+     */
+    public void systemMessage(Object msg);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
index 0bcfe64..7424531 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
@@ -51,6 +51,12 @@ public class GridNioSessionImpl implements GridNioSession {
     /** Received bytes counter. */
     private volatile long bytesRcvd;
 
+    /** Sent bytes since last NIO sessions balancing. */
+    private volatile long bytesSent0;
+
+    /** Received bytes since last NIO sessions balancing. */
+    private volatile long bytesRcvd0;
+
     /** Last send schedule timestamp. */
     private volatile long sndSchedTime;
 
@@ -99,7 +105,7 @@ public class GridNioSessionImpl implements GridNioSession {
         try {
             resetSendScheduleTime();
 
-            return chain().onSessionWrite(this, msg);
+            return chain().onSessionWrite(this, msg, true);
         }
         catch (IgniteCheckedException e) {
             close();
@@ -109,6 +115,18 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
+    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+        try {
+            chain().onSessionWrite(this, msg, false);
+        }
+        catch (IgniteCheckedException e) {
+            close();
+
+            throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public GridNioFuture<?> resumeReads() {
         try {
             return chain().onResumeReads(this);
@@ -163,6 +181,28 @@ public class GridNioSessionImpl implements GridNioSession {
         return bytesRcvd;
     }
 
+    /**
+     * @return Sent bytes since last NIO sessions balancing.
+     */
+    public long bytesSent0() {
+        return bytesSent0;
+    }
+
+    /**
+     * @return Received bytes since last NIO sessions balancing.
+     */
+    public long bytesReceived0() {
+        return bytesRcvd0;
+    }
+
+    /**
+     *
+     */
+    public void reset0() {
+        bytesSent0 = 0;
+        bytesRcvd0 = 0;
+    }
+
     /** {@inheritDoc} */
     @Override public long createTime() {
         return createTime;
@@ -240,6 +280,7 @@ public class GridNioSessionImpl implements GridNioSession {
      */
     public void bytesSent(int cnt) {
         bytesSent += cnt;
+        bytesSent0 += cnt;
 
         lastSndTime = U.currentTimeMillis();
     }
@@ -253,6 +294,7 @@ public class GridNioSessionImpl implements GridNioSession {
      */
     public void bytesReceived(int cnt) {
         bytesRcvd += cnt;
+        bytesRcvd0 += cnt;
 
         lastRcvTime = U.currentTimeMillis();
     }
@@ -296,17 +338,32 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
         return null;
     }
 
     /** {@inheritDoc} */
+    @Override public void systemMessage(Object msg) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNioSessionImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
new file mode 100644
index 0000000..62985ff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+import java.util.Collection;
+import java.util.List;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+interface GridNioWorker {
+    /**
+     * @param req Change request.
+     */
+    public void offer(GridNioServer.SessionChangeRequest req);
+
+    /**
+     * @param reqs Change requests.
+     */
+    public void offer(Collection<GridNioServer.SessionChangeRequest> reqs);
+
+    /**
+     * @param ses Session.
+     * @return Session state change requests.
+     */
+    @Nullable public List<GridNioServer.SessionChangeRequest> clearSessionRequests(GridNioSession ses);
+
+    /**
+     * @param ses Session to register write interest for.
+     */
+    public void registerWrite(GridSelectorNioSessionImpl ses);
+}


[38/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
deleted file mode 100644
index 582426e..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.platform.provider.js
+++ /dev/null
@@ -1,49 +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 _ from 'lodash';
-
-const enumValueMapper = (val) => _.capitalize(val);
-
-const DFLT_CLUSTER = {
-    atomics: {
-        cacheMode: {
-            clsName: 'Apache.Ignite.Core.Cache.Configuration.CacheMode',
-            mapper: enumValueMapper
-        }
-    },
-    transactionConfiguration: {
-        defaultTxConcurrency: {
-            clsName: 'Apache.Ignite.Core.Transactions.TransactionConcurrency',
-            mapper: enumValueMapper
-        },
-        defaultTxIsolation: {
-            clsName: 'Apache.Ignite.Core.Transactions.TransactionIsolation',
-            mapper: enumValueMapper
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CLUSTER, dflts);
-    };
-
-    this.$get = ['igniteClusterDefaults', (clusterDefaults) => {
-        return _.merge({}, clusterDefaults, DFLT_CLUSTER);
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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
deleted file mode 100644
index 726581d..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/cluster.provider.js
+++ /dev/null
@@ -1,293 +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.
- */
-
-const DFLT_CLUSTER = {
-    localHost: '0.0.0.0',
-    discovery: {
-        localPort: 47500,
-        localPortRange: 100,
-        socketTimeout: 5000,
-        ackTimeout: 5000,
-        maxAckTimeout: 600000,
-        networkTimeout: 5000,
-        joinTimeout: 0,
-        threadPriority: 10,
-        heartbeatFrequency: 2000,
-        maxMissedHeartbeats: 1,
-        maxMissedClientHeartbeats: 5,
-        topHistorySize: 1000,
-        reconnectCount: 10,
-        statisticsPrintFrequency: 0,
-        ipFinderCleanFrequency: 60000,
-        forceServerMode: false,
-        clientReconnectDisabled: false,
-        Multicast: {
-            multicastGroup: '228.1.2.4',
-            multicastPort: 47400,
-            responseWaitTime: 500,
-            addressRequestAttempts: 2,
-            localAddress: '0.0.0.0'
-        },
-        Jdbc: {
-            initSchema: false
-        },
-        SharedFs: {
-            path: 'disco/tcp'
-        },
-        ZooKeeper: {
-            basePath: '/services',
-            serviceName: 'ignite',
-            allowDuplicateRegistrations: false,
-            ExponentialBackoff: {
-                baseSleepTimeMs: 1000,
-                maxRetries: 10
-            },
-            BoundedExponentialBackoffRetry: {
-                baseSleepTimeMs: 1000,
-                maxSleepTimeMs: 2147483647,
-                maxRetries: 10
-            },
-            UntilElapsed: {
-                maxElapsedTimeMs: 60000,
-                sleepMsBetweenRetries: 1000
-            },
-            RetryNTimes: {
-                n: 10,
-                sleepMsBetweenRetries: 1000
-            },
-            OneTime: {
-                sleepMsBetweenRetry: 1000
-            },
-            Forever: {
-                retryIntervalMs: 1000
-            }
-        }
-    },
-    atomics: {
-        atomicSequenceReserveSize: 1000,
-        backups: 0,
-        cacheMode: {
-            clsName: 'org.apache.ignite.cache.CacheMode',
-            value: 'PARTITIONED'
-        }
-    },
-    binary: {
-        compactFooter: true,
-        typeConfigurations: {
-            enum: false
-        }
-    },
-    collision: {
-        kind: null,
-        JobStealing: {
-            activeJobsThreshold: 95,
-            waitJobsThreshold: 0,
-            messageExpireTime: 1000,
-            maximumStealingAttempts: 5,
-            stealingEnabled: true,
-            stealingAttributes: {
-                keyClsName: 'java.lang.String',
-                valClsName: 'java.io.Serializable',
-                items: []
-            }
-        },
-        PriorityQueue: {
-            priorityAttributeKey: 'grid.task.priority',
-            jobPriorityAttributeKey: 'grid.job.priority',
-            defaultPriority: 0,
-            starvationIncrement: 1,
-            starvationPreventionEnabled: true
-        }
-    },
-    communication: {
-        localPort: 47100,
-        localPortRange: 100,
-        sharedMemoryPort: 48100,
-        directBuffer: false,
-        directSendBuffer: false,
-        idleConnectionTimeout: 30000,
-        connectTimeout: 5000,
-        maxConnectTimeout: 600000,
-        reconnectCount: 10,
-        socketSendBuffer: 32768,
-        socketReceiveBuffer: 32768,
-        messageQueueLimit: 1024,
-        tcpNoDelay: true,
-        ackSendThreshold: 16,
-        unacknowledgedMessagesBufferSize: 0,
-        socketWriteTimeout: 2000
-    },
-    networkTimeout: 5000,
-    networkSendRetryDelay: 1000,
-    networkSendRetryCount: 3,
-    discoveryStartupDelay: 60000,
-    connector: {
-        port: 11211,
-        portRange: 100,
-        idleTimeout: 7000,
-        idleQueryCursorTimeout: 600000,
-        idleQueryCursorCheckFrequency: 60000,
-        receiveBufferSize: 32768,
-        sendBufferSize: 32768,
-        sendQueueLimit: 0,
-        directBuffer: false,
-        noDelay: true,
-        sslEnabled: false,
-        sslClientAuth: false
-    },
-    deploymentMode: {
-        clsName: 'org.apache.ignite.configuration.DeploymentMode',
-        value: 'SHARED'
-    },
-    peerClassLoadingEnabled: false,
-    peerClassLoadingMissedResourcesCacheSize: 100,
-    peerClassLoadingThreadPoolSize: 2,
-    failoverSpi: {
-        JobStealing: {
-            maximumFailoverAttempts: 5
-        },
-        Always: {
-            maximumFailoverAttempts: 5
-        }
-    },
-    logger: {
-        Log4j: {
-            level: {
-                clsName: 'org.apache.log4j.Level'
-            }
-        },
-        Log4j2: {
-            level: {
-                clsName: 'org.apache.logging.log4j.Level'
-            }
-        }
-    },
-    marshalLocalJobs: false,
-    marshallerCacheKeepAliveTime: 10000,
-    metricsHistorySize: 10000,
-    metricsLogFrequency: 60000,
-    metricsUpdateFrequency: 2000,
-    clockSyncSamples: 8,
-    clockSyncFrequency: 120000,
-    timeServerPortBase: 31100,
-    timeServerPortRange: 100,
-    transactionConfiguration: {
-        defaultTxConcurrency: {
-            clsName: 'org.apache.ignite.transactions.TransactionConcurrency',
-            value: 'PESSIMISTIC'
-        },
-        defaultTxIsolation: {
-            clsName: 'org.apache.ignite.transactions.TransactionIsolation',
-            value: 'REPEATABLE_READ'
-        },
-        defaultTxTimeout: 0,
-        pessimisticTxLogLinger: 10000
-    },
-    attributes: {
-        keyClsName: 'java.lang.String',
-        valClsName: 'java.lang.String',
-        items: []
-    },
-    odbcConfiguration: {
-        endpointAddress: '0.0.0.0:10800..10810',
-        maxOpenCursors: 128
-    },
-    eventStorage: {
-        Memory: {
-            expireCount: 10000
-        }
-    },
-    checkpointSpi: {
-        S3: {
-            bucketNameSuffix: 'default-bucket',
-            clientConfiguration: {
-                protocol: {
-                    clsName: 'com.amazonaws.Protocol',
-                    value: 'HTTPS'
-                },
-                maxConnections: 50,
-                retryPolicy: {
-                    retryCondition: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    },
-                    backoffStrategy: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    },
-                    maxErrorRetry: {
-                        clsName: 'com.amazonaws.retry.PredefinedRetryPolicies'
-                    }
-                },
-                maxErrorRetry: -1,
-                socketTimeout: 50000,
-                connectionTimeout: 50000,
-                requestTimeout: 0,
-                socketSendBufferSizeHints: 0,
-                connectionTTL: -1,
-                connectionMaxIdleMillis: 60000,
-                responseMetadataCacheSize: 50,
-                useReaper: true,
-                useGzip: false,
-                preemptiveBasicProxyAuth: false,
-                useTcpKeepAlive: false
-            }
-        },
-        JDBC: {
-            checkpointTableName: 'CHECKPOINTS',
-            keyFieldName: 'NAME',
-            keyFieldType: 'VARCHAR',
-            valueFieldName: 'VALUE',
-            valueFieldType: 'BLOB',
-            expireDateFieldName: 'EXPIRE_DATE',
-            expireDateFieldType: 'DATETIME',
-            numberOfRetries: 2
-        }
-    },
-    loadBalancingSpi: {
-        RoundRobin: {
-            perTask: false
-        },
-        Adaptive: {
-            loadProbe: {
-                Job: {
-                    useAverage: true
-                },
-                CPU: {
-                    useAverage: true,
-                    useProcessors: true,
-                    processorCoefficient: 1
-                },
-                ProcessingTime: {
-                    useAverage: true
-                }
-            }
-        },
-        WeightedRandom: {
-            nodeWeight: 10,
-            useWeights: false
-        }
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_CLUSTER, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_CLUSTER;
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js b/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
deleted file mode 100644
index c556336..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/defaults/igfs.provider.js
+++ /dev/null
@@ -1,68 +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.
- */
-
-const DFLT_IGFS = {
-    defaultMode: {
-        clsName: 'org.apache.ignite.igfs.IgfsMode',
-        value: 'DUAL_ASYNC'
-    },
-    secondaryFileSystem: {
-
-    },
-    ipcEndpointConfiguration: {
-        type: {
-            clsName: 'org.apache.ignite.igfs.IgfsIpcEndpointType'
-        },
-        host: '127.0.0.1',
-        port: 10500,
-        memorySize: 262144,
-        tokenDirectoryPath: 'ipc/shmem'
-    },
-    fragmentizerConcurrentFiles: 0,
-    fragmentizerThrottlingBlockLength: 16777216,
-    fragmentizerThrottlingDelay: 200,
-    dualModeMaxPendingPutsSize: 0,
-    dualModePutExecutorServiceShutdown: false,
-    blockSize: 65536,
-    streamBufferSize: 65536,
-    maxSpaceSize: 0,
-    maximumTaskRangeLength: 0,
-    managementPort: 11400,
-    perNodeBatchSize: 100,
-    perNodeParallelBatchCount: 8,
-    prefetchBlocks: 0,
-    sequentialReadsBeforePrefetch: 0,
-    trashPurgeTimeout: 1000,
-    colocateMetadata: true,
-    relaxedConsistency: true,
-    pathModes: {
-        keyClsName: 'java.lang.String',
-        keyField: 'path',
-        valClsName: 'org.apache.ignite.igfs.IgfsMode',
-        valField: 'mode'
-    }
-};
-
-export default function() {
-    this.append = (dflts) => {
-        _.merge(DFLT_IGFS, dflts);
-    };
-
-    this.$get = [() => {
-        return DFLT_IGFS;
-    }];
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
deleted file mode 100644
index d502c8a..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-common.js
+++ /dev/null
@@ -1,625 +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.
- */
-
-// Entry point for common functions for code generation.
-const $generatorCommon = {};
-
-// Add leading zero.
-$generatorCommon.addLeadingZero = function(numberStr, minSize) {
-    if (typeof (numberStr) !== 'string')
-        numberStr = String(numberStr);
-
-    while (numberStr.length < minSize)
-        numberStr = '0' + numberStr;
-
-    return numberStr;
-};
-
-// Format date to string.
-$generatorCommon.formatDate = function(date) {
-    const dd = $generatorCommon.addLeadingZero(date.getDate(), 2);
-    const mm = $generatorCommon.addLeadingZero(date.getMonth() + 1, 2);
-
-    const yyyy = date.getFullYear();
-
-    return mm + '/' + dd + '/' + yyyy + ' ' + $generatorCommon.addLeadingZero(date.getHours(), 2) + ':' + $generatorCommon.addLeadingZero(date.getMinutes(), 2);
-};
-
-/**
- * Generate title comment for XML, Java, ... files.
- *
- * @param sbj {string} What is generated.
- * @returns {string} Text to add as title comment in generated java class.
- */
-$generatorCommon.mainComment = function mainComment(sbj) {
-    return 'This ' + sbj + ' was generated by Ignite Web Console (' + $generatorCommon.formatDate(new Date()) + ')';
-};
-
-// Create result holder with service functions and properties for XML and java code generation.
-$generatorCommon.builder = function(deep) {
-    if (_.isNil($generatorCommon.JavaTypes))
-        $generatorCommon.JavaTypes = angular.element(document.getElementById('app')).injector().get('JavaTypes');
-
-    const res = [];
-
-    res.deep = deep || 0;
-    res.needEmptyLine = false;
-    res.lineStart = true;
-    res.datasources = [];
-    res.imports = {};
-    res.staticImports = {};
-    res.vars = {};
-
-    res.safeDeep = 0;
-    res.safeNeedEmptyLine = false;
-    res.safeImports = {};
-    res.safeDatasources = [];
-    res.safePoint = -1;
-
-    res.mergeProps = function(fromRes) {
-        if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) {
-            res.datasources = fromRes.datasources;
-
-            angular.extend(res.imports, fromRes.imports);
-            angular.extend(res.staticImports, fromRes.staticImports);
-            angular.extend(res.vars, fromRes.vars);
-        }
-    };
-
-    res.mergeLines = function(fromRes) {
-        if ($generatorCommon.isDefinedAndNotEmpty(fromRes)) {
-            if (res.needEmptyLine)
-                res.push('');
-
-            _.forEach(fromRes, function(line) {
-                res.append(line);
-            });
-        }
-    };
-
-    res.startSafeBlock = function() {
-        res.safeDeep = this.deep;
-        this.safeNeedEmptyLine = this.needEmptyLine;
-        this.safeImports = _.cloneDeep(this.imports);
-        this.safeStaticImports = _.cloneDeep(this.staticImports);
-        this.safeDatasources = this.datasources.slice();
-        this.safePoint = this.length;
-    };
-
-    res.rollbackSafeBlock = function() {
-        if (this.safePoint >= 0) {
-            this.splice(this.safePoint, this.length - this.safePoint);
-
-            this.deep = res.safeDeep;
-            this.needEmptyLine = this.safeNeedEmptyLine;
-            this.datasources = this.safeDatasources;
-            this.imports = this.safeImports;
-            this.staticImports = this.safeStaticImports;
-            this.safePoint = -1;
-        }
-    };
-
-    res.asString = function() {
-        return this.join('\n');
-    };
-
-    res.append = function(s) {
-        this.push((this.lineStart ? _.repeat('    ', this.deep) : '') + s);
-
-        return this;
-    };
-
-    res.line = function(s) {
-        if (s) {
-            if (res.needEmptyLine)
-                res.push('');
-
-            res.append(s);
-        }
-
-        res.needEmptyLine = false;
-
-        res.lineStart = true;
-
-        return res;
-    };
-
-    res.startBlock = function(s) {
-        if (s) {
-            if (this.needEmptyLine)
-                this.push('');
-
-            this.append(s);
-        }
-
-        this.needEmptyLine = false;
-
-        this.lineStart = true;
-
-        this.deep++;
-
-        return this;
-    };
-
-    res.endBlock = function(s) {
-        this.deep--;
-
-        if (s)
-            this.append(s);
-
-        this.lineStart = true;
-
-        return this;
-    };
-
-    res.softEmptyLine = function() {
-        this.needEmptyLine = this.length > 0;
-    };
-
-    res.emptyLineIfNeeded = function() {
-        if (this.needEmptyLine) {
-            this.push('');
-            this.lineStart = true;
-
-            this.needEmptyLine = false;
-        }
-    };
-
-    /**
-     * Add class to imports.
-     *
-     * @param clsName Full class name.
-     * @returns {String} Short class name or full class name in case of names conflict.
-     */
-    res.importClass = function(clsName) {
-        if ($generatorCommon.JavaTypes.isJavaPrimitive(clsName))
-            return clsName;
-
-        const fullClassName = $generatorCommon.JavaTypes.fullClassName(clsName);
-
-        const dotIdx = fullClassName.lastIndexOf('.');
-
-        const shortName = dotIdx > 0 ? fullClassName.substr(dotIdx + 1) : fullClassName;
-
-        if (this.imports[shortName]) {
-            if (this.imports[shortName] !== fullClassName)
-                return fullClassName; // Short class names conflict. Return full name.
-        }
-        else
-            this.imports[shortName] = fullClassName;
-
-        return shortName;
-    };
-
-    /**
-     * Add class to imports.
-     *
-     * @param member Static member.
-     * @returns {String} Short class name or full class name in case of names conflict.
-     */
-    res.importStatic = function(member) {
-        const dotIdx = member.lastIndexOf('.');
-
-        const shortName = dotIdx > 0 ? member.substr(dotIdx + 1) : member;
-
-        if (this.staticImports[shortName]) {
-            if (this.staticImports[shortName] !== member)
-                return member; // Short class names conflict. Return full name.
-        }
-        else
-            this.staticImports[shortName] = member;
-
-        return shortName;
-    };
-
-    /**
-     * @returns String with "java imports" section.
-     */
-    res.generateImports = function() {
-        const genImports = [];
-
-        for (const clsName in this.imports) {
-            if (this.imports.hasOwnProperty(clsName) && this.imports[clsName].lastIndexOf('java.lang.', 0) !== 0)
-                genImports.push('import ' + this.imports[clsName] + ';');
-        }
-
-        genImports.sort();
-
-        return genImports.join('\n');
-    };
-
-    /**
-     * @returns String with "java imports" section.
-     */
-    res.generateStaticImports = function() {
-        const statImports = [];
-
-        for (const clsName in this.staticImports) {
-            if (this.staticImports.hasOwnProperty(clsName) && this.staticImports[clsName].lastIndexOf('java.lang.', 0) !== 0)
-                statImports.push('import static ' + this.staticImports[clsName] + ';');
-        }
-
-        statImports.sort();
-
-        return statImports.join('\n');
-    };
-
-    return res;
-};
-
-// Eviction policies code generation descriptors.
-$generatorCommon.EVICTION_POLICIES = {
-    LRU: {
-        className: 'org.apache.ignite.cache.eviction.lru.LruEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    },
-    FIFO: {
-        className: 'org.apache.ignite.cache.eviction.fifo.FifoEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    },
-    SORTED: {
-        className: 'org.apache.ignite.cache.eviction.sorted.SortedEvictionPolicy',
-        fields: {batchSize: {dflt: 1}, maxMemorySize: null, maxSize: {dflt: 100000}}
-    }
-};
-
-// Marshaller code generation descriptors.
-$generatorCommon.MARSHALLERS = {
-    OptimizedMarshaller: {
-        className: 'org.apache.ignite.marshaller.optimized.OptimizedMarshaller',
-        fields: {poolSize: null, requireSerializable: null }
-    },
-    JdkMarshaller: {
-        className: 'org.apache.ignite.marshaller.jdk.JdkMarshaller',
-        fields: {}
-    }
-};
-
-// Pairs of supported databases and their JDBC dialects.
-$generatorCommon.JDBC_DIALECTS = {
-    Generic: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect',
-    Oracle: 'org.apache.ignite.cache.store.jdbc.dialect.OracleDialect',
-    DB2: 'org.apache.ignite.cache.store.jdbc.dialect.DB2Dialect',
-    SQLServer: 'org.apache.ignite.cache.store.jdbc.dialect.SQLServerDialect',
-    MySQL: 'org.apache.ignite.cache.store.jdbc.dialect.MySQLDialect',
-    PostgreSQL: 'org.apache.ignite.cache.store.jdbc.dialect.BasicJdbcDialect',
-    H2: 'org.apache.ignite.cache.store.jdbc.dialect.H2Dialect'
-};
-
-// Return JDBC dialect full class name for specified database.
-$generatorCommon.jdbcDialectClassName = function(db) {
-    const dialectClsName = $generatorCommon.JDBC_DIALECTS[db];
-
-    return dialectClsName ? dialectClsName : 'Unknown database: ' + db;
-};
-
-// Generate default data cache for specified igfs instance.
-$generatorCommon.igfsDataCache = function(igfs) {
-    return {
-        name: igfs.name + '-data',
-        cacheMode: 'PARTITIONED',
-        atomicityMode: 'TRANSACTIONAL',
-        writeSynchronizationMode: 'FULL_SYNC',
-        backups: 0,
-        igfsAffinnityGroupSize: igfs.affinnityGroupSize || 512
-    };
-};
-
-// Generate default meta cache for specified igfs instance.
-$generatorCommon.igfsMetaCache = function(igfs) {
-    return {
-        name: igfs.name + '-meta',
-        cacheMode: 'REPLICATED',
-        atomicityMode: 'TRANSACTIONAL',
-        writeSynchronizationMode: 'FULL_SYNC'
-    };
-};
-
-// Pairs of supported databases and their data sources.
-$generatorCommon.DATA_SOURCES = {
-    Generic: 'com.mchange.v2.c3p0.ComboPooledDataSource',
-    Oracle: 'oracle.jdbc.pool.OracleDataSource',
-    DB2: 'com.ibm.db2.jcc.DB2DataSource',
-    SQLServer: 'com.microsoft.sqlserver.jdbc.SQLServerDataSource',
-    MySQL: 'com.mysql.jdbc.jdbc2.optional.MysqlDataSource',
-    PostgreSQL: 'org.postgresql.ds.PGPoolingDataSource',
-    H2: 'org.h2.jdbcx.JdbcDataSource'
-};
-
-// Return data source full class name for specified database.
-$generatorCommon.dataSourceClassName = function(db) {
-    const dsClsName = $generatorCommon.DATA_SOURCES[db];
-
-    return dsClsName ? dsClsName : 'Unknown database: ' + db;
-};
-
-// Store factories code generation descriptors.
-$generatorCommon.STORE_FACTORIES = {
-    CacheJdbcPojoStoreFactory: {
-        className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory',
-        suffix: 'JdbcPojo',
-        fields: {
-            configuration: {type: 'bean'}
-        }
-    },
-    CacheJdbcBlobStoreFactory: {
-        className: 'org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory',
-        suffix: 'JdbcBlob',
-        fields: {
-            initSchema: null,
-            createTableQuery: null,
-            loadQuery: null,
-            insertQuery: null,
-            updateQuery: null,
-            deleteQuery: null
-        }
-    },
-    CacheHibernateBlobStoreFactory: {
-        className: 'org.apache.ignite.cache.store.hibernate.CacheHibernateBlobStoreFactory',
-        suffix: 'Hibernate',
-        fields: {hibernateProperties: {type: 'propertiesAsList', propVarName: 'props'}}
-    }
-};
-
-// Swap space SPI code generation descriptor.
-$generatorCommon.SWAP_SPACE_SPI = {
-    className: 'org.apache.ignite.spi.swapspace.file.FileSwapSpaceSpi',
-    fields: {
-        baseDirectory: {type: 'path'},
-        readStripesNumber: null,
-        maximumSparsity: {type: 'float'},
-        maxWriteQueueSize: null,
-        writeBufferSize: null
-    }
-};
-
-// Transaction configuration code generation descriptor.
-$generatorCommon.TRANSACTION_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.TransactionConfiguration',
-    fields: {
-        defaultTxConcurrency: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionConcurrency', dflt: 'PESSIMISTIC'},
-        defaultTxIsolation: {type: 'enum', enumClass: 'org.apache.ignite.transactions.TransactionIsolation', dflt: 'REPEATABLE_READ'},
-        defaultTxTimeout: {dflt: 0},
-        pessimisticTxLogLinger: {dflt: 10000},
-        pessimisticTxLogSize: null,
-        txSerializableEnabled: null,
-        txManagerFactory: {type: 'bean'}
-    }
-};
-
-// SSL configuration code generation descriptor.
-$generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY = {
-    className: 'org.apache.ignite.ssl.SslContextFactory',
-    fields: {
-        keyAlgorithm: null,
-        keyStoreFilePath: {type: 'path'},
-        keyStorePassword: {type: 'raw'},
-        keyStoreType: null,
-        protocol: null,
-        trustStoreFilePath: {type: 'path'},
-        trustStorePassword: {type: 'raw'},
-        trustStoreType: null
-    }
-};
-
-// SSL configuration code generation descriptor.
-$generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY = {
-    className: 'org.apache.ignite.ssl.SslContextFactory',
-    fields: {
-        keyAlgorithm: null,
-        keyStoreFilePath: {type: 'path'},
-        keyStorePassword: {type: 'raw'},
-        keyStoreType: null,
-        protocol: null,
-        trustManagers: {type: 'array'}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.CONNECTOR_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.ConnectorConfiguration',
-    fields: {
-        jettyPath: null,
-        host: null,
-        port: {dflt: 11211},
-        portRange: {dflt: 100},
-        idleTimeout: {dflt: 7000},
-        idleQueryCursorTimeout: {dflt: 600000},
-        idleQueryCursorCheckFrequency: {dflt: 60000},
-        receiveBufferSize: {dflt: 32768},
-        sendBufferSize: {dflt: 32768},
-        sendQueueLimit: {dflt: 0},
-        directBuffer: {dflt: false},
-        noDelay: {dflt: true},
-        selectorCount: null,
-        threadPoolSize: null,
-        messageInterceptor: {type: 'bean'},
-        secretKey: null,
-        sslEnabled: {dflt: false}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.COMMUNICATION_CONFIGURATION = {
-    className: 'org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi',
-    fields: {
-        listener: {type: 'bean'},
-        localAddress: null,
-        localPort: {dflt: 47100},
-        localPortRange: {dflt: 100},
-        sharedMemoryPort: {dflt: 48100},
-        directBuffer: {dflt: false},
-        directSendBuffer: {dflt: false},
-        idleConnectionTimeout: {dflt: 30000},
-        connectTimeout: {dflt: 5000},
-        maxConnectTimeout: {dflt: 600000},
-        reconnectCount: {dflt: 10},
-        socketSendBuffer: {dflt: 32768},
-        socketReceiveBuffer: {dflt: 32768},
-        messageQueueLimit: {dflt: 1024},
-        slowClientQueueLimit: null,
-        tcpNoDelay: {dflt: true},
-        ackSendThreshold: {dflt: 16},
-        unacknowledgedMessagesBufferSize: {dflt: 0},
-        socketWriteTimeout: {dflt: 2000},
-        selectorsCount: null,
-        addressResolver: {type: 'bean'}
-    }
-};
-
-// Communication configuration code generation descriptor.
-$generatorCommon.IGFS_IPC_CONFIGURATION = {
-    className: 'org.apache.ignite.igfs.IgfsIpcEndpointConfiguration',
-    fields: {
-        type: {type: 'enum', enumClass: 'org.apache.ignite.igfs.IgfsIpcEndpointType'},
-        host: {dflt: '127.0.0.1'},
-        port: {dflt: 10500},
-        memorySize: {dflt: 262144},
-        tokenDirectoryPath: {dflt: 'ipc/shmem'},
-        threadCount: null
-    }
-};
-
-$generatorCommon.ODBC_CONFIGURATION = {
-    className: 'org.apache.ignite.configuration.OdbcConfiguration',
-    fields: {
-        endpointAddress: {dflt: '0.0.0.0:10800..10810'},
-        maxOpenCursors: {dflt: 128}
-    }
-};
-
-// Check that cache has datasource.
-$generatorCommon.cacheHasDatasource = function(cache) {
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const storeFactory = cache.cacheStoreFactory[cache.cacheStoreFactory.kind];
-
-        return !!(storeFactory && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : false) : storeFactory.dialect)); // eslint-disable-line no-nested-ternary
-    }
-
-    return false;
-};
-
-$generatorCommon.secretPropertiesNeeded = function(cluster) {
-    return !_.isNil(_.find(cluster.caches, $generatorCommon.cacheHasDatasource)) || cluster.sslEnabled;
-};
-
-// Check that binary is configured.
-$generatorCommon.binaryIsDefined = function(binary) {
-    return binary && ($generatorCommon.isDefinedAndNotEmpty(binary.idMapper) || $generatorCommon.isDefinedAndNotEmpty(binary.nameMapper) ||
-        $generatorCommon.isDefinedAndNotEmpty(binary.serializer) || $generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations) ||
-        (!_.isNil(binary.compactFooter) && !binary.compactFooter));
-};
-
-// Extract domain model metadata location.
-$generatorCommon.domainQueryMetadata = function(domain) {
-    return domain.queryMetadata ? domain.queryMetadata : 'Configuration';
-};
-
-/**
- * @param {Object} obj Object to check.
- * @param {Array<String>} props Array of properties names.
- * @returns {boolean} 'true' if
- */
-$generatorCommon.hasAtLeastOneProperty = function(obj, props) {
-    return obj && props && _.findIndex(props, (prop) => !_.isNil(obj[prop])) >= 0;
-};
-
-/**
- * Convert some name to valid java name.
- *
- * @param prefix To append to java name.
- * @param name to convert.
- * @returns {string} Valid java name.
- */
-$generatorCommon.toJavaName = function(prefix, name) {
-    const javaName = name ? name.replace(/[^A-Za-z_0-9]+/g, '_') : 'dflt';
-
-    return prefix + javaName.charAt(0).toLocaleUpperCase() + javaName.slice(1);
-};
-
-/**
- * @param v Value to check.
- * @returns {boolean} 'true' if value defined and not empty string.
- */
-$generatorCommon.isDefinedAndNotEmpty = function(v) {
-    let defined = !_.isNil(v);
-
-    if (defined && (_.isString(v) || _.isArray(v)))
-        defined = v.length > 0;
-
-    return defined;
-};
-
-/**
- * @param {Object} obj Object to check.
- * @param {Array<String>} props Properties names.
- * @returns {boolean} 'true' if object contains at least one from specified properties.
- */
-$generatorCommon.hasProperty = function(obj, props) {
-    for (const propName in props) {
-        if (props.hasOwnProperty(propName)) {
-            if (obj[propName])
-                return true;
-        }
-    }
-
-    return false;
-};
-
-/**
- * Get class for selected implementation of Failover SPI.
- *
- * @param spi Failover SPI configuration.
- * @returns {*} Class for selected implementation of Failover SPI.
- */
-$generatorCommon.failoverSpiClass = function(spi) {
-    switch (spi.kind) {
-        case 'JobStealing': return 'org.apache.ignite.spi.failover.jobstealing.JobStealingFailoverSpi';
-        case 'Never': return 'org.apache.ignite.spi.failover.never.NeverFailoverSpi';
-        case 'Always': return 'org.apache.ignite.spi.failover.always.AlwaysFailoverSpi';
-        case 'Custom': return _.get(spi, 'Custom.class');
-        default: return 'Unknown';
-    }
-};
-
-$generatorCommon.loggerConfigured = function(logger) {
-    if (logger && logger.kind) {
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2': return log && $generatorCommon.isDefinedAndNotEmpty(log.path);
-
-            case 'Log4j':
-                if (!log || !log.mode)
-                    return false;
-
-                if (log.mode === 'Path')
-                    return $generatorCommon.isDefinedAndNotEmpty(log.path);
-
-                return true;
-
-            case 'Custom': return log && $generatorCommon.isDefinedAndNotEmpty(log.class);
-
-            default:
-                return true;
-        }
-    }
-
-    return false;
-};
-
-export default $generatorCommon;


[18/50] [abbrv] ignite git commit: ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response - enabled striped pool usage for GridDistributedTxFinishResponse - convert exceptions from CacheStoreSessionListener.onSession

Posted by vo...@apache.org.
ignite-4371 Neat TX finish request processing may fall into sync wait of dht finish response
- enabled striped pool usage for GridDistributedTxFinishResponse
- convert exceptions from CacheStoreSessionListener.onSessionStart into IgniteCheckedException
- should not use IgniteTxLocalAdapter.commitErr for rollback exceptions (otherwise cyclic reference in exception->cause is possible, this leads to infinite loop in 'X.hasCause')

(cherry picked from commit cf7b6a0)


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

Branch: refs/heads/ignite-2.0
Commit: 147277d89fcdbd134425b74fc0b63596572a4af1
Parents: c1fde05
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 14:49:25 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 15:17:38 2016 +0300

----------------------------------------------------------------------
 .../GridDistributedTxFinishResponse.java        |  6 ----
 .../cache/store/CacheStoreManager.java          |  4 ++-
 .../store/GridCacheStoreManagerAdapter.java     | 34 ++++++++++++--------
 .../cache/store/GridCacheWriteBehindStore.java  |  6 ++--
 .../transactions/IgniteTxLocalAdapter.java      | 27 ++++++----------
 5 files changed, 36 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/147277d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index c5cf332..109d665 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,12 +86,6 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public int partition() {
-        // TODO https://issues.apache.org/jira/browse/IGNITE-4371
-        return Integer.MIN_VALUE;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/147277d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
index 67c9334..8d6b63d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/CacheStoreManager.java
@@ -171,8 +171,10 @@ public interface CacheStoreManager<K, V> extends GridCacheManager<K, V> {
 
     /**
      * End session initiated by write-behind store.
+     *
+     * @throws IgniteCheckedException If failed.
      */
-    public void writeBehindSessionInit();
+    public void writeBehindSessionInit() throws IgniteCheckedException;
 
     /**
      * End session initiated by write-behind store.

http://git-wip-us.apache.org/repos/asf/ignite/blob/147277d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index 024375e..8b93afa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -288,7 +288,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable public Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
+    @Override @Nullable public final Object load(@Nullable IgniteInternalTx tx, KeyCacheObject key)
         throws IgniteCheckedException {
         return loadFromStore(tx, key, true);
     }
@@ -377,7 +377,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
+    @Override public final void localStoreLoadAll(@Nullable IgniteInternalTx tx, Collection keys, GridInClosure3 vis)
         throws IgniteCheckedException {
         assert store != null;
         assert locStore;
@@ -386,7 +386,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
+    @Override public final boolean loadAll(@Nullable IgniteInternalTx tx, Collection keys, IgniteBiInClosure vis)
         throws IgniteCheckedException {
         if (store != null) {
             loadAllFromStore(tx, keys, vis, null);
@@ -499,7 +499,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
+    @Override public final boolean loadCache(final GridInClosure3 vis, Object[] args) throws IgniteCheckedException {
         if (store != null) {
             if (log.isDebugEnabled())
                 log.debug("Loading all values from store.");
@@ -554,7 +554,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
+    @Override public final boolean put(@Nullable IgniteInternalTx tx, Object key, Object val, GridCacheVersion ver)
         throws IgniteCheckedException {
         if (store != null) {
             // Never persist internal keys.
@@ -599,7 +599,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
+    @Override public final boolean putAll(@Nullable IgniteInternalTx tx, Map map) throws IgniteCheckedException {
         if (F.isEmpty(map))
             return true;
 
@@ -658,7 +658,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
+    @Override public final boolean remove(@Nullable IgniteInternalTx tx, Object key) throws IgniteCheckedException {
         if (store != null) {
             // Never remove internal key from store as it is never persisted.
             if (key instanceof GridCacheInternal)
@@ -701,7 +701,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
+    @Override public final boolean removeAll(@Nullable IgniteInternalTx tx, Collection keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))
             return true;
 
@@ -763,7 +763,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
+    @Override public final void sessionEnd(IgniteInternalTx tx, boolean commit, boolean last) throws IgniteCheckedException {
         assert store != null;
 
         sessionInit0(tx);
@@ -807,7 +807,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
     }
 
     /** {@inheritDoc} */
-    @Override public void writeBehindSessionInit() {
+    @Override public void writeBehindSessionInit() throws IgniteCheckedException {
         sessionInit0(null);
     }
 
@@ -818,8 +818,9 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
     /**
      * @param tx Current transaction.
+     * @throws IgniteCheckedException If failed.
      */
-    private void sessionInit0(@Nullable IgniteInternalTx tx) {
+    private void sessionInit0(@Nullable IgniteInternalTx tx) throws IgniteCheckedException {
         assert sesHolder != null;
 
         SessionData ses;
@@ -841,9 +842,14 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
 
         sesHolder.set(ses);
 
-        if (sesLsnrs != null && !ses.started(this)) {
-            for (CacheStoreSessionListener lsnr : sesLsnrs)
-                lsnr.onSessionStart(locSes);
+        try {
+            if (sesLsnrs != null && !ses.started(this)) {
+                for (CacheStoreSessionListener lsnr : sesLsnrs)
+                    lsnr.onSessionStart(locSes);
+            }
+        }
+        catch (Exception e) {
+            throw new IgniteCheckedException("Failed to start store session: " + e, e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/147277d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 858d9a7..f5c0c12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -673,10 +673,10 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
         Map<K, Entry<? extends K, ? extends  V>> vals,
         boolean initSes) {
 
-        if (initSes && storeMgr != null)
-            storeMgr.writeBehindSessionInit();
-
         try {
+            if (initSes && storeMgr != null)
+                storeMgr.writeBehindSessionInit();
+
             boolean threwEx = true;
 
             try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/147277d8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 8d0a2b1..9f794b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1063,7 +1063,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
             setRollbackOnly();
 
             throw new IgniteCheckedException("Invalid transaction state for rollback [state=" + state +
-                ", tx=" + this + ']', commitErr);
+                ", tx=" + this + ']');
         }
 
         if (near()) {
@@ -1074,28 +1074,21 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
         }
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {
-            try {
-                cctx.tm().rollbackTx(this);
+            cctx.tm().rollbackTx(this);
 
-                if (!internal()) {
-                    Collection<CacheStoreManager> stores = txState.stores(cctx);
+            if (!internal()) {
+                Collection<CacheStoreManager> stores = txState.stores(cctx);
 
-                    if (stores != null && !stores.isEmpty()) {
-                        assert isWriteToStoreFromDhtValid(stores) :
-                            "isWriteToStoreFromDht can't be different within one transaction";
+                if (stores != null && !stores.isEmpty()) {
+                    assert isWriteToStoreFromDhtValid(stores) :
+                        "isWriteToStoreFromDht can't be different within one transaction";
 
-                        boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
+                    boolean isWriteToStoreFromDht = F.first(stores).isWriteToStoreFromDht();
 
-                        if (stores != null && !stores.isEmpty() && (near() || isWriteToStoreFromDht))
-                            sessionEnd(stores, false);
-                    }
+                    if (!stores.isEmpty() && (near() || isWriteToStoreFromDht))
+                        sessionEnd(stores, false);
                 }
             }
-            catch (Error | IgniteCheckedException | RuntimeException e) {
-                U.addLastCause(e, commitErr, log);
-
-                throw e;
-            }
         }
     }
 


[16/50] [abbrv] ignite git commit: ignite-2412 Do not call 'asyncOp' for synchronous operations

Posted by vo...@apache.org.
ignite-2412 Do not call 'asyncOp' for synchronous operations

(cherry picked from commit c530d47)


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

Branch: refs/heads/ignite-2.0
Commit: ad785cbd192ca3f34d62bc2155f61a74f4962102
Parents: 28dab6e
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 16 19:23:29 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 11:40:18 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 721 +++++++------------
 .../processors/cache/IgniteCacheProxy.java      |   8 -
 .../dht/atomic/GridDhtAtomicCache.java          | 472 +++++++-----
 .../dht/colocated/GridDhtColocatedCache.java    |  13 -
 .../distributed/near/GridNearAtomicCache.java   |  65 +-
 .../local/atomic/GridLocalAtomicCache.java      | 177 +----
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |   2 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |  14 +-
 8 files changed, 597 insertions(+), 875 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index a8d9f1d..5f0b8a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -248,16 +248,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** Grid configuration. */
     @GridToStringExclude
-    protected IgniteConfiguration gridCfg;
+    private IgniteConfiguration gridCfg;
 
     /** Cache metrics. */
     protected CacheMetricsImpl metrics;
 
     /** Cache localMxBean. */
-    protected CacheMetricsMXBean localMxBean;
+    private CacheMetricsMXBean locMxBean;
 
     /** Cache mxBean. */
-    protected CacheMetricsMXBean clusterMxBean;
+    private CacheMetricsMXBean clusterMxBean;
 
     /** Logger. */
     protected IgniteLogger log;
@@ -288,9 +288,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** Asynchronous operations limit semaphore. */
     private Semaphore asyncOpsSem;
 
-    /** */
-    protected volatile boolean asyncToggled;
-
     /** {@inheritDoc} */
     @Override public String name() {
         return cacheCfg.getName();
@@ -334,7 +331,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         metrics = new CacheMetricsImpl(ctx);
 
-        localMxBean = new CacheLocalMetricsMXBeanImpl(this);
+        locMxBean = new CacheLocalMetricsMXBeanImpl(this);
         clusterMxBean = new CacheClusterMetricsMXBeanImpl(this);
 
         FileSystemConfiguration[] igfsCfgs = gridCfg.getFileSystemConfiguration();
@@ -367,18 +364,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
-     * Toggles async flag if someone calls {@code withAsync()}
-     * on proxy and since that we have to properly handle all cache
-     * operations (sync and async) to put them in proper sequence.
-     *
-     * TODO: https://issues.apache.org/jira/browse/IGNITE-4393
-     */
-    void toggleAsync() {
-        if (!asyncToggled)
-            asyncToggled = true;
-    }
-
-    /**
      * Prints memory stats.
      */
     public void printMemoryStats() {
@@ -471,49 +456,49 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     public abstract GridCachePreloader preloader();
 
     /** {@inheritDoc} */
-    @Override public Affinity<K> affinity() {
+    @Override public final Affinity<K> affinity() {
         return aff;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantCast"})
-    @Override public <K1, V1> IgniteInternalCache<K1, V1> cache() {
+    @Override public final <K1, V1> IgniteInternalCache<K1, V1> cache() {
         return (IgniteInternalCache<K1, V1>)this;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> forSubjectId(UUID subjId) {
+    @Override public final GridCacheProxyImpl<K, V> forSubjectId(UUID subjId) {
         CacheOperationContext opCtx = new CacheOperationContext(false, subjId, false, null, false, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean skipStore() {
+    @Override public final boolean skipStore() {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> setSkipStore(boolean skipStore) {
+    @Override public final GridCacheProxyImpl<K, V> setSkipStore(boolean skipStore) {
         CacheOperationContext opCtx = new CacheOperationContext(true, null, false, null, false, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public <K1, V1> GridCacheProxyImpl<K1, V1> keepBinary() {
+    @Override public final <K1, V1> GridCacheProxyImpl<K1, V1> keepBinary() {
         CacheOperationContext opCtx = new CacheOperationContext(false, null, true, null, false, null);
 
         return new GridCacheProxyImpl<>((GridCacheContext<K1, V1>)ctx, (GridCacheAdapter<K1, V1>)this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public ExpiryPolicy expiry() {
+    @Nullable @Override public final ExpiryPolicy expiry() {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
+    @Override public final GridCacheProxyImpl<K, V> withExpiryPolicy(ExpiryPolicy plc) {
         assert !CU.isUtilityCache(ctx.name());
         assert !CU.isAtomicsCache(ctx.name());
         assert !CU.isMarshallerCache(ctx.name());
@@ -524,14 +509,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalCache<K, V> withNoRetries() {
+    @Override public final IgniteInternalCache<K, V> withNoRetries() {
         CacheOperationContext opCtx = new CacheOperationContext(false, null, false, null, true, null);
 
         return new GridCacheProxyImpl<>(ctx, this, opCtx);
     }
 
     /** {@inheritDoc} */
-    @Override public CacheConfiguration configuration() {
+    @Override public final CacheConfiguration configuration() {
         return ctx.config();
     }
 
@@ -630,7 +615,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean isEmpty() {
+    @Override public final boolean isEmpty() {
         try {
             return localSize(CachePeekModes.ONHEAP_ONLY) == 0;
         }
@@ -640,7 +625,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean containsKey(K key) {
+    @Override public final boolean containsKey(K key) {
         try {
             return containsKeyAsync(key).get();
         }
@@ -667,7 +652,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean containsKeys(Collection<? extends K> keys) {
+    @Override public final boolean containsKeys(Collection<? extends K> keys) {
         try {
             return containsKeysAsync(keys).get();
         }
@@ -677,7 +662,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> containsKeysAsync(final Collection<? extends K> keys) {
+    @Override public final IgniteInternalFuture<Boolean> containsKeysAsync(final Collection<? extends K> keys) {
         A.notNull(keys, "keys");
 
         return getAllAsync(
@@ -708,7 +693,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Iterable<Cache.Entry<K, V>> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException {
+    @Override public final Iterable<Cache.Entry<K, V>> localEntries(CachePeekMode[] peekModes) throws IgniteCheckedException {
         assert peekModes != null;
 
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
@@ -765,7 +750,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    @Nullable @Override public V localPeek(K key,
+    @Nullable @Override public final V localPeek(K key,
         CachePeekMode[] peekModes,
         @Nullable IgniteCacheExpiryPolicy plc)
         throws IgniteCheckedException {
@@ -907,7 +892,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      *
      * @param ldr Class loader to undeploy.
      */
-    public void onUndeploy(ClassLoader ldr) {
+    public final void onUndeploy(ClassLoader ldr) {
         ctx.deploy().onUndeploy(ldr, context());
     }
 
@@ -916,7 +901,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry or <tt>null</tt>.
      */
-    @Nullable public GridCacheEntryEx peekEx(KeyCacheObject key) {
+    @Nullable public final GridCacheEntryEx peekEx(KeyCacheObject key) {
         return entry0(key, ctx.affinity().affinityTopologyVersion(), false, false);
     }
 
@@ -925,7 +910,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry or <tt>null</tt>.
      */
-    @Nullable public GridCacheEntryEx peekEx(Object key) {
+    @Nullable public final GridCacheEntryEx peekEx(Object key) {
         return entry0(ctx.toCacheKeyObject(key), ctx.affinity().affinityTopologyVersion(), false, false);
     }
 
@@ -933,7 +918,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry (never {@code null}).
      */
-    public GridCacheEntryEx entryEx(Object key) {
+    public final GridCacheEntryEx entryEx(Object key) {
         return entryEx(ctx.toCacheKeyObject(key), false);
     }
 
@@ -941,7 +926,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Entry key.
      * @return Entry (never {@code null}).
      */
-    public GridCacheEntryEx entryEx(KeyCacheObject key) {
+    public final GridCacheEntryEx entryEx(KeyCacheObject key) {
         return entryEx(key, false);
     }
 
@@ -996,24 +981,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @return Set of internal cached entry representations.
      */
-    public Iterable<? extends GridCacheEntryEx> entries() {
+    public final Iterable<? extends GridCacheEntryEx> entries() {
         return allEntries();
     }
 
     /**
      * @return Set of internal cached entry representations.
      */
-    public Iterable<? extends GridCacheEntryEx> allEntries() {
+    public final Iterable<? extends GridCacheEntryEx> allEntries() {
         return map.entries();
     }
 
     /** {@inheritDoc} */
-    @Override public Set<Cache.Entry<K, V>> entrySet() {
+    @Override public final Set<Cache.Entry<K, V>> entrySet() {
         return entrySet((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
-    @Override public Set<Cache.Entry<K, V>> entrySetx(final CacheEntryPredicate... filter) {
+    @Override public final Set<Cache.Entry<K, V>> entrySetx(final CacheEntryPredicate... filter) {
         boolean keepBinary = ctx.keepBinary();
 
         return new EntrySet(map.entrySet(filter), keepBinary);
@@ -1025,17 +1010,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> keySet() {
+    @Override public final Set<K> keySet() {
         return new KeySet(map.entrySet());
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> keySetx() {
+    @Override public final Set<K> keySetx() {
         return keySet();
     }
 
     /** {@inheritDoc} */
-    @Override public Set<K> primaryKeySet() {
+    @Override public final Set<K> primaryKeySet() {
         return new KeySet(map.entrySet(CU.cachePrimary(ctx.grid().affinity(ctx.name()), ctx.localNode())));
     }
 
@@ -1057,7 +1042,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filters.
      * @return Collection of cached values.
      */
-    public Iterable<V> values(final CacheEntryPredicate... filter) {
+    public final Iterable<V> values(final CacheEntryPredicate... filter) {
         return new Iterable<V>() {
             @Override public Iterator<V> iterator() {
                 return new Iterator<V>() {
@@ -1083,12 +1068,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      *
      * @param key Entry key.
      */
-    public void removeIfObsolete(KeyCacheObject key) {
+    public final void removeIfObsolete(KeyCacheObject key) {
         assert key != null;
 
         GridCacheMapEntry entry = map.getEntry(key);
 
-        if (entry.obsolete())
+        if (entry != null && entry.obsolete())
             removeEntry(entry);
     }
 
@@ -1272,11 +1257,11 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * @param entry Removes entry from cache if currently mapped value is the same as passed.
      */
-    public void removeEntry(GridCacheEntryEx entry) {
-        boolean removed = map.removeEntry(entry);
+    public final void removeEntry(GridCacheEntryEx entry) {
+        boolean rmvd = map.removeEntry(entry);
 
         if (log.isDebugEnabled()) {
-            if (removed)
+            if (rmvd)
                 log.debug("Removed entry from cache: " + entry);
             else
                 log.debug("Remove will not be done for key (entry got replaced or removed): " + entry.key());
@@ -1311,7 +1296,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public V getForcePrimary(K key) throws IgniteCheckedException {
+    @Override public final V getForcePrimary(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1328,7 +1313,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
+    @Override public final IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1349,7 +1334,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    public V getTopologySafe(K key) throws IgniteCheckedException {
+    public final V getTopologySafe(K key) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(
@@ -1366,12 +1351,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
+    @Nullable @Override public final Map<K, V> getAllOutTx(Set<? extends K> keys) throws IgniteCheckedException {
         return getAllOutTxAsync(keys).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
+    @Override public final IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(keys,
@@ -1385,15 +1370,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             false);
     }
 
-    /**
-     * @param key Key.
-     * @param topVer Topology version.
-     * @return Entry.
-     */
-    @Nullable protected GridCacheEntryEx entryExSafe(KeyCacheObject key, AffinityTopologyVersion topVer) {
-        return entryEx(key);
-    }
-
     /** {@inheritDoc} */
     @Nullable @Override public V get(K key) throws IgniteCheckedException {
         A.notNull(key, "key");
@@ -1533,14 +1509,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+    @Override public final Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
         A.notNull(keys, "keys");
 
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        Map<K, V> map = getAll(keys, !ctx.keepBinary(), false);
+        Map<K, V> map = getAll0(keys, !ctx.keepBinary(), false);
 
         if (ctx.config().getInterceptor() != null)
             map = interceptGet(keys, map);
@@ -1560,7 +1536,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        Map<K, T2<V, GridCacheVersion>> map = (Map<K, T2<V, GridCacheVersion>>)getAll(keys, !ctx.keepBinary(), true);
+        Map<K, T2<V, GridCacheVersion>> map = (Map<K, T2<V, GridCacheVersion>>)getAll0(keys, !ctx.keepBinary(), true);
 
         Collection<CacheEntry<K, V>> res = new HashSet<>();
 
@@ -1875,7 +1851,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param needVer If {@code true} returns values as tuples containing value and version.
      * @return Future.
      */
-    public final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(
+    protected final <K1, V1> IgniteInternalFuture<Map<K1, V1>> getAllAsync0(
         @Nullable final Collection<KeyCacheObject> keys,
         final boolean readThrough,
         boolean checkTx,
@@ -2138,7 +2114,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPut(K key, V val) throws IgniteCheckedException {
+    @Override public final V getAndPut(K key, V val) throws IgniteCheckedException {
         return getAndPut(key, val, null);
     }
 
@@ -2160,7 +2136,24 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        V prevVal = syncOp(new SyncOp<V>(true) {
+        V prevVal = getAndPut0(key, val, filter);
+
+        if (statsEnabled)
+            metrics0().addPutAndGetTimeNanos(System.nanoTime() - start);
+
+        return prevVal;
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param filter Optional filter.
+     * @return Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected V getAndPut0(final K key, final V val, @Nullable final CacheEntryPredicate filter)
+        throws IgniteCheckedException {
+        return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return (V)tx.putAsync(ctx, null, key, val, true, filter).get().value();
             }
@@ -2169,15 +2162,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "put [key=" + key + ", val=" + val + ", filter=" + filter + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addPutAndGetTimeNanos(System.nanoTime() - start);
-
-        return prevVal;
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutAsync(K key, V val) {
+    @Override public final IgniteInternalFuture<V> getAndPutAsync(K key, V val) {
         return getAndPutAsync(key, val, null);
     }
 
@@ -2187,11 +2175,16 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filter.
      * @return Put operation future.
      */
-    public IgniteInternalFuture<V> getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+    protected final IgniteInternalFuture<V> getAndPutAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
+        A.notNull(key, "key", val, "val");
+
+        if (keyCheck)
+            validateCacheKey(key);
+
         IgniteInternalFuture<V> fut = getAndPutAsync0(key, val, filter);
 
         if (statsEnabled)
@@ -2206,13 +2199,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Optional filter.
      * @return Put operation future.
      */
-    public IgniteInternalFuture<V> getAndPutAsync0(final K key, final V val,
-        @Nullable final CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
+    public IgniteInternalFuture<V> getAndPutAsync0(final K key,
+        final V val,
+        @Nullable final CacheEntryPredicate filter)
+    {
         return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAsync(ctx, readyTopVer, key, val, true, filter)
@@ -2226,7 +2216,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(final K key, final V val) throws IgniteCheckedException {
+    @Override public final boolean put(final K key, final V val) throws IgniteCheckedException {
         return put(key, val, null);
     }
 
@@ -2250,7 +2240,26 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        Boolean stored = syncOp(new SyncOp<Boolean>(true) {
+        boolean stored = put0(key, val, filter);
+
+        if (statsEnabled && stored)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
+
+        return stored;
+    }
+
+    /**
+     * @param key Key.
+     * @param val Value.
+     * @param filter Filter.
+     * @return {@code True} if optional filter passed and value was stored in cache,
+     *      {@code false} otherwise. Note that this method will return {@code true} if filter is not
+     *      specified.
+     * @throws IgniteCheckedException If put operation failed.
+     */
+    protected boolean put0(final K key, final V val, final CacheEntryPredicate filter)
+        throws IgniteCheckedException {
+        Boolean res = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.putAsync(ctx, null, key, val, false, filter).get().success();
             }
@@ -2260,10 +2269,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         });
 
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
-        return stored;
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -2305,7 +2313,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public <T> EntryProcessorResult<T> invoke(@Nullable AffinityTopologyVersion topVer,
+    @Nullable @Override public final <T> EntryProcessorResult<T> invoke(@Nullable AffinityTopologyVersion topVer,
         K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) throws IgniteCheckedException {
@@ -2538,7 +2546,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putAsync(K key, V val) {
+    @Override public final IgniteInternalFuture<Boolean> putAsync(K key, V val) {
         return putAsync(key, val, null);
     }
 
@@ -2548,9 +2556,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param filter Filter.
      * @return Put future.
      */
-    public IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+    public final IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
         A.notNull(key, "key", val, "val");
 
+        if (keyCheck)
+            validateCacheKey(key);
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2571,9 +2582,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     public IgniteInternalFuture<Boolean> putAsync0(final K key, final V val,
         @Nullable final CacheEntryPredicate filter) {
-        if (keyCheck)
-            validateCacheKey(key);
-
         return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAsync(ctx,
@@ -2598,267 +2606,82 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<V>(true) {
-            @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return (V)tx.putAsync(ctx, null, key, val, true, ctx.noVal()).get().value();
-            }
-
-            @Override public String toString() {
-                return "putIfAbsent [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Nullable @Override public final V getAndPutIfAbsent(final K key, final V val) throws IgniteCheckedException {
+        return getAndPut(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
-            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.noVal())
-                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
-            }
-
-            @Override public String toString() {
-                return "putIfAbsentAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<V> getAndPutIfAbsentAsync(final K key, final V val) {
+        return getAndPutAsync(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        Boolean stored = syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return tx.putAsync(ctx, null, key, val, false, ctx.noVal()).get().success();
-            }
-
-            @Override public String toString() {
-                return "putxIfAbsent [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled && stored)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
-
-        return stored;
+    @Override public final boolean putIfAbsent(final K key, final V val) throws IgniteCheckedException {
+        return put(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx,
-                    readyTopVer,
-                    key,
-                    val,
-                    false,
-                    ctx.noVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "putxIfAbsentAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<Boolean> putIfAbsentAsync(final K key, final V val) {
+        return putAsync(key, val, ctx.noVal());
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V getAndReplace(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<V>(true) {
-            @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return (V)tx.putAsync(ctx, null, key, val, true, ctx.hasVal()).get().value();
-            }
-
-            @Override public String toString() {
-                return "replace [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Nullable @Override public final V getAndReplace(final K key, final V val) throws IgniteCheckedException {
+        return getAndPut(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
-            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, true, ctx.hasVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, V>)RET2VAL);
-            }
-
-            @Override public String toString() {
-                return "replaceAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutAndGetTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
+    @Override public final IgniteInternalFuture<V> getAndReplaceAsync(final K key, final V val) {
+        return getAndPutAsync(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V val) throws IgniteCheckedException {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                return tx.putAsync(ctx, null, key, val, false, ctx.hasVal()).get().success();
-            }
-
-            @Override public String toString() {
-                return "replacex [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final boolean replace(final K key, final V val) throws IgniteCheckedException {
+        return put(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(final K key, final V val) {
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        return asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                return tx.putAsync(ctx, readyTopVer, key, val, false, ctx.hasVal()).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>) RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "replacexAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final IgniteInternalFuture<Boolean> replaceAsync(final K key, final V val) {
+        return putAsync(key, val, ctx.hasVal());
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException {
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        if (keyCheck)
-            validateCacheKey(key);
+    @Override public final boolean replace(final K key, final V oldVal, final V newVal) throws IgniteCheckedException {
+        A.notNull(oldVal, "oldVal");
 
-        return syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled())
-                    ctx.deploy().registerClass(oldVal);
-
-                return tx.putAsync(ctx, null, key, newVal, false, ctx.equalsVal(oldVal)).get()
-                    .success();
-            }
-
-            @Override public String toString() {
-                return "replace [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']';
-            }
-        });
+        return put(key, newVal, ctx.equalsVal(oldVal));
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Boolean> replaceAsync(final K key, final V oldVal, final V newVal) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
+        A.notNull(oldVal, "oldVal");
 
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled()) {
-                    try {
-                        ctx.deploy().registerClass(oldVal);
-                    }
-                    catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(e);
-                    }
-                }
-
-                return tx.putAsync(ctx, readyTopVer, key, newVal, false, ctx.equalsVal(oldVal)).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
-
-            @Override public String toString() {
-                return "replaceAsync [key=" + key + ", oldVal=" + oldVal + ", newVal=" + newVal + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdatePutAndGetTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+        return putAsync(key, newVal, ctx.equalsVal(oldVal));
     }
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        if (F.isEmpty(m))
+            return;
+
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
 
-        if (F.isEmpty(m))
-            return;
-
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
+        putAll0(m);
+
+        if (statsEnabled)
+            metrics0().addPutTimeNanos(System.nanoTime() - start);
+    }
+
+    /**
+     * @param m Map.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void putAll0(final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
         syncOp(new SyncInOp(m.size() == 1) {
             @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 tx.putAllAsync(ctx, null, m, false).get();
@@ -2868,9 +2691,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "putAll [map=" + m + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addPutTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
@@ -2881,6 +2701,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(m.keySet());
 
+        return putAllAsync0(m);
+    }
+
+    /**
+     * @param m Map.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<?> putAllAsync0(final Map<? extends K, ? extends V> m) {
         return asyncOp(new AsyncOp(m.keySet()) {
             @Override public IgniteInternalFuture<?> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.putAllAsync(ctx,
@@ -2906,11 +2734,25 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
+        V prevVal = getAndRemove0(key);
+
+        if (statsEnabled)
+            metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start);
+
+        return prevVal;
+    }
+
+    /**
+     * @param key Key.
+     * @return Previous value.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected V getAndRemove0(final K key) throws IgniteCheckedException {
         final boolean keepBinary = ctx.keepBinary();
 
-        V prevVal = syncOp(new SyncOp<V>(true) {
+        return syncOp(new SyncOp<V>(true) {
             @Override public V op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                K key0 = keepBinary ? (K)ctx.toCacheKeyObject(key) : key;
+                K key0 = keepBinary ? (K) ctx.toCacheKeyObject(key) : key;
 
                 V ret = tx.removeAllAsync(ctx,
                     null,
@@ -2920,9 +2762,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     /*singleRmv*/false).get().value();
 
                 if (ctx.config().getInterceptor() != null) {
-                    K key = keepBinary ? (K)ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
+                    K key = keepBinary ? (K) ctx.unwrapBinaryIfNeeded(key0, true, false) : key0;
 
-                    return (V)ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
+                    return (V) ctx.config().getInterceptor().onBeforeRemove(new CacheEntryImpl(key, ret)).get2();
                 }
 
                 return ret;
@@ -2932,11 +2774,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "remove [key=" + key + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addRemoveAndGetTimeNanos(System.nanoTime() - start);
-
-        return prevVal;
     }
 
     /** {@inheritDoc} */
@@ -2950,7 +2787,20 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteInternalFuture<V> fut = asyncOp(new AsyncOp<V>() {
+        IgniteInternalFuture<V> fut = getAndRemoveAsync0(key);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<V>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param key Key.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<V> getAndRemoveAsync0(final K key) {
+        return asyncOp(new AsyncOp<V>() {
             @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 // TODO should we invoke interceptor here?
                 return tx.removeAllAsync(ctx,
@@ -2965,11 +2815,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAsync [key=" + key + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<V>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
@@ -3002,6 +2847,17 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKeys(keys);
 
+        removeAll0(keys);
+
+        if (statsEnabled)
+            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+    }
+
+    /**
+     * @param keys Keys.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected void removeAll0(final Collection<? extends K> keys) throws IgniteCheckedException {
         syncOp(new SyncInOp(keys.size() == 1) {
             @Override public void inOp(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 tx.removeAllAsync(ctx,
@@ -3016,24 +2872,34 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAll [keys=" + keys + ']';
             }
         });
-
-        if (statsEnabled)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable final Collection<? extends K> keys) {
+        if (F.isEmpty(keys))
+            return new GridFinishedFuture<Object>();
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        if (F.isEmpty(keys))
-            return new GridFinishedFuture<Object>();
-
         if (keyCheck)
             validateCacheKeys(keys);
 
-        IgniteInternalFuture<Object> fut = asyncOp(new AsyncOp(keys) {
+        IgniteInternalFuture<Object> fut = removeAllAsync0(keys);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param keys Keys.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<Object> removeAllAsync0(final Collection<? extends K> keys) {
+        return asyncOp(new AsyncOp(keys) {
             @Override public IgniteInternalFuture<?> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.removeAllAsync(ctx,
                     readyTopVer,
@@ -3047,15 +2913,20 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAllAsync [keys=" + keys + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(final K key) throws IgniteCheckedException {
+        return remove(key, (CacheEntryPredicate)null);
+    }
+
+    /**
+     * @param key Key.
+     * @param filter Filter.
+     * @return {@code True} if entry was removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean remove(final K key, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
@@ -3065,13 +2936,27 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        boolean rmv = syncOp(new SyncOp<Boolean>(true) {
+        boolean rmv = remove0(key, filter);
+
+        if (statsEnabled && rmv)
+            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+
+        return rmv;
+    }
+
+    /**
+     * @param key Key.
+     * @return {@code True} if entry was removed.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected boolean remove0(final K key, final CacheEntryPredicate filter) throws IgniteCheckedException {
+        Boolean res = syncOp(new SyncOp<Boolean>(true) {
             @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
                 return tx.removeAllAsync(ctx,
                     null,
                     Collections.singletonList(key),
                     /*retval*/false,
-                    null,
+                    filter,
                     /*singleRmv*/true).get().success();
             }
 
@@ -3080,10 +2965,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
             }
         });
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
+        assert res != null;
 
-        return rmv;
+        return res;
     }
 
     /** {@inheritDoc} */
@@ -3108,7 +2992,21 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
+        IgniteInternalFuture<Boolean> fut = removeAsync0(key, filter);
+
+        if (statsEnabled)
+            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
+
+        return fut;
+    }
+
+    /**
+     * @param key Key.
+     * @param filter Filter.
+     * @return Future.
+     */
+    protected IgniteInternalFuture<Boolean> removeAsync0(final K key, @Nullable final CacheEntryPredicate filter) {
+        return asyncOp(new AsyncOp<Boolean>() {
             @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
                 return tx.removeAllAsync(ctx,
                     readyTopVer,
@@ -3123,11 +3021,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 return "removeAsync [key=" + key + ", filter=" + filter + ']';
             }
         });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
     }
 
     /** {@inheritDoc} */
@@ -3169,86 +3062,21 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(final K key, final V val) throws IgniteCheckedException {
-        boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        boolean rmv = syncOp(new SyncOp<Boolean>(true) {
-            @Override public Boolean op(IgniteTxLocalAdapter tx) throws IgniteCheckedException {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled())
-                    ctx.deploy().registerClass(val);
-
-                return tx.removeAllAsync(ctx,
-                    null,
-                    Collections.singletonList(key),
-                    /*retval*/false,
-                    ctx.equalsVal(val),
-                    /*singleRmv*/false).get().success();
-            }
-
-            @Override public String toString() {
-                return "remove [key=" + key + ", val=" + val + ']';
-            }
-        });
+    @Override public final boolean remove(final K key, final V val) throws IgniteCheckedException {
+        A.notNull(val, "val");
 
-        if (statsEnabled && rmv)
-            metrics0().addRemoveTimeNanos(System.nanoTime() - start);
-
-        return rmv;
+        return remove(key, ctx.equalsVal(val));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(final K key, final V val) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
-        A.notNull(key, "key", val, "val");
-
-        if (keyCheck)
-            validateCacheKey(key);
-
-        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>() {
-            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter tx, AffinityTopologyVersion readyTopVer) {
-                // Register before hiding in the filter.
-                if (ctx.deploymentEnabled()) {
-                    try {
-                        ctx.deploy().registerClass(val);
-                    }
-                    catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(e);
-                    }
-                }
-
-                return tx.removeAllAsync(ctx,
-                    readyTopVer,
-                    Collections.singletonList(key),
-                    /*retval*/false,
-                    ctx.equalsVal(val),
-                    /*singleRmv*/false).chain(
-                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn>, Boolean>)RET2FLAG);
-            }
+    @Override public final IgniteInternalFuture<Boolean> removeAsync(final K key, final V val) {
+        A.notNull(key, "val");
 
-            @Override public String toString() {
-                return "removeAsync [key=" + key + ", val=" + val + ']';
-            }
-        });
-
-        if (statsEnabled)
-            fut.listen(new UpdateRemoveTimeStatClosure<Boolean>(metrics0(), start));
-
-        return fut;
+        return removeAsync(key, ctx.equalsVal(val));
     }
 
     /** {@inheritDoc} */
-    @Override public CacheMetrics clusterMetrics() {
+    @Override public final CacheMetrics clusterMetrics() {
         return clusterMetrics(ctx.grid().cluster().forCacheNodes(ctx.name()));
     }
 
@@ -3277,7 +3105,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public CacheMetricsMXBean localMxBean() {
-        return localMxBean;
+        return locMxBean;
     }
 
     /** {@inheritDoc} */
@@ -4607,9 +4435,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Failed future if waiting was interrupted.
      */
     @Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
-        if (!asyncToggled)
-            return null;
-
         try {
             if (asyncOpsSem != null)
                 asyncOpsSem.acquire();
@@ -4627,8 +4452,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /**
      * Releases asynchronous operations permit, if limited.
      */
-    protected void asyncOpRelease() {
-        if (asyncOpsSem != null && asyncToggled)
+    private void asyncOpRelease() {
+        if (asyncOpsSem != null)
             asyncOpsSem.release();
     }
 
@@ -4793,12 +4618,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
-        checkJta();
-
+    @Nullable public final V get(K key, boolean deserializeBinary, final boolean needVer) throws IgniteCheckedException {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
-        return get(key, taskName, deserializeBinary, needVer);
+        return get0(key, taskName, deserializeBinary, needVer);
     }
 
     /**
@@ -4809,11 +4632,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Cached value.
      * @throws IgniteCheckedException If failed.
      */
-    protected V get(
+    protected V get0(
         final K key,
         String taskName,
         boolean deserializeBinary,
         boolean needVer) throws IgniteCheckedException {
+        checkJta();
+
         try {
             return getAsync(key,
                 !ctx.config().isReadFromBackup(),
@@ -4867,7 +4692,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Map of cached values.
      * @throws IgniteCheckedException If read failed.
      */
-    public Map<K, V> getAll(Collection<? extends K> keys, boolean deserializeBinary,
+    protected Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary,
         boolean needVer) throws IgniteCheckedException {
         checkJta();
 
@@ -4922,7 +4747,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param key Cache key.
      * @throws IllegalArgumentException If validation fails.
      */
-    protected void validateCacheKey(Object key) {
+    protected final void validateCacheKey(Object key) {
         if (keyCheck) {
             CU.validateCacheKey(key);
 
@@ -4937,7 +4762,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param keys Cache keys.
      * @throws IgniteException If validation fails.
      */
-    protected void validateCacheKeys(Iterable<?> keys) {
+    protected final void validateCacheKeys(Iterable<?> keys) {
         if (keys == null)
             return;
 
@@ -4958,7 +4783,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param deserializeBinary Deserialize binary flag.
      * @return Public API iterator.
      */
-    protected Iterator<Cache.Entry<K, V>> iterator(final Iterator<? extends GridCacheEntryEx> it,
+    protected final Iterator<Cache.Entry<K, V>> iterator(final Iterator<? extends GridCacheEntryEx> it,
         final boolean deserializeBinary) {
         return new Iterator<Cache.Entry<K, V>>() {
             {
@@ -5276,7 +5101,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @param plc Explicitly specified expiry policy for cache operation.
      * @return Expiry policy wrapper.
      */
-    @Nullable public IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) {
+    @Nullable public final IgniteCacheExpiryPolicy expiryPolicy(@Nullable ExpiryPolicy plc) {
         if (plc == null)
             plc = ctx.expiry();
 
@@ -5401,7 +5226,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
          * @param opCtx Operation context.
          * @return Operation future.
          */
-        protected IgniteInternalFuture<T> waitTopologyFuture(IgniteInternalFuture<?> topFut,
+        private IgniteInternalFuture<T> waitTopologyFuture(IgniteInternalFuture<?> topFut,
             final AffinityTopologyVersion topVer,
             final IgniteTxLocalAdapter tx,
             final CacheOperationContext opCtx) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index b9e6e82..f87fa1d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -334,14 +334,6 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteCache<K, V> withAsync() {
-        if (delegate instanceof GridCacheAdapter)
-            ((GridCacheAdapter)delegate).toggleAsync();
-
-        return super.withAsync();
-    }
-
-    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withSkipStore() {
         return skipStore();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 0e60ff4..a67a903 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -139,7 +139,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     private CI2<GridNearAtomicAbstractUpdateRequest, GridNearAtomicUpdateResponse> updateReplyClos;
 
     /** Pending */
-    private GridDeferredAckMessageSender deferredUpdateMessageSender;
+    private GridDeferredAckMessageSender deferredUpdateMsgSnd;
 
     /** */
     private GridNearAtomicCache<K, V> near;
@@ -174,6 +174,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected void checkJta() throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isDhtAtomic() {
         return true;
     }
@@ -235,7 +240,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public void start() throws IgniteCheckedException {
         super.start();
 
-        deferredUpdateMessageSender = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) {
+        deferredUpdateMsgSnd = new GridDeferredAckMessageSender(ctx.time(), ctx.closures()) {
             @Override public int getTimeout() {
                 return DEFERRED_UPDATE_RESPONSE_TIMEOUT;
             }
@@ -447,7 +452,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public void stop() {
-        deferredUpdateMessageSender.stop();
+        deferredUpdateMsgSnd.stop();
     }
 
     /**
@@ -463,7 +468,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected V get(K key, String taskName, boolean deserializeBinary, boolean needVer)
+    @Override protected V get0(K key, String taskName, boolean deserializeBinary, boolean needVer)
         throws IgniteCheckedException {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
@@ -540,6 +545,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override protected Map<K, V> getAll0(Collection<? extends K> keys, boolean deserializeBinary, boolean needVer)
+        throws IgniteCheckedException {
+        return getAllAsyncInternal(keys,
+            !ctx.config().isReadFromBackup(),
+            true,
+            null,
+            ctx.kernalContext().job().currentTaskName(),
+            deserializeBinary,
+            false,
+            true,
+            needVer,
+            false).get();
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         final boolean forcePrimary,
@@ -551,6 +571,43 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final boolean canRemap,
         final boolean needVer
     ) {
+        return getAllAsyncInternal(keys,
+            forcePrimary,
+            skipTx,
+            subjId,
+            taskName,
+            deserializeBinary,
+            skipVals,
+            canRemap,
+            needVer,
+            true);
+    }
+
+    /**
+     * @param keys Keys.
+     * @param forcePrimary Force primary flag.
+     * @param skipTx Skip tx flag.
+     * @param subjId Subject ID.
+     * @param taskName Task name.
+     * @param deserializeBinary Deserialize binary flag.
+     * @param skipVals Skip values flag.
+     * @param canRemap Can remap flag.
+     * @param needVer Need version flag.
+     * @param asyncOp Async operation flag.
+     * @return Future.
+     */
+    private IgniteInternalFuture<Map<K, V>> getAllAsyncInternal(
+        @Nullable final Collection<? extends K> keys,
+        final boolean forcePrimary,
+        boolean skipTx,
+        @Nullable UUID subjId,
+        final String taskName,
+        final boolean deserializeBinary,
+        final boolean skipVals,
+        final boolean canRemap,
+        final boolean needVer,
+        boolean asyncOp
+    ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
@@ -561,7 +618,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         CacheOperationContext opCtx = ctx.operationContextPerCall();
 
-        subjId = ctx.subjectIdPerCall(null, opCtx);
+        subjId = ctx.subjectIdPerCall(subjId, opCtx);
 
         final UUID subjId0 = subjId;
 
@@ -569,57 +626,91 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean skipStore = opCtx != null && opCtx.skipStore();
 
-        return asyncOp(new CO<IgniteInternalFuture<Map<K, V>>>() {
-            @Override public IgniteInternalFuture<Map<K, V>> apply() {
-                return getAllAsync0(ctx.cacheKeysView(keys),
-                    forcePrimary,
-                    subjId0,
-                    taskName,
-                    deserializeBinary,
-                    expiryPlc,
-                    skipVals,
-                    skipStore,
-                    canRemap,
-                    needVer);
-            }
-        });
+        if (asyncOp) {
+            return asyncOp(new CO<IgniteInternalFuture<Map<K, V>>>() {
+                @Override public IgniteInternalFuture<Map<K, V>> apply() {
+                    return getAllAsync0(ctx.cacheKeysView(keys),
+                        forcePrimary,
+                        subjId0,
+                        taskName,
+                        deserializeBinary,
+                        expiryPlc,
+                        skipVals,
+                        skipStore,
+                        canRemap,
+                        needVer);
+                }
+            });
+        }
+        else {
+            return getAllAsync0(ctx.cacheKeysView(keys),
+                forcePrimary,
+                subjId0,
+                taskName,
+                deserializeBinary,
+                expiryPlc,
+                skipVals,
+                skipStore,
+                canRemap,
+                needVer);
+        }
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPut(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
-        return getAndPutAsync0(key, val, filter).get();
+    @Override protected V getAndPut0(K key, V val, @Nullable CacheEntryPredicate filter) throws IgniteCheckedException {
+        return (V)update0(
+            key,
+            val,
+            null,
+            null,
+            true,
+            filter,
+            true,
+            false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public boolean put(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
-        return putAsync(key, val, filter).get();
+    @Override protected boolean put0(K key, V val, CacheEntryPredicate filter) throws IgniteCheckedException {
+        Boolean res = (Boolean)update0(
+            key,
+            val,
+            null,
+            null,
+            false,
+            filter,
+            true,
+            false).get();
+
+        assert res != null;
+
+        return res;
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<V> getAndPutAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
-        return updateAsync0(
+        return update0(
             key,
             val,
             null,
             null,
             true,
             filter,
+            true,
             true);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        return updateAsync0(
+        return update0(
             key,
             val,
             null,
             null,
             false,
             filter,
+            true,
             true);
     }
 
@@ -627,84 +718,34 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public V tryGetAndPut(K key, V val) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
-        return (V)updateAsync0(
+        return (V) update0(
             key,
             val,
             null,
             null,
             true,
             null,
+            false,
             false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndPutIfAbsent(K key, V val) throws IgniteCheckedException {
-        return getAndPutIfAbsentAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndPutIfAbsentAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return getAndPutAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean putIfAbsent(K key, V val) throws IgniteCheckedException {
-        return putIfAbsentAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return putAsync(key, val, ctx.noVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public V getAndReplace(K key, V val) throws IgniteCheckedException {
-        return getAndReplaceAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<V> getAndReplaceAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return getAndPutAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V val) throws IgniteCheckedException {
-        return replaceAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return putAsync(key, val, ctx.hasVal());
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        return replaceAsync(key, oldVal, newVal).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
-        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
-
-        return putAsync(key, newVal, ctx.equalsVal(oldVal));
-    }
-
-    /** {@inheritDoc} */
-    @Override public void putAll(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
-        putAllAsync(m).get();
+    @Override protected void putAll0(Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        updateAll0(m,
+            null,
+            null,
+            null,
+            null,
+            false,
+            false,
+            true,
+            UPDATE,
+            false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m) {
-        return updateAllAsync0(m,
+    @Override public IgniteInternalFuture<?> putAllAsync0(Map<? extends K, ? extends V> m) {
+        return updateAll0(m,
             null,
             null,
             null,
@@ -712,7 +753,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            UPDATE).chain(RET2NULL);
+            UPDATE,
+            true).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
@@ -725,7 +767,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public IgniteInternalFuture<?> putAllConflictAsync(Map<KeyCacheObject, GridCacheDrInfo> conflictMap) {
         ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size());
 
-        return updateAllAsync0(null,
+        return updateAll0(null,
             null,
             null,
             conflictMap,
@@ -733,57 +775,40 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            UPDATE);
+            UPDATE,
+            true);
     }
 
     /** {@inheritDoc} */
-    @Override public V getAndRemove(K key) throws IgniteCheckedException {
-        return getAndRemoveAsync(key).get();
+    @Override public V getAndRemove0(K key) throws IgniteCheckedException {
+        return (V)remove0(key, true, null, false).get();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<V> getAndRemoveAsync(K key) {
-        A.notNull(key, "key");
-
-        return removeAsync0(key, true, null);
+    @Override public IgniteInternalFuture<V> getAndRemoveAsync0(K key) {
+        return remove0(key, true, null, true);
     }
 
     /** {@inheritDoc} */
-    @Override public void removeAll(Collection<? extends K> keys) throws IgniteCheckedException {
-        removeAllAsync(keys).get();
+    @Override protected void removeAll0(Collection<? extends K> keys) throws IgniteCheckedException {
+        removeAllAsync0(keys, null, false, false, false).get();
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys) {
-        A.notNull(keys, "keys");
-
-        return removeAllAsync0(keys, null, false, false).chain(RET2NULL);
+    @Override public IgniteInternalFuture<Object> removeAllAsync0(Collection<? extends K> keys) {
+        return removeAllAsync0(keys, null, false, false, true).chain(RET2NULL);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key) throws IgniteCheckedException {
-        return removeAsync(key, (CacheEntryPredicate)null).get();
+    @Override protected boolean remove0(K key, CacheEntryPredicate filter) throws IgniteCheckedException {
+        return (Boolean)remove0(key, false, filter, false).get();
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key");
-
-        return removeAsync0(key, false, filter);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        return removeAsync(key, val).get();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
-        A.notNull(key, "key", val, "val");
-
-        return removeAsync(key, ctx.equalsVal(val));
+    @Override public IgniteInternalFuture<Boolean> removeAsync0(K key, @Nullable CacheEntryPredicate filter) {
+        return remove0(key, false, filter, true);
     }
 
     /** {@inheritDoc} */
@@ -796,7 +821,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public IgniteInternalFuture<?> removeAllConflictAsync(Map<KeyCacheObject, GridCacheVersion> conflictMap) {
         ctx.dr().onReceiveCacheEntriesReceived(conflictMap.size());
 
-        return removeAllAsync0(null, conflictMap, false, false);
+        return removeAllAsync0(null, conflictMap, false, false, true);
     }
 
     /**
@@ -811,10 +836,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
-        if (!asyncToggled)
-            return op.apply();
-
+    private <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
         IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)
@@ -871,7 +893,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @Override public <T> EntryProcessorResult<T> invoke(K key, EntryProcessor<K, V, T> entryProcessor, Object... args)
         throws IgniteCheckedException {
-        IgniteInternalFuture<EntryProcessorResult<T>> invokeFut = invokeAsync(key, entryProcessor, args);
+        IgniteInternalFuture<EntryProcessorResult<T>> invokeFut = invoke0(false, key, entryProcessor, args);
 
         EntryProcessorResult<T> res = invokeFut.get();
 
@@ -881,16 +903,30 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
-        Object... args)
-        throws IgniteCheckedException {
-        return invokeAllAsync(keys, entryProcessor, args).get();
+        Object... args) throws IgniteCheckedException
+    {
+        return invokeAll0(false, keys, entryProcessor, args).get();
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
     @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
+        return invoke0(true, key, entryProcessor, args);
+    }
+
+    /**
+     * @param async Async operation flag.
+     * @param key Key.
+     * @param entryProcessor Entry processor.
+     * @param args Entry processor arguments.
+     * @return Future.
+     */
+    private <T> IgniteInternalFuture<EntryProcessorResult<T>> invoke0(
+        boolean async,
+        K key,
+        EntryProcessor<K, V, T> entryProcessor,
+        Object... args) {
         A.notNull(key, "key", entryProcessor, "entryProcessor");
 
         if (keyCheck)
@@ -900,14 +936,15 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAsync0(
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = update0(
             key,
             null,
             entryProcessor,
             args,
             false,
             null,
-            true);
+            true,
+            async);
 
         return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
             @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut)
@@ -940,6 +977,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         final EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
+        return invokeAll0(true, keys, entryProcessor, args);
+    }
+
+    /**
+     * @param async Async operation flag.
+     * @param keys Keys.
+     * @param entryProcessor Entry processor.
+     * @param args Entry processor arguments.
+     * @return Future.
+     */
+    private <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAll0(
+        boolean async,
+        Set<? extends K> keys,
+        final EntryProcessor<K, V, T> entryProcessor,
+        Object... args) {
         A.notNull(keys, "keys", entryProcessor, "entryProcessor");
 
         if (keyCheck)
@@ -955,7 +1007,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final boolean keepBinary = opCtx != null && opCtx.isKeepBinary();
 
-        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> resFut = updateAllAsync0(null,
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> resFut = updateAll0(null,
             invokeMap,
             args,
             null,
@@ -963,7 +1015,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            TRANSFORM);
+            TRANSFORM,
+            async);
 
         return resFut.chain(
             new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, Map<K, EntryProcessorResult<T>>>() {
@@ -981,7 +1034,21 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) throws IgniteCheckedException {
-        return invokeAllAsync(map, args).get();
+        A.notNull(map, "map");
+
+        if (keyCheck)
+            validateCacheKeys(map.keySet());
+
+        return (Map<K, EntryProcessorResult<T>>)updateAll0(null,
+            map,
+            args,
+            null,
+            null,
+            false,
+            false,
+            true,
+            TRANSFORM,
+            false).get();
     }
 
     /** {@inheritDoc} */
@@ -994,7 +1061,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         if (keyCheck)
             validateCacheKeys(map.keySet());
 
-        return updateAllAsync0(null,
+        return updateAll0(null,
             map,
             args,
             null,
@@ -1002,7 +1069,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             false,
             false,
             true,
-            TRANSFORM);
+            TRANSFORM,
+            true);
     }
 
     /**
@@ -1016,10 +1084,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param retval Return value required flag.
      * @param rawRetval Return {@code GridCacheReturn} instance.
      * @param waitTopFut Whether to wait for topology future.
+     * @param async Async operation flag.
      * @return Completion future.
      */
     @SuppressWarnings("ConstantConditions")
-    private IgniteInternalFuture updateAllAsync0(
+    private IgniteInternalFuture updateAll0(
         @Nullable Map<? extends K, ? extends V> map,
         @Nullable Map<? extends K, ? extends EntryProcessor> invokeMap,
         @Nullable Object[] invokeArgs,
@@ -1028,7 +1097,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final boolean retval,
         final boolean rawRetval,
         final boolean waitTopFut,
-        final GridCacheOperation op
+        final GridCacheOperation op,
+        boolean async
     ) {
         assert ctx.updatesAllowed();
 
@@ -1105,13 +1175,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES,
             waitTopFut);
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-                return updateFut;
-            }
-        });
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
+
+            return updateFut;
+        }
     }
 
     /**
@@ -1124,16 +1201,18 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param retval Return value flag.
      * @param filter Filter.
      * @param waitTopFut Whether to wait for topology future.
+     * @param async Async operation flag.
      * @return Future.
      */
-    private IgniteInternalFuture updateAsync0(
+    private IgniteInternalFuture update0(
         K key,
         @Nullable V val,
         @Nullable EntryProcessor proc,
         @Nullable Object[] invokeArgs,
         final boolean retval,
         @Nullable final CacheEntryPredicate filter,
-        final boolean waitTopFut
+        final boolean waitTopFut,
+        boolean async
     ) {
         assert val == null || proc == null;
 
@@ -1146,13 +1225,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         final GridNearAtomicAbstractUpdateFuture updateFut =
             createSingleUpdateFuture(key, val, proc, invokeArgs, retval, filter, waitTopFut);
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-                return updateFut;
-            }
-        });
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
+
+            return updateFut;
+        }
     }
 
     /**
@@ -1161,33 +1247,38 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param key Key.
      * @param retval Whether to return
      * @param filter Filter.
+     * @param async Async operation flag.
      * @return Future.
      */
-    private IgniteInternalFuture removeAsync0(K key, final boolean retval,
-        @Nullable CacheEntryPredicate filter) {
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
+    private IgniteInternalFuture remove0(K key, final boolean retval,
+        @Nullable CacheEntryPredicate filter,
+        boolean async) {
         assert ctx.updatesAllowed();
 
-        validateCacheKey(key);
-
         ctx.checkSecurity(SecurityPermission.CACHE_REMOVE);
 
-        final GridNearAtomicAbstractUpdateFuture updateFut =
-            createSingleUpdateFuture(key, null, null, null, retval, filter, true);
+        final GridNearAtomicAbstractUpdateFuture updateFut = createSingleUpdateFuture(key,
+            null,
+            null,
+            null,
+            retval,
+            filter,
+            true);
 
-        if (statsEnabled)
-            updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
 
-                return updateFut;
-            }
-        });
+            return updateFut;
+        }
     }
 
     /**
@@ -1326,14 +1417,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable Collection<? extends K> keys,
         @Nullable Map<KeyCacheObject, GridCacheVersion> conflictMap,
         final boolean retval,
-        boolean rawRetval
+        boolean rawRetval,
+        boolean async
     ) {
         assert ctx.updatesAllowed();
 
-        final boolean statsEnabled = ctx.config().isStatisticsEnabled();
-
-        final long start = statsEnabled ? System.nanoTime() : 0L;
-
         assert keys != null || conflictMap != null;
 
         if (keyCheck)
@@ -1380,16 +1468,20 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             opCtx != null && opCtx.noRetries() ? 1 : MAX_RETRIES,
             true);
 
-        if (statsEnabled)
-            updateFut.listen(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
+        if (async) {
+            return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+                @Override public IgniteInternalFuture<Object> apply() {
+                    updateFut.map();
 
-        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
-            @Override public IgniteInternalFuture<Object> apply() {
-                updateFut.map();
+                    return updateFut;
+                }
+            });
+        }
+        else {
+            updateFut.map();
 
-                return updateFut;
-            }
-        });
+            return updateFut;
+        }
     }
 
     /**
@@ -3248,7 +3340,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param ver Version to ack.
      */
     private void sendDeferredUpdateResponse(UUID nodeId, GridCacheVersion ver) {
-        deferredUpdateMessageSender.sendDeferredAckMessage(nodeId, ver);
+        deferredUpdateMsgSnd.sendDeferredAckMessage(nodeId, ver);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ad785cbd/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 176a90f..9cf400d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -327,19 +327,6 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             needVer);
     }
 
-    /** {@inheritDoc} */
-    @Override protected GridCacheEntryEx entryExSafe(
-        KeyCacheObject key,
-        AffinityTopologyVersion topVer
-    ) {
-        try {
-            return ctx.affinity().localNode(key, topVer) ? entryEx(key) : null;
-        }
-        catch (GridDhtInvalidPartitionException ignored) {
-            return null;
-        }
-    }
-
     /**
      * @param keys Keys to load.
      * @param readThrough Read through flag.


[17/50] [abbrv] ignite git commit: ignite-4371 Avoid synchronous 'rollback' call from system threads

Posted by vo...@apache.org.
ignite-4371 Avoid synchronous 'rollback' call from system threads

(cherry picked from commit 0c782b0)


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

Branch: refs/heads/ignite-2.0
Commit: c1fde05d05251353f0e75125aca44ab145798c8e
Parents: ad785cb
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 16 19:15:48 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 11:43:34 2016 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      |  37 +++---
 .../processors/cache/GridCacheProcessor.java    |   2 +-
 .../processors/cache/GridCacheUtils.java        |  10 +-
 .../GridDistributedTxRemoteAdapter.java         |   2 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  74 +++++------
 .../cache/distributed/dht/GridDhtTxLocal.java   | 125 +++++++------------
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   7 +-
 .../near/GridNearTxFinishFuture.java            |  44 +++----
 .../cache/distributed/near/GridNearTxLocal.java |  48 ++++---
 .../cache/transactions/IgniteTxHandler.java     |  13 +-
 .../transactions/IgniteTxLocalAdapter.java      |   3 +-
 .../cache/transactions/IgniteTxManager.java     |   7 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |   1 +
 14 files changed, 161 insertions(+), 218 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 503b334..d26031c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.events.DiscoveryCustomEvent;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
@@ -854,24 +855,30 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     ready = cacheCtx.started();
 
                 if (ready) {
-                    GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
+                    GridAffinityAssignmentCache affCache = cacheCtx.affinity().affinityCache();
 
-                    if (useOldApi) {
-                        locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
-                            locMap.nodeOrder(),
-                            locMap.updateSequence(),
-                            locMap);
-                    }
+                    if (affCache != null) {
+                        GridDhtPartitionFullMap locMap = cacheCtx.topology().partitionMap(true);
+
+                        if (useOldApi) {
+                            locMap = new GridDhtPartitionFullMap(locMap.nodeId(),
+                                locMap.nodeOrder(),
+                                locMap.updateSequence(),
+                                locMap);
+                        }
 
-                    addFullPartitionsMap(m,
-                        dupData,
-                        compress,
-                        cacheCtx.cacheId(),
-                        locMap,
-                        cacheCtx.affinity().affinityCache().similarAffinityKey());
+                        addFullPartitionsMap(m,
+                            dupData,
+                            compress,
+                            cacheCtx.cacheId(),
+                            locMap,
+                            affCache.similarAffinityKey());
 
-                    if (exchId != null)
-                        m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+                        if (exchId != null)
+                            m.addPartitionUpdateCounters(cacheCtx.cacheId(), cacheCtx.topology().updateCounters(true));
+                    }
+                    else
+                        assert cctx.cacheContext(cacheCtx.cacheId()) == null : cacheCtx.name();
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/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..cb96225 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
@@ -1761,7 +1761,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /**
      * @param req Stop request.
      */
-    public void prepareCacheStop(DynamicCacheChangeRequest req) {
+    private void prepareCacheStop(DynamicCacheChangeRequest req) {
         assert req.stop() || req.close() : req;
 
         GridCacheAdapter<?, ?> cache = caches.remove(maskNull(req.cacheName()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 3178203..fef6ddd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -928,9 +928,13 @@ public class GridCacheUtils {
         if (tx == null)
             return "null";
 
-        return tx.getClass().getSimpleName() + "[id=" + tx.xid() + ", concurrency=" + tx.concurrency() +
-            ", isolation=" + tx.isolation() + ", state=" + tx.state() + ", invalidate=" + tx.isInvalidate() +
-            ", rollbackOnly=" + tx.isRollbackOnly() + ", nodeId=" + tx.nodeId() +
+        return tx.getClass().getSimpleName() + "[id=" + tx.xid() +
+            ", concurrency=" + tx.concurrency() +
+            ", isolation=" + tx.isolation() +
+            ", state=" + tx.state() +
+            ", invalidate=" + tx.isInvalidate() +
+            ", rollbackOnly=" + tx.isRollbackOnly() +
+            ", nodeId=" + tx.nodeId() +
             ", duration=" + (U.currentTimeMillis() - tx.startTime()) + ']';
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index 4adfa8b..68c0e57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -386,7 +386,7 @@ public class GridDistributedTxRemoteAdapter extends IgniteTxAdapter
         // If another thread is doing prepare or rollback.
         if (!state(PREPARING)) {
             // In optimistic mode prepare may be called multiple times.
-            if(state() != PREPARING || !optimistic()) {
+            if (state() != PREPARING || !optimistic()) {
                 if (log.isDebugEnabled())
                     log.debug("Invalid transaction state for prepare: " + this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index ac2ab41..147cbea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -33,8 +33,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedTxMapping;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry;
-import org.apache.ignite.internal.transactions.IgniteTxHeuristicCheckedException;
-import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException;
 import org.apache.ignite.internal.util.future.GridCompoundIdentityFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -94,9 +92,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     /** Near mappings. */
     private Map<UUID, GridDistributedTxMapping> nearMap;
 
-    /** Trackable flag. */
-    private boolean trackable = true;
-
     /**
      * @param cctx Context.
      * @param tx Transaction.
@@ -151,46 +146,24 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
 
     /** {@inheritDoc} */
     @Override public boolean trackable() {
-        return trackable;
+        return true;
     }
 
     /** {@inheritDoc} */
     @Override public void markNotTrackable() {
-        trackable = false;
+        assert false;
     }
 
     /**
      * @param e Error.
      */
-    public void onError(Throwable e) {
-        if (ERR_UPD.compareAndSet(this, null, e)) {
-            boolean marked = tx.setRollbackOnly();
-
-            if (e instanceof IgniteTxRollbackCheckedException) {
-                if (marked) {
-                    try {
-                        tx.rollback();
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
-                    }
-                }
-            }
-            else if (tx.isSystemInvalidate()) { // Invalidate remote transactions on heuristic error.
-                finish();
+    public void rollbackOnError(Throwable e) {
+        assert e != null;
 
-                try {
-                    get();
-                }
-                catch (IgniteTxHeuristicCheckedException ignore) {
-                    // Future should complete with GridCacheTxHeuristicException.
-                }
-                catch (IgniteCheckedException err) {
-                    U.error(log, "Failed to invalidate transaction: " + tx, err);
-                }
-            }
+        if (ERR_UPD.compareAndSet(this, null, e)) {
+            tx.setRollbackOnly();
 
-            onComplete();
+            finish(false);
         }
     }
 
@@ -240,12 +213,21 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     /** {@inheritDoc} */
     @Override public boolean onDone(IgniteInternalTx tx, Throwable err) {
         if (initialized() || err != null) {
-            if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING))
-                this.tx.tmFinish(err == null);
-
             Throwable e = this.err;
 
-            if (e == null && commit)
+            if (this.tx.onePhaseCommit() && (this.tx.state() == COMMITTING)) {
+                try {
+                    this.tx.tmFinish(err == null);
+                }
+                catch (IgniteCheckedException finishErr) {
+                    U.error(log, "Failed to finish tx: " + tx, e);
+
+                    if (e == null)
+                        e = finishErr;
+                }
+            }
+
+            if (commit && e == null)
                 e = this.tx.commitError();
 
             Throwable finishErr = e != null ? e : err;
@@ -255,7 +237,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                     finishErr = this.tx.commitError();
 
                 if (this.tx.syncMode() != PRIMARY_SYNC)
-                    this.tx.sendFinishReply(commit, finishErr);
+                    this.tx.sendFinishReply(finishErr);
 
                 // Don't forget to clean up.
                 cctx.mvcc().removeFuture(futId);
@@ -284,13 +266,15 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
 
     /**
      * Initializes future.
+     *
+     * @param commit Commit flag.
      */
     @SuppressWarnings({"SimplifiableIfStatement", "IfMayBeConditional"})
-    public void finish() {
+    public void finish(boolean commit) {
         boolean sync;
 
         if (!F.isEmpty(dhtMap) || !F.isEmpty(nearMap))
-            sync = finish(dhtMap, nearMap);
+            sync = finish(commit, dhtMap, nearMap);
         else if (!commit && !F.isEmpty(tx.lockTransactionNodes()))
             sync = rollbackLockTransactions(tx.lockTransactionNodes());
         else
@@ -308,7 +292,6 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for.
      */
     private boolean rollbackLockTransactions(Collection<ClusterNode> nodes) {
-        assert !commit;
         assert !F.isEmpty(nodes);
 
         if (tx.onePhaseCommit())
@@ -337,7 +320,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                 tx.commitVersion(),
                 tx.threadId(),
                 tx.isolation(),
-                commit,
+                false,
                 tx.isInvalidate(),
                 tx.system(),
                 tx.ioPolicy(),
@@ -390,11 +373,14 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
     }
 
     /**
+     * @param commit Commit flag.
      * @param dhtMap DHT map.
      * @param nearMap Near map.
      * @return {@code True} in case there is at least one synchronous {@code MiniFuture} to wait for.
      */
-    private boolean finish(Map<UUID, GridDistributedTxMapping> dhtMap, Map<UUID, GridDistributedTxMapping> nearMap) {
+    private boolean finish(boolean commit,
+        Map<UUID, GridDistributedTxMapping> dhtMap,
+        Map<UUID, GridDistributedTxMapping> nearMap) {
         if (tx.onePhaseCommit())
             return false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index b659abb..4e39e9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -503,52 +503,57 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
     }
 
     /**
+     * @param commit Commit flag.
      * @param prepFut Prepare future.
      * @param fut Finish future.
      */
-    private void finishCommit(@Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
+    private void finishTx(boolean commit, @Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
+        assert prepFut == null || prepFut.isDone();
+
         boolean primarySync = syncMode() == PRIMARY_SYNC;
 
         IgniteCheckedException err = null;
 
-        try {
-            if (prepFut != null)
-                prepFut.get(); // Check for errors.
-
-            if (finish(true)) {
-                if (primarySync)
-                    sendFinishReply(true, null);
-
-                fut.finish();
+        if (!commit && prepFut != null) {
+            try {
+                prepFut.get();
             }
-            else {
-                err = new IgniteCheckedException("Failed to commit transaction: " + CU.txString(this));
-
-                fut.onError(err);
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to prepare transaction [tx=" + this + ", e=" + e + ']');
+            }
+            finally {
+                prepFut = null;
             }
         }
-        catch (IgniteTxOptimisticCheckedException e) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to optimistically prepare transaction [tx=" + this + ", e=" + e + ']');
 
-            err = e;
+        try {
+            if (prepFut != null)
+                prepFut.get(); // Check for errors.
 
-            fut.onError(e);
+            boolean finished = finish(commit);
+
+            if (!finished)
+                err = new IgniteCheckedException("Failed to finish transaction [commit=" + commit +
+                    ", tx=" + CU.txString(this) + ']');
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to prepare transaction: " + this, e);
+            U.error(log, "Failed to finish transaction [commit=" + commit + ", tx=" + this + ']', e);
 
             err = e;
-
-            fut.onError(e);
         }
 
-        if (primarySync && err != null)
-            sendFinishReply(true, err);
+        if (primarySync)
+            sendFinishReply(err);
+
+        if (err != null)
+            fut.rollbackOnError(err);
+        else
+            fut.finish(commit);
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"ThrowableInstanceNeverThrown"})
+    @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<IgniteInternalTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing dht local tx: " + this);
@@ -557,7 +562,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         if (pessimistic())
             prepareAsync();
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, true);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
@@ -565,11 +570,11 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
 
         if (prep != null) {
             if (prep.isDone())
-                finishCommit(prep, fut);
+                finishTx(true, prep, fut);
             else {
                 prep.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> f) {
-                        finishCommit(f, fut);
+                        finishTx(true, f, fut);
                     }
                 });
             }
@@ -577,7 +582,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         else {
             assert optimistic();
 
-            finishCommit(null, fut);
+            finishTx(true, null, fut);
         }
 
         return fut;
@@ -590,70 +595,26 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
         PREP_FUT_UPD.compareAndSet(this, fut, null);
     }
 
-    /**
-     * @param prepFut Prepare future.
-     * @param fut Finish future.
-     */
-    private void finishRollback(@Nullable IgniteInternalFuture prepFut, GridDhtTxFinishFuture fut) {
-        try {
-            if (prepFut != null)
-                prepFut.get();
-        }
-        catch (IgniteCheckedException e) {
-            if (log.isDebugEnabled())
-                log.debug("Failed to prepare or rollback transaction [tx=" + this + ", e=" + e + ']');
-        }
-
-        boolean primarySync = syncMode() == PRIMARY_SYNC;
-
-        IgniteCheckedException err = null;
-
-        try {
-            if (finish(false) || state() == UNKNOWN) {
-                if (primarySync)
-                    sendFinishReply(false, null);
-
-                fut.finish();
-            }
-            else {
-                err = new IgniteCheckedException("Failed to rollback transaction: " +
-                    CU.txString(GridDhtTxLocal.this));
-
-                fut.onError(err);
-            }
-        }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to gracefully rollback transaction: " + CU.txString(GridDhtTxLocal.this),
-                e);
-
-            err = e;
-
-            fut.onError(e);
-        }
-
-        if (primarySync && err != null)
-            sendFinishReply(false, err);
-    }
-
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<IgniteInternalTx> rollbackAsync() {
-        GridDhtTxPrepareFuture prepFut = this.prepFut;
-
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*rollback*/false);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, false);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
+        GridDhtTxPrepareFuture prepFut = this.prepFut;
+
         if (prepFut != null) {
             prepFut.complete();
 
             prepFut.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> f) {
-                    finishRollback(f, fut);
+                    finishTx(false, f, fut);
                 }
             });
         }
         else
-            finishRollback(null, fut);
+            finishTx(false, null, fut);
 
         return fut;
     }
@@ -672,7 +633,7 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
     }
 
     /** {@inheritDoc} */
-    @Override protected void sendFinishReply(boolean commit, @Nullable Throwable err) {
+    @Override protected void sendFinishReply(@Nullable Throwable err) {
         if (nearFinFutId != null) {
             if (nearNodeId.equals(cctx.localNodeId())) {
                 if (log.isDebugEnabled())
@@ -701,8 +662,8 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
                 }
             }
             catch (Throwable ex) {
-                U.error(log, "Failed to send finish response to node (transaction was " +
-                    (commit ? "committed" : "rolledback") + ") [txId=" + nearXidVersion() +
+                U.error(log, "Failed to send finish response to node [txId=" + nearXidVersion() +
+                    ", txState=" + state() +
                     ", dhtTxId=" + xidVersion() +
                     ", node=" + nearNodeId +
                     ", res=" + res + ']', ex);

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index 35dfb62..1d88d84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -236,10 +236,9 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
         AffinityTopologyVersion topVer);
 
     /**
-     * @param commit Commit flag.
      * @param err Error, if any.
      */
-    protected abstract void sendFinishReply(boolean commit, @Nullable Throwable err);
+    protected abstract void sendFinishReply(@Nullable Throwable err);
 
     /** {@inheritDoc} */
     @Override public boolean needsCompletedVersions() {
@@ -249,7 +248,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
     /**
      * @return Versions for all pending locks that were in queue before tx locks were released.
      */
-    public Collection<GridCacheVersion> pendingVersions() {
+    Collection<GridCacheVersion> pendingVersions() {
         return pendingVers == null ? Collections.<GridCacheVersion>emptyList() : pendingVers;
     }
 
@@ -726,7 +725,7 @@ public abstract class GridDhtTxLocalAdapter extends IgniteTxLocalAdapter {
                         /*read*/read,
                         accessTtl,
                         filter == null ? CU.empty0() : filter,
-                        /**computeInvoke*/false);
+                        /*computeInvoke*/false);
 
                     return ret;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 54bd543..9acab56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -298,34 +298,16 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             if (isDone())
                 return false;
 
-            if (err != null) {
-                tx.commitError(err);
-
-                boolean marked = tx.setRollbackOnly();
-
-                if (err instanceof IgniteTxRollbackCheckedException) {
-                    if (marked) {
-                        try {
-                            tx.rollback();
-                        }
-                        catch (IgniteCheckedException ex) {
-                            U.error(log, "Failed to automatically rollback transaction: " + tx, ex);
-                        }
-                    }
-                }
-                else if (tx.implicit() && tx.isSystemInvalidate()) { // Finish implicit transaction on heuristic error.
-                    try {
-                        tx.close();
-                    }
-                    catch (IgniteCheckedException ex) {
-                        U.error(log, "Failed to invalidate transaction: " + tx, ex);
-                    }
-                }
+            if (err != null)
+                tx.setRollbackOnly();
+
+            if (commit) {
+                if (tx.commitError() != null)
+                    err = tx.commitError();
+                else if (err != null)
+                    tx.commitError(err);
             }
 
-            if (commit && tx.commitError() != null)
-                err = tx.commitError();
-
             if (initialized() || err != null) {
                 if (tx.needCheckBackup()) {
                     assert tx.onePhaseCommit();
@@ -349,7 +331,15 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
                     finishOnePhase(commit);
 
-                    tx.tmFinish(commit);
+                    try {
+                        tx.tmFinish(commit);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to finish tx: " + tx, e);
+
+                        if (err == null)
+                            err = e;
+                    }
                 }
 
                 if (super.onDone(tx0, err)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index ed37059..0730300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -231,7 +231,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override protected void sendFinishReply(boolean commit, @Nullable Throwable err) {
+    @Override protected void sendFinishReply(@Nullable Throwable err) {
         // We are in near transaction, do not send finish reply to local node.
     }
 
@@ -1062,50 +1062,48 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
             return new GridFinishedFuture<IgniteInternalTx>(this);
         }
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, true);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
         if (prep == null || prep.isDone()) {
             assert prep != null || optimistic();
 
+            IgniteCheckedException err = null;
+
             try {
                 if (prep != null)
                     prep.get(); // Check for errors of a parent future.
-
-                fut.finish();
-            }
-            catch (IgniteTxOptimisticCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed optimistically to prepare transaction [tx=" + this + ", e=" + e + ']');
-
-                fut.onError(e);
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to prepare transaction: " + this, e);
+                err = e;
 
-                fut.onError(e);
+                U.error(log, "Failed to prepare transaction: " + this, e);
             }
+
+            if (err != null)
+                fut.rollbackOnError(err);
+            else
+                fut.finish(true);
         }
         else
             prep.listen(new CI1<IgniteInternalFuture<?>>() {
                 @Override public void apply(IgniteInternalFuture<?> f) {
+                    IgniteCheckedException err = null;
+
                     try {
                         f.get(); // Check for errors of a parent future.
-
-                        fut.finish();
-                    }
-                    catch (IgniteTxOptimisticCheckedException e) {
-                        if (log.isDebugEnabled())
-                            log.debug("Failed optimistically to prepare transaction [tx=" + this + ", e=" + e + ']');
-
-                        fut.onError(e);
                     }
                     catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to prepare transaction: " + this, e);
+                        err = e;
 
-                        fut.onError(e);
+                        U.error(log, "Failed to prepare transaction: " + this, e);
                     }
+
+                    if (err != null)
+                        fut.rollbackOnError(err);
+                    else
+                        fut.finish(true);
                 }
             });
 
@@ -1121,7 +1119,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
         if (log.isDebugEnabled())
             log.debug("Rolling back colocated tx locally: " + this);
 
-        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/false);
+        final GridDhtTxFinishFuture fut = new GridDhtTxFinishFuture<>(cctx, this, false);
 
         cctx.mvcc().addFuture(fut, fut.futureId());
 
@@ -1138,7 +1136,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                         e.getMessage() + ']');
             }
 
-            fut.finish();
+            fut.finish(false);
         }
         else
             prep.listen(new CI1<IgniteInternalFuture<?>>() {
@@ -1151,7 +1149,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter {
                             e.getMessage() + ']');
                     }
 
-                    fut.finish();
+                    fut.finish(false);
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 4b99079..b6e3c48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -472,13 +472,8 @@ public class IgniteTxHandler {
                 req.last());
 
             if (tx.isRollbackOnly() && !tx.commitOnPrepare()) {
-                try {
-                    if (tx.state() != TransactionState.ROLLED_BACK && tx.state() != TransactionState.ROLLING_BACK)
-                        tx.rollback();
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to rollback transaction: " + tx, e);
-                }
+                if (tx.state() != TransactionState.ROLLED_BACK && tx.state() != TransactionState.ROLLING_BACK)
+                    tx.rollbackAsync();
             }
 
             final GridDhtTxLocal tx0 = tx;
@@ -872,7 +867,7 @@ public class IgniteTxHandler {
 
             U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
 
-            IgniteInternalFuture<IgniteInternalTx> res = null;
+            IgniteInternalFuture<IgniteInternalTx> res;
 
             IgniteInternalFuture<IgniteInternalTx> rollbackFut = tx.rollbackAsync();
 
@@ -884,7 +879,7 @@ public class IgniteTxHandler {
             if (e instanceof Error)
                 throw (Error)e;
 
-            return res == null ? new GridFinishedFuture<IgniteInternalTx>(e) : res;
+            return res;
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 393fb1a..8d0a2b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1000,8 +1000,9 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
      * Commits transaction to transaction manager. Used for one-phase commit transactions only.
      *
      * @param commit If {@code true} commits transaction, otherwise rollbacks.
+     * @throws IgniteCheckedException If failed.
      */
-    public void tmFinish(boolean commit) {
+    public void tmFinish(boolean commit) throws IgniteCheckedException {
         assert onePhaseCommit();
 
         if (DONE_FLAG_UPD.compareAndSet(this, 0, 1)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/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 e2e9868..036fb0a 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
@@ -1191,8 +1191,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * Commits a transaction.
      *
      * @param tx Transaction to commit.
+     * @throws IgniteCheckedException If failed.
      */
-    public void commitTx(IgniteInternalTx tx) {
+    public void commitTx(IgniteInternalTx tx) throws IgniteCheckedException {
         assert tx != null;
         assert tx.state() == COMMITTING : "Invalid transaction state for commit from tm [state=" + tx.state() +
             ", expected=COMMITTING, tx=" + tx + ']';
@@ -1210,12 +1211,12 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         Boolean committed = committed0 != null && !committed0.equals(Boolean.FALSE);
 
         // 1. Make sure that committed version has been recorded.
-        if (!((committed != null && committed) || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) {
+        if (!(committed || tx.writeSet().isEmpty() || tx.isSystemInvalidate())) {
             uncommitTx(tx);
 
             tx.errorWhenCommitting();
 
-            throw new IgniteException("Missing commit version (consider increasing " +
+            throw new IgniteCheckedException("Missing commit version (consider increasing " +
                 IGNITE_MAX_COMPLETED_TX_COUNT + " system property) [ver=" + tx.xidVersion() +
                 ", tx=" + tx.getClass().getSimpleName() + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
index 19e49f3..ac56d18 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
@@ -43,7 +43,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
     private volatile boolean putFailed;
 
     /** */
-    private String maxCompletedTxCount;
+    private String maxCompletedTxCnt;
 
     /**
      */
@@ -53,7 +53,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration() throws Exception {
-        maxCompletedTxCount = System.getProperty(IGNITE_MAX_COMPLETED_TX_COUNT);
+        maxCompletedTxCnt = System.getProperty(IGNITE_MAX_COMPLETED_TX_COUNT);
 
         System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, String.valueOf(5));
 
@@ -78,7 +78,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
-        System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, maxCompletedTxCount != null ? maxCompletedTxCount : "");
+        System.setProperty(IGNITE_MAX_COMPLETED_TX_COUNT, maxCompletedTxCnt != null ? maxCompletedTxCnt : "");
 
         super.afterTest();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c1fde05d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
index b65b441..795ab81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
@@ -357,6 +357,7 @@ public abstract class IgniteTxStoreExceptionAbstractSelfTest extends GridCacheAb
 
     /**
      * @param key Key.
+     * @param putBefore If {@code true} expects non-null values.
      * @throws Exception If failed.
      */
     private void checkValue(final Integer key, boolean putBefore) throws Exception {


[20/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-1.7.4' into ignite-1.8.2

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

# Conflicts:
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
#	modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/ignite-2.0
Commit: 09c634e28200e0dd94c3c2e2271b5cdf60304888
Parents: 64247b9
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 15:55:02 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 15:55:02 2016 +0300

----------------------------------------------------------------------
 .../cache/store/cassandra/CassandraCacheStore.java  |  3 +++
 .../org/apache/ignite/internal/IgniteKernal.java    | 16 ----------------
 2 files changed, 3 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/09c634e2/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
----------------------------------------------------------------------
diff --git a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
index b4bed0d..70d798b 100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
@@ -43,6 +43,9 @@ import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant;
 import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant;
 import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker;
+import org.apache.ignite.cache.store.cassandra.session.transaction.DeleteMutation;
+import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
+import org.apache.ignite.cache.store.cassandra.session.transaction.WriteMutation;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiInClosure;

http://git-wip-us.apache.org/repos/asf/ignite/blob/09c634e2/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 e4a6115..4972d1f 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
@@ -2877,22 +2877,6 @@ 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();


[34/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/services/Messages.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/Messages.service.js b/modules/web-console/frontend/app/services/Messages.service.js
index e679488..fefdae9 100644
--- a/modules/web-console/frontend/app/services/Messages.service.js
+++ b/modules/web-console/frontend/app/services/Messages.service.js
@@ -24,6 +24,9 @@ export default ['IgniteMessages', ['$alert', ($alert) => {
         prefix = prefix || '';
 
         if (err) {
+            if (err.hasOwnProperty('data'))
+                err = err.data;
+
             if (err.hasOwnProperty('message'))
                 return prefix + err.message;
 
@@ -38,26 +41,26 @@ export default ['IgniteMessages', ['$alert', ($alert) => {
             msgModal.hide();
     };
 
-    const _showMessage = (err, type, duration, icon) => {
+    const _showMessage = (message, err, type, duration) => {
         hideAlert();
 
-        const title = errorMessage(null, err);
+        const title = err ? errorMessage(message, err) : errorMessage(null, message);
 
         msgModal = $alert({type, title, duration});
 
-        msgModal.$scope.icon = icon;
+        msgModal.$scope.icon = `icon-${type}`;
     };
 
     return {
         errorMessage,
         hideAlert,
-        showError(err) {
-            _showMessage(err, 'danger', 10, 'fa-exclamation-triangle');
+        showError(message, err) {
+            _showMessage(message, err, 'danger', 10);
 
             return false;
         },
-        showInfo(err) {
-            _showMessage(err, 'success', 3, 'fa-check-circle-o');
+        showInfo(message) {
+            _showMessage(message, null, 'success', 3);
         }
     };
 }]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/admin-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/admin-controller.js b/modules/web-console/frontend/controllers/admin-controller.js
index 7004301..cf7fd71 100644
--- a/modules/web-console/frontend/controllers/admin-controller.js
+++ b/modules/web-console/frontend/controllers/admin-controller.js
@@ -15,79 +15,220 @@
  * limitations under the License.
  */
 
+const ICON_SORT = '<span ui-grid-one-bind-id-grid="col.uid + \'-sortdir-text\'" ui-grid-visible="col.sort.direction" aria-label="Sort Descending"><i ng-class="{ \'ui-grid-icon-up-dir\': col.sort.direction == asc, \'ui-grid-icon-down-dir\': col.sort.direction == desc, \'ui-grid-icon-blank\': !col.sort.direction }" title="" aria-hidden="true"></i></span>';
+
+const CLUSTER_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-sitemap'></i>${ICON_SORT}</div>`;
+const MODEL_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-object-group'></i>${ICON_SORT}</div>`;
+const CACHE_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-database'></i>${ICON_SORT}</div>`;
+const IGFS_HEADER_TEMPLATE = `<div class='ui-grid-cell-contents' bs-tooltip data-title='{{ col.headerTooltip(col) }}' data-placement='top'><i class='fa fa-folder-o'></i>${ICON_SORT}</div>`;
+
+const ACTIONS_TEMPLATE = `
+<div class='text-center ui-grid-cell-actions'>
+    <a class='btn btn-default dropdown-toggle' bs-dropdown='' ng-show='row.entity._id != $root.user._id' data-placement='bottom-right' data-container='.panel'>
+        <i class='fa fa-gear'></i>&nbsp;
+        <span class='caret'></span>
+    </a>
+    <ul class='dropdown-menu' role='menu'>
+        <li>
+            <a ng-click='grid.api.becomeUser(row.entity)'>Become this user</a>
+        </li>
+        <li>
+            <a ng-click='grid.api.toggleAdmin(row.entity)' ng-if='row.entity.admin && row.entity._id !== $root.user._id'>Revoke admin</a>
+            <a ng-click='grid.api.toggleAdmin(row.entity)' ng-if='!row.entity.admin && row.entity._id !== $root.user._id'>Grant admin</a>
+        </li>
+        <li>
+            <a ng-click='grid.api.removeUser(row.entity)'>Remove user</a>
+        </li>
+</div>`;
+
+const EMAIL_TEMPLATE = '<div class="ui-grid-cell-contents"><a ng-href="mailto:{{ COL_FIELD }}">{{ COL_FIELD }}</a></div>';
+
 // Controller for Admin screen.
 export default ['adminController', [
-    '$rootScope', '$scope', '$http', '$q', '$state', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries',
-    ($rootScope, $scope, $http, $q, $state, Messages, Confirm, User, Notebook, Countries) => {
+    '$rootScope', '$scope', '$http', '$q', '$state', '$filter', 'uiGridConstants', 'IgniteMessages', 'IgniteConfirm', 'User', 'IgniteNotebookData', 'IgniteCountries',
+    ($rootScope, $scope, $http, $q, $state, $filter, uiGridConstants, Messages, Confirm, User, Notebook, Countries) => {
         $scope.users = null;
 
-        const _reloadUsers = () => {
-            $http.post('/api/v1/admin/list')
-                .success((users) => {
-                    $scope.users = users;
+        const companySelectOptions = [];
+        const countrySelectOptions = [];
 
-                    _.forEach($scope.users, (user) => {
-                        user.userName = user.firstName + ' ' + user.lastName;
-                        user.countryCode = Countries.getByName(user.country).code;
-                        user.label = user.userName + ' ' + user.email + ' ' +
-                            (user.company || '') + ' ' + (user.countryCode || '');
-                    });
-                })
-                .error(Messages.showError);
-        };
+        const COLUMNS_DEFS = [
+            {displayName: 'Actions', cellTemplate: ACTIONS_TEMPLATE, field: 'test', minWidth: 80, width: 80, enableFiltering: false, enableSorting: false},
+            {displayName: 'User', field: 'userName', minWidth: 65, enableFiltering: true, filter: { placeholder: 'Filter by name...' }},
+            {displayName: 'Email', field: 'email', cellTemplate: EMAIL_TEMPLATE, minWidth: 160, enableFiltering: true, filter: { placeholder: 'Filter by email...' }},
+            {displayName: 'Company', field: 'company', minWidth: 160, filter: {
+                selectOptions: companySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT }
+            },
+            {displayName: 'Country', field: 'countryCode', minWidth: 80, filter: {
+                selectOptions: countrySelectOptions, type: uiGridConstants.filter.SELECT, condition: uiGridConstants.filter.EXACT }
+            },
+            {displayName: 'Last login', field: 'lastLogin', cellFilter: 'date:"medium"', minWidth: 175, width: 175, enableFiltering: false, sort: { direction: 'desc', priority: 0 }},
+            {displayName: 'Clusters count', headerCellTemplate: CLUSTER_HEADER_TEMPLATE, field: '_clusters', type: 'number', headerTooltip: 'Clusters count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'Models count', headerCellTemplate: MODEL_HEADER_TEMPLATE, field: '_models', type: 'number', headerTooltip: 'Models count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'Caches count', headerCellTemplate: CACHE_HEADER_TEMPLATE, field: '_caches', type: 'number', headerTooltip: 'Caches count', minWidth: 50, width: 50, enableFiltering: false},
+            {displayName: 'IGFS count', headerCellTemplate: IGFS_HEADER_TEMPLATE, field: '_igfs', type: 'number', headerTooltip: 'IGFS count', minWidth: 50, width: 50, enableFiltering: false}
+        ];
 
-        _reloadUsers();
+        const ctrl = $scope.ctrl = {};
 
-        $scope.becomeUser = function(user) {
+        const becomeUser = function(user) {
             $http.get('/api/v1/admin/become', { params: {viewedUserId: user._id}})
-                .catch(({data}) => Promise.reject(data))
                 .then(() => User.load())
-                .then((becomeUser) => {
-                    $rootScope.$broadcast('user', becomeUser);
-
-                    $state.go('base.configuration.clusters');
-                })
+                .then(() => $state.go('base.configuration.clusters'))
                 .then(() => Notebook.load())
                 .catch(Messages.showError);
         };
 
-        $scope.removeUser = (user) => {
-            Confirm.confirm('Are you sure you want to remove user: "' + user.userName + '"?')
+        const removeUser = (user) => {
+            Confirm.confirm(`Are you sure you want to remove user: "${user.userName}"?`)
                 .then(() => {
                     $http.post('/api/v1/admin/remove', {userId: user._id})
-                        .success(() => {
+                        .then(() => {
                             const i = _.findIndex($scope.users, (u) => u._id === user._id);
 
                             if (i >= 0)
                                 $scope.users.splice(i, 1);
 
-                            Messages.showInfo('User has been removed: "' + user.userName + '"');
+                            Messages.showInfo(`User has been removed: "${user.userName}"`);
                         })
-                        .error((err, status) => {
+                        .catch(({data, status}) => {
                             if (status === 503)
-                                Messages.showInfo(err);
+                                Messages.showInfo(data);
                             else
-                                Messages.showError(Messages.errorMessage('Failed to remove user: ', err));
+                                Messages.showError('Failed to remove user: ', data);
                         });
                 });
         };
 
-        $scope.toggleAdmin = (user) => {
+        const toggleAdmin = (user) => {
             if (user.adminChanging)
                 return;
 
             user.adminChanging = true;
 
             $http.post('/api/v1/admin/save', {userId: user._id, adminFlag: !user.admin})
-                .success(() => {
+                .then(() => {
                     user.admin = !user.admin;
 
-                    Messages.showInfo('Admin right was successfully toggled for user: "' + user.userName + '"');
+                    Messages.showInfo(`Admin right was successfully toggled for user: "${user.userName}"`);
                 })
-                .error((err) => {
-                    Messages.showError(Messages.errorMessage('Failed to toggle admin right for user: ', err));
+                .catch((res) => {
+                    Messages.showError('Failed to toggle admin right for user: ', res);
                 })
                 .finally(() => user.adminChanging = false);
         };
+
+
+        ctrl.gridOptions = {
+            data: [],
+            columnVirtualizationThreshold: 30,
+            columnDefs: COLUMNS_DEFS,
+            categories: [
+                {name: 'Actions', visible: true, selectable: true},
+                {name: 'User', visible: true, selectable: true},
+                {name: 'Email', visible: true, selectable: true},
+                {name: 'Company', visible: true, selectable: true},
+                {name: 'Country', visible: true, selectable: true},
+                {name: 'Last login', visible: true, selectable: true},
+
+                {name: 'Clusters count', visible: true, selectable: true},
+                {name: 'Models count', visible: true, selectable: true},
+                {name: 'Caches count', visible: true, selectable: true},
+                {name: 'IGFS count', visible: true, selectable: true}
+            ],
+            enableFiltering: true,
+            enableRowSelection: false,
+            enableRowHeaderSelection: false,
+            enableColumnMenus: false,
+            multiSelect: false,
+            modifierKeysToMultiSelect: true,
+            noUnselect: true,
+            flatEntityAccess: true,
+            fastWatch: true,
+            onRegisterApi: (api) => {
+                ctrl.gridApi = api;
+
+                api.becomeUser = becomeUser;
+                api.removeUser = removeUser;
+                api.toggleAdmin = toggleAdmin;
+            }
+        };
+
+        /**
+         * Set grid height.
+         *
+         * @param {Number} rows Rows count.
+         * @private
+         */
+        const adjustHeight = (rows) => {
+            const height = Math.min(rows, 20) * 30 + 75;
+
+            // Remove header height.
+            ctrl.gridApi.grid.element.css('height', height + 'px');
+
+            ctrl.gridApi.core.handleWindowResize();
+        };
+
+        const usersToFilterOptions = (column) => {
+            return _.sortBy(
+                _.map(
+                    _.groupBy($scope.users, (usr) => {
+                        const fld = usr[column];
+
+                        return _.isNil(fld) ? fld : fld.toUpperCase();
+                    }),
+                    (arr, value) => ({label: `${_.head(arr)[column] || 'Not set'} (${arr.length})`, value})
+                ),
+                'value');
+        };
+
+        const _reloadUsers = () => {
+            $http.post('/api/v1/admin/list')
+                .then(({ data }) => {
+                    $scope.users = data;
+
+                    companySelectOptions.length = 0;
+                    countrySelectOptions.length = 0;
+
+                    _.forEach($scope.users, (user) => {
+                        user.userName = user.firstName + ' ' + user.lastName;
+                        user.countryCode = Countries.getByName(user.country).code;
+
+                        user._clusters = user.counters.clusters;
+                        user._models = user.counters.models;
+                        user._caches = user.counters.caches;
+                        user._igfs = user.counters.igfs;
+                    });
+
+                    companySelectOptions.push(...usersToFilterOptions('company'));
+                    countrySelectOptions.push(...usersToFilterOptions('countryCode'));
+
+                    $scope.ctrl.gridOptions.data = data;
+
+                    adjustHeight(data.length);
+                })
+                .catch(Messages.showError);
+        };
+
+        _reloadUsers();
+
+        const _enableColumns = (categories, visible) => {
+            _.forEach(categories, (cat) => {
+                cat.visible = visible;
+
+                _.forEach(ctrl.gridOptions.columnDefs, (col) => {
+                    if (col.displayName === cat.name)
+                        col.visible = visible;
+                });
+            });
+
+            ctrl.gridApi.grid.refresh();
+        };
+
+        const _selectableColumns = () => _.filter(ctrl.gridOptions.categories, (cat) => cat.selectable);
+
+        ctrl.toggleColumns = (category, visible) => _enableColumns([category], visible);
+        ctrl.selectAllColumns = () => _enableColumns(_selectableColumns(), true);
+        ctrl.clearAllColumns = () => _enableColumns(_selectableColumns(), false);
     }
 ]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/caches-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/caches-controller.js b/modules/web-console/frontend/controllers/caches-controller.js
index 8c01173..e7521b5 100644
--- a/modules/web-console/frontend/controllers/caches-controller.js
+++ b/modules/web-console/frontend/controllers/caches-controller.js
@@ -467,14 +467,14 @@ export default ['cachesController', [
         // Save cache in database.
         function save(item) {
             $http.post('/api/v1/configuration/caches/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     item.label = _cacheLbl(item);
 
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.caches, function(cache) {
-                        return cache._id === _id;
-                    });
+                    const idx = _.findIndex($scope.caches, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.caches[idx], item);
@@ -487,21 +487,21 @@ export default ['cachesController', [
                         if (_.includes(item.clusters, cluster.value))
                             cluster.caches = _.union(cluster.caches, [_id]);
                         else
-                            _.remove(cluster.caches, (id) => id === _id);
+                            _.pull(cluster.caches, _id);
                     });
 
                     _.forEach($scope.domains, (domain) => {
                         if (_.includes(item.domains, domain.value))
                             domain.meta.caches = _.union(domain.meta.caches, [_id]);
                         else
-                            _.remove(domain.meta.caches, (id) => id === _id);
+                            _.pull(domain.meta.caches, _id);
                     });
 
                     $scope.selectItem(item);
 
                     Messages.showInfo('Cache "' + item.name + '" saved.');
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save cache.
@@ -559,7 +559,7 @@ export default ['cachesController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/caches/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Cache has been removed: ' + selectedItem.name);
 
                             const caches = $scope.caches;
@@ -582,7 +582,7 @@ export default ['cachesController', [
                                 _.forEach($scope.domains, (domain) => _.remove(domain.meta.caches, (id) => id === _id));
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -591,7 +591,7 @@ export default ['cachesController', [
             Confirm.confirm('Are you sure you want to remove all caches?')
                 .then(function() {
                     $http.post('/api/v1/configuration/caches/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All caches have been removed');
 
                             $scope.caches = [];
@@ -603,7 +603,7 @@ export default ['cachesController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 f92a2f1..7f90b90 100644
--- a/modules/web-console/frontend/controllers/clusters-controller.js
+++ b/modules/web-console/frontend/controllers/clusters-controller.js
@@ -17,7 +17,7 @@
 
 // Controller for Clusters screen.
 export default ['clustersController', [
-    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'igniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
+    '$rootScope', '$scope', '$http', '$state', '$timeout', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteConfirm', 'IgniteClone', 'IgniteLoading', 'IgniteModelNormalizer', 'IgniteUnsavedChangesGuard', 'IgniteEventGroups', 'DemoInfo', 'IgniteLegacyTable', 'IgniteConfigurationResource', 'IgniteErrorPopover', 'IgniteFormUtils',
     function($root, $scope, $http, $state, $timeout, LegacyUtils, Messages, Confirm, Clone, Loading, ModelNormalizer, UnsavedChangesGuard, igniteEventGroups, DemoInfo, LegacyTable, Resource, ErrorPopover, FormUtils) {
         UnsavedChangesGuard.install($scope);
 
@@ -31,6 +31,12 @@ export default ['clustersController', [
             cacheKeyConfiguration: [],
             communication: {},
             connector: {},
+            deploymentSpi: {
+                URI: {
+                    uriList: [],
+                    scanners: []
+                }
+            },
             discovery: {
                 Cloud: {
                     regions: [],
@@ -38,6 +44,7 @@ export default ['clustersController', [
                 }
             },
             marshaller: {},
+            peerClassLoadingLocalClassPathExclude: [],
             sslContextFactory: {
                 trustManagers: []
             },
@@ -276,6 +283,16 @@ export default ['clustersController', [
 
                     if (!cluster.eventStorage)
                         cluster.eventStorage = { kind: 'Memory' };
+
+                    if (!cluster.peerClassLoadingLocalClassPathExclude)
+                        cluster.peerClassLoadingLocalClassPathExclude = [];
+
+                    if (!cluster.deploymentSpi) {
+                        cluster.deploymentSpi = {URI: {
+                            uriList: [],
+                            scanners: []
+                        }};
+                    }
                 });
 
                 if ($state.params.linkId)
@@ -699,17 +716,20 @@ export default ['clustersController', [
         // Save cluster in database.
         function save(item) {
             $http.post('/api/v1/configuration/clusters/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     item.label = _clusterLbl(item);
 
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.clusters, (cluster) => cluster._id === _id);
+                    const idx = _.findIndex($scope.clusters, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.clusters[idx], item);
                     else {
                         item._id = _id;
+
                         $scope.clusters.push(item);
                     }
 
@@ -717,21 +737,21 @@ export default ['clustersController', [
                         if (_.includes(item.caches, cache.value))
                             cache.cache.clusters = _.union(cache.cache.clusters, [_id]);
                         else
-                            _.remove(cache.cache.clusters, (id) => id === _id);
+                            _.pull(cache.cache.clusters, _id);
                     });
 
                     _.forEach($scope.igfss, (igfs) => {
                         if (_.includes(item.igfss, igfs.value))
                             igfs.igfs.clusters = _.union(igfs.igfs.clusters, [_id]);
                         else
-                            _.remove(igfs.igfs.clusters, (id) => id === _id);
+                            _.pull(igfs.igfs.clusters, _id);
                     });
 
                     $scope.selectItem(item);
 
-                    Messages.showInfo('Cluster "' + item.name + '" saved.');
+                    Messages.showInfo(`Cluster "${item.name}" saved.`);
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save cluster.
@@ -774,7 +794,7 @@ export default ['clustersController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/clusters/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Cluster has been removed: ' + selectedItem.name);
 
                             const clusters = $scope.clusters;
@@ -795,7 +815,7 @@ export default ['clustersController', [
                                 _.forEach($scope.igfss, (igfs) => _.remove(igfs.igfs.clusters, (id) => id === _id));
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -804,7 +824,7 @@ export default ['clustersController', [
             Confirm.confirm('Are you sure you want to remove all clusters?')
                 .then(function() {
                     $http.post('/api/v1/configuration/clusters/remove/all')
-                        .success(() => {
+                        .then(() => {
                             Messages.showInfo('All clusters have been removed');
 
                             $scope.clusters = [];
@@ -816,7 +836,7 @@ export default ['clustersController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 2d7b875..303110e 100644
--- a/modules/web-console/frontend/controllers/domains-controller.js
+++ b/modules/web-console/frontend/controllers/domains-controller.js
@@ -756,15 +756,15 @@ export default ['domainsController', [
                 Loading.start('importDomainFromDb');
 
                 $http.post('/api/v1/configuration/domains/save/batch', batch)
-                    .success(function(savedBatch) {
+                    .then(({data}) => {
                         let lastItem;
                         const newItems = [];
 
-                        _.forEach(_mapCaches(savedBatch.generatedCaches), function(cache) {
+                        _.forEach(_mapCaches(data.generatedCaches), function(cache) {
                             $scope.caches.push(cache);
                         });
 
-                        _.forEach(savedBatch.savedDomains, function(savedItem) {
+                        _.forEach(data.savedDomains, function(savedItem) {
                             const idx = _.findIndex($scope.domains, function(domain) {
                                 return domain._id === savedItem._id;
                             });
@@ -792,7 +792,7 @@ export default ['domainsController', [
 
                         $scope.ui.showValid = true;
                     })
-                    .error(Messages.showError)
+                    .catch(Messages.showError)
                     .finally(() => {
                         Loading.finish('importDomainFromDb');
 
@@ -1382,10 +1382,10 @@ export default ['domainsController', [
                 item.kind = 'store';
 
             $http.post('/api/v1/configuration/domains/save', item)
-                .success(function(res) {
+                .then(({data}) => {
                     $scope.ui.inputForm.$setPristine();
 
-                    const savedMeta = res.savedDomains[0];
+                    const savedMeta = data.savedDomains[0];
 
                     const idx = _.findIndex($scope.domains, function(domain) {
                         return domain._id === savedMeta._id;
@@ -1400,16 +1400,16 @@ export default ['domainsController', [
                         if (_.includes(item.caches, cache.value))
                             cache.cache.domains = _.union(cache.cache.domains, [savedMeta._id]);
                         else
-                            _.remove(cache.cache.domains, (id) => id === savedMeta._id);
+                            _.pull(cache.cache.domains, savedMeta._id);
                     });
 
                     $scope.selectItem(savedMeta);
 
-                    Messages.showInfo('Domain model "' + item.valueType + '" saved.');
+                    Messages.showInfo(`Domain model "${item.valueType}" saved.`);
 
                     _checkShowValidPresentation();
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save domain model.
@@ -1469,14 +1469,12 @@ export default ['domainsController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/domains/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('Domain model has been removed: ' + selectedItem.valueType);
 
                             const domains = $scope.domains;
 
-                            const idx = _.findIndex(domains, function(domain) {
-                                return domain._id === _id;
-                            });
+                            const idx = _.findIndex(domains, {_id});
 
                             if (idx >= 0) {
                                 domains.splice(idx, 1);
@@ -1488,12 +1486,12 @@ export default ['domainsController', [
                                 else
                                     $scope.backupItem = emptyDomain;
 
-                                _.forEach($scope.caches, (cache) => _.remove(cache.cache.domains, (id) => id === _id));
+                                _.forEach($scope.caches, (cache) => _.pull(cache.cache.domains, _id));
                             }
 
                             _checkShowValidPresentation();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -1504,7 +1502,7 @@ export default ['domainsController', [
             Confirm.confirm('Are you sure you want to remove all domain models?')
                 .then(function() {
                     $http.post('/api/v1/configuration/domains/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All domain models have been removed');
 
                             $scope.domains = [];
@@ -1516,7 +1514,7 @@ export default ['domainsController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/igfs-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/igfs-controller.js b/modules/web-console/frontend/controllers/igfs-controller.js
index e505f1c..b3c6043 100644
--- a/modules/web-console/frontend/controllers/igfs-controller.js
+++ b/modules/web-console/frontend/controllers/igfs-controller.js
@@ -296,12 +296,12 @@ export default ['igfsController', [
         // Save IGFS in database.
         function save(item) {
             $http.post('/api/v1/configuration/igfs/save', item)
-                .success(function(_id) {
+                .then(({data}) => {
+                    const _id = data;
+
                     $scope.ui.inputForm.$setPristine();
 
-                    const idx = _.findIndex($scope.igfss, function(igfs) {
-                        return igfs._id === _id;
-                    });
+                    const idx = _.findIndex($scope.igfss, {_id});
 
                     if (idx >= 0)
                         _.assign($scope.igfss[idx], item);
@@ -312,9 +312,9 @@ export default ['igfsController', [
 
                     $scope.selectItem(item);
 
-                    Messages.showInfo('IGFS "' + item.name + '" saved.');
+                    Messages.showInfo(`IGFS "${item.name}" saved.`);
                 })
-                .error(Messages.showError);
+                .catch(Messages.showError);
         }
 
         // Save IGFS.
@@ -359,7 +359,7 @@ export default ['igfsController', [
                     const _id = selectedItem._id;
 
                     $http.post('/api/v1/configuration/igfs/remove', {_id})
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('IGFS has been removed: ' + selectedItem.name);
 
                             const igfss = $scope.igfss;
@@ -379,7 +379,7 @@ export default ['igfsController', [
                                     $scope.backupItem = emptyIgfs;
                             }
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 
@@ -390,7 +390,7 @@ export default ['igfsController', [
             Confirm.confirm('Are you sure you want to remove all IGFS?')
                 .then(function() {
                     $http.post('/api/v1/configuration/igfs/remove/all')
-                        .success(function() {
+                        .then(() => {
                             Messages.showInfo('All IGFS have been removed');
 
                             $scope.igfss = [];
@@ -398,7 +398,7 @@ export default ['igfsController', [
                             $scope.ui.inputForm.$error = {};
                             $scope.ui.inputForm.$setPristine();
                         })
-                        .error(Messages.showError);
+                        .catch(Messages.showError);
                 });
         };
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/controllers/profile-controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/controllers/profile-controller.js b/modules/web-console/frontend/controllers/profile-controller.js
index fd595d9..87a8805 100644
--- a/modules/web-console/frontend/controllers/profile-controller.js
+++ b/modules/web-console/frontend/controllers/profile-controller.js
@@ -74,7 +74,6 @@ export default ['profileController', [
 
         $scope.saveUser = () => {
             $http.post('/api/v1/profile/save', $scope.user)
-                .catch(({data}) => Promise.reject(data))
                 .then(User.load)
                 .then(() => {
                     if ($scope.expandedPassword)
@@ -89,7 +88,7 @@ export default ['profileController', [
 
                     $root.$broadcast('user', $scope.user);
                 })
-                .catch((err) => Messages.showError(Messages.errorMessage('Failed to save profile: ', err)));
+                .catch((res) => Messages.showError('Failed to save profile: ', res));
         };
     }
 ]];

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 2463d24..7360ac4 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/common.js
@@ -20,7 +20,7 @@ import fs from 'fs';
 import webpack from 'webpack';
 import autoprefixer from 'autoprefixer-core';
 import jade from 'jade';
-import progressPlugin from './plugins/progress';
+import ProgressBarPlugin from 'progress-bar-webpack-plugin';
 import eslintFormatter from 'eslint-friendly-formatter';
 
 import ExtractTextPlugin from 'extract-text-webpack-plugin';
@@ -61,7 +61,6 @@ export default () => {
         // Output system.
         output: {
             path: destDir,
-            publicPath: './',
             filename: '[name].js'
         },
 
@@ -111,8 +110,10 @@ export default () => {
                     loader: 'babel-loader',
                     query: {
                         cacheDirectory: true,
-                        plugins: ['transform-runtime',
-                            'add-module-exports'],
+                        plugins: [
+                            'transform-runtime',
+                            'add-module-exports'
+                        ],
                         presets: ['angular']
 
                     }
@@ -126,10 +127,8 @@ export default () => {
                     loader: development ? `style-loader!${stylesLoader}` : ExtractTextPlugin.extract('style-loader', stylesLoader)
                 },
                 {
-                    test: /\.(woff2|woff|ttf|eot|svg)?(\?v=[0-9]\.[0-9]\.[0-9])?$/,
-                    loaders: [
-                        `${assetsLoader}?name=assets/fonts/[name].[ext]`
-                    ]
+                    test: /\.(ttf|eot|svg|woff(2)?)(\?v=[\d.]+)?(\?[a-z0-9#-]+)?$/,
+                    loaders: [`${assetsLoader}?name=assets/fonts/[name].[ext]`]
                 },
                 {
                     test: /\.(jpe?g|png|gif)$/i,
@@ -186,7 +185,7 @@ export default () => {
                 },
                 favicon
             }),
-            progressPlugin
+            new ProgressBarPlugin()
         ]
     };
 };

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 cad9133..34e1f6a 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
@@ -20,9 +20,8 @@ import webpack from 'webpack';
 
 import {destDir, rootDir, srcDir} from '../../paths';
 
-const devServerHost = 'localhost';
+const backendPort = 3000;
 const devServerPort = 9000;
-const devServerUrl = `http://${devServerHost}:${devServerPort}/`;
 
 export default () => {
     const plugins = [
@@ -31,11 +30,10 @@ export default () => {
 
     return {
         entry: {
-            webpack: `webpack-dev-server/client?${devServerUrl}`,
             app: [path.join(srcDir, 'app.js'), 'webpack/hot/only-dev-server']
         },
         output: {
-            publicPath: devServerUrl
+            publicPath: `http://localhost:${devServerPort}/`
         },
         context: rootDir,
         debug: true,
@@ -44,24 +42,22 @@ export default () => {
         devServer: {
             compress: true,
             historyApiFallback: true,
-            publicPath: '/',
             contentBase: destDir,
-            info: true,
             hot: true,
             inline: true,
             proxy: {
                 '/socket.io': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     ws: true
                 },
                 '/agents': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     ws: true
                 },
                 '/api/v1/*': {
-                    target: 'http://localhost:3000',
+                    target: `http://localhost:${backendPort}`,
                     changeOrigin: true,
                     pathRewrite: {
                         '^/api/v1': ''

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
index db66720..1194568 100644
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
+++ b/modules/web-console/frontend/gulpfile.babel.js/webpack/environments/production.js
@@ -37,8 +37,7 @@ export default () => {
         devtool: 'cheap-source-map',
         output: {
             publicPath: '/',
-            filename: '[name].[chunkhash].js',
-            path: destDir
+            filename: '[name].[chunkhash].js'
         },
         plugins
     };

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js b/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
deleted file mode 100644
index 5f753c7..0000000
--- a/modules/web-console/frontend/gulpfile.babel.js/webpack/plugins/progress.js
+++ /dev/null
@@ -1,82 +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 ProgressPlugin from 'webpack/lib/ProgressPlugin';
-
-let chars = 0;
-let lastState = 0;
-let lastStateTime = 0;
-
-const outputStream = process.stdout;
-
-const _goToLineStart = (nextMessage) => {
-    let str = '';
-
-    for (; chars > nextMessage.length; chars--)
-        str += '\b \b';
-
-    chars = nextMessage.length;
-
-    for (let i = 0; i < chars; i++)
-        str += '\b';
-
-    if (str)
-        outputStream.write(str);
-};
-
-export default new ProgressPlugin((percentage, msg) => {
-    let state = msg;
-
-    if (percentage < 1) {
-        percentage = Math.floor(percentage * 100);
-
-        msg = percentage + '% ' + msg;
-
-        if (percentage < 100)
-            msg = ' ' + msg;
-
-        if (percentage < 10)
-            msg = ' ' + msg;
-    }
-
-    state = state.replace(/^\d+\/\d+\s+/, '');
-
-    if (percentage === 0) {
-        lastState = null;
-        lastStateTime = (new Date()).getTime();
-    }
-    else if (state !== lastState || percentage === 1) {
-        const now = (new Date()).getTime();
-
-        if (lastState) {
-            const stateMsg = (now - lastStateTime) + 'ms ' + lastState;
-
-            _goToLineStart(stateMsg);
-
-            outputStream.write(stateMsg + '\n');
-
-            chars = 0;
-        }
-
-        lastState = state;
-        lastStateTime = now;
-    }
-
-    _goToLineStart(msg);
-
-    outputStream.write(msg);
-});

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/package.json
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/package.json b/modules/web-console/frontend/package.json
index b511ca1..fd50d5b 100644
--- a/modules/web-console/frontend/package.json
+++ b/modules/web-console/frontend/package.json
@@ -29,97 +29,99 @@
     "win32"
   ],
   "dependencies": {
-    "angular": "^1.5.5",
-    "angular-acl": "^0.1.7",
-    "angular-animate": "^1.5.5",
-    "angular-aria": "^1.5.5",
-    "angular-cookies": "^1.5.5",
-    "angular-drag-and-drop-lists": "^1.4.0",
-    "angular-gridster": "^0.13.3",
-    "angular-motion": "^0.4.4",
-    "angular-nvd3": "^1.0.7",
-    "angular-retina": "^0.3.13",
-    "angular-sanitize": "^1.5.5",
-    "angular-smart-table": "^2.1.8",
-    "angular-socket-io": "^0.7.0",
-    "angular-strap": "^2.3.8",
-    "angular-touch": "^1.5.5",
-    "angular-tree-control": "^0.2.26",
-    "angular-ui-grid": "^3.1.1",
-    "angular-ui-router": "^0.3.1",
-    "bootstrap-sass": "^3.3.6",
-    "brace": "^0.8.0",
-    "es6-promise": "^3.0.2",
-    "file-saver": "^1.3.2",
-    "font-awesome": "^4.6.3",
-    "glob": "^7.0.3",
-    "jquery": "^3.0.0",
-    "jszip": "^3.0.0",
-    "lodash": "^4.8.2",
-    "nvd3": "^1.8.3",
-    "raleway-webfont": "^3.0.1",
-    "roboto-font": "^0.1.0",
-    "socket.io-client": "^1.4.6",
-    "ui-router-metatags": "^1.0.3"
+    "angular": "~1.5.9",
+    "angular-acl": "~0.1.7",
+    "angular-animate": "~1.5.9",
+    "angular-aria": "~1.5.9",
+    "angular-cookies": "~1.5.9",
+    "angular-drag-and-drop-lists": "~1.4.0",
+    "angular-gridster": "~0.13.3",
+    "angular-motion": "~0.4.4",
+    "angular-nvd3": "~1.0.9",
+    "angular-retina": "~0.3.13",
+    "angular-sanitize": "~1.5.9",
+    "angular-smart-table": "~2.1.8",
+    "angular-socket-io": "~0.7.0",
+    "angular-strap": "~2.3.8",
+    "angular-touch": "~1.5.9",
+    "angular-tree-control": "~0.2.26",
+    "angular-ui-grid": "~3.2.9",
+    "angular-ui-router": "~0.3.1",
+    "bootstrap-sass": "~3.3.6",
+    "brace": "~0.8.0",
+    "es6-promise": "~3.3.1",
+    "file-saver": "~1.3.2",
+    "font-awesome": "~4.7.0",
+    "glob": "~7.1.1",
+    "jquery": "~3.1.1",
+    "jszip": "~3.1.3",
+    "lodash": "~4.17.2",
+    "nvd3": "1.8.4",
+    "raleway-webfont": "~3.0.1",
+    "roboto-font": "~0.1.0",
+    "socket.io-client": "~1.7.2",
+    "ui-router-metatags": "~1.0.3"
   },
   "devDependencies": {
-    "assets-webpack-plugin": "^3.2.0",
-    "autoprefixer-core": "^6.0.1",
-    "babel-core": "^6.7.6",
-    "babel-eslint": "^7.0.0",
-    "babel-loader": "^6.2.4",
-    "babel-plugin-add-module-exports": "^0.2.1",
-    "babel-plugin-transform-builtin-extend": "^1.1.0",
-    "babel-plugin-transform-runtime": "^6.7.5",
-    "babel-polyfill": "^6.7.4",
-    "babel-preset-angular": "^6.0.15",
-    "babel-preset-es2015": "^6.9.0",
-    "babel-runtime": "^6.6.1",
-    "chai": "^3.5.0",
-    "cross-env": "^1.0.7",
-    "css-loader": "^0.23.0",
-    "eslint": "^3.0.0",
-    "eslint-friendly-formatter": "^2.0.5",
-    "eslint-loader": "^1.0.0",
-    "expose-loader": "^0.7.1",
-    "extract-text-webpack-plugin": "^1.0.1",
-    "file-loader": "^0.9.0",
-    "gulp": "^3.9.1",
-    "gulp-eslint": "^3.0.0",
-    "gulp-inject": "^4.0.0",
-    "gulp-jade": "^1.1.0",
-    "gulp-ll": "^1.0.4",
-    "gulp-rimraf": "^0.2.0",
-    "gulp-sequence": "^0.4.1",
-    "gulp-util": "^3.0.7",
-    "html-loader": "^0.4.3",
-    "html-webpack-plugin": "^2.21.0",
-    "jade": "^1.11.0",
+    "assets-webpack-plugin": "~3.5.0",
+    "autoprefixer-core": "~6.0.1",
+    "babel-core": "~6.20.0",
+    "babel-eslint": "~7.0.0",
+    "babel-loader": "~6.2.4",
+    "babel-plugin-add-module-exports": "~0.2.1",
+    "babel-plugin-transform-builtin-extend": "~1.1.0",
+    "babel-plugin-transform-runtime": "~6.15.0",
+    "babel-polyfill": "~6.20.0",
+    "babel-preset-angular": "~6.0.15",
+    "babel-preset-es2015": "~6.18.0",
+    "babel-runtime": "~6.20.0",
+    "chai": "~3.5.0",
+    "cross-env": "~1.0.7",
+    "css-loader": "~0.23.0",
+    "eslint": "~3.12.2",
+    "eslint-friendly-formatter": "~2.0.5",
+    "eslint-loader": "~1.6.1",
+    "expose-loader": "~0.7.1",
+    "extract-text-webpack-plugin": "~1.0.1",
+    "file-loader": "~0.9.0",
+    "gulp": "~3.9.1",
+    "gulp-eslint": "~3.0.0",
+    "gulp-inject": "~4.1.0",
+    "gulp-jade": "~1.1.0",
+    "gulp-ll": "~1.0.4",
+    "gulp-rimraf": "~0.2.0",
+    "gulp-sequence": "~0.4.1",
+    "gulp-util": "~3.0.7",
+    "html-loader": "~0.4.3",
+    "html-webpack-plugin": "~2.24.1",
+    "jade": "~1.11.0",
     "jade-html-loader": "git://github.com/courcelan/jade-html-loader",
-    "jasmine-core": "^2.4.1",
-    "json-loader": "^0.5.4",
-    "karma": "^0.13.22",
-    "karma-babel-preprocessor": "^6.0.1",
-    "karma-jasmine": "^1.0.2",
-    "karma-mocha": "^1.0.1",
-    "karma-mocha-reporter": "^2.2.0",
-    "karma-phantomjs-launcher": "^1.0.0",
-    "karma-teamcity-reporter": "^1.0.0",
-    "karma-webpack": "^1.7.0",
+    "jasmine-core": "~2.5.2",
+    "json-loader": "~0.5.4",
+    "karma": "~0.13.22",
+    "karma-babel-preprocessor": "~6.0.1",
+    "karma-jasmine": "~1.1.0",
+    "karma-mocha": "~1.3.0",
+    "karma-mocha-reporter": "~2.2.0",
+    "karma-phantomjs-launcher": "~1.0.0",
+    "karma-teamcity-reporter": "~1.0.0",
+    "karma-webpack": "~1.8.0",
     "mocha": "~2.5.3",
-    "mocha-teamcity-reporter": "^1.0.0",
-    "morgan": "^1.7.0",
-    "ngtemplate-loader": "^1.3.1",
-    "node-sass": "^3.4.2",
-    "phantomjs-prebuilt": "^2.1.7",
-    "postcss-loader": "^0.9.1",
-    "require-dir": "^0.3.0",
-    "resolve-url-loader": "^1.4.3",
-    "sass-loader": "^3.1.1",
-    "style-loader": "^0.13.1",
-    "url": "^0.11.0",
-    "url-loader": "^0.5.6",
-    "webpack": "^1.13.1",
-    "webpack-dev-server": "^1.15.0"
+    "mocha-teamcity-reporter": "~1.1.1",
+    "morgan": "~1.7.0",
+    "ngtemplate-loader": "~1.3.1",
+    "node-sass": "~3.13.1",
+    "phantomjs-prebuilt": "~2.1.7",
+    "postcss-loader": "~0.9.1",
+    "progress-bar-webpack-plugin": "~1.9.0",
+    "require-dir": "~0.3.0",
+    "resolve-url-loader": "~1.6.1",
+    "sass-loader": "~3.1.1",
+    "style-loader": "~0.13.1",
+    "url": "~0.11.0",
+    "url-loader": "~0.5.6",
+    "webpack": "~1.14.0",
+    "webpack-dev-server": "~1.16.2",
+    "worker-loader": "~0.7.1"
   }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/cache.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/cache.png b/modules/web-console/frontend/public/images/cache.png
index 83fd987..3ff3103 100644
Binary files a/modules/web-console/frontend/public/images/cache.png and b/modules/web-console/frontend/public/images/cache.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/domains.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/domains.png b/modules/web-console/frontend/public/images/domains.png
index 39abfcb..41c0470 100644
Binary files a/modules/web-console/frontend/public/images/domains.png and b/modules/web-console/frontend/public/images/domains.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/igfs.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/igfs.png b/modules/web-console/frontend/public/images/igfs.png
index 47c659e..b62c27b 100644
Binary files a/modules/web-console/frontend/public/images/igfs.png and b/modules/web-console/frontend/public/images/igfs.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-chart.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-chart.png b/modules/web-console/frontend/public/images/query-chart.png
index c6e4cce..1b7ef41 100644
Binary files a/modules/web-console/frontend/public/images/query-chart.png and b/modules/web-console/frontend/public/images/query-chart.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-metadata.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-metadata.png b/modules/web-console/frontend/public/images/query-metadata.png
index 698cd6e..1b6c73c 100644
Binary files a/modules/web-console/frontend/public/images/query-metadata.png and b/modules/web-console/frontend/public/images/query-metadata.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/query-table.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/query-table.png b/modules/web-console/frontend/public/images/query-table.png
index 53becda..4d63a68 100644
Binary files a/modules/web-console/frontend/public/images/query-table.png and b/modules/web-console/frontend/public/images/query-table.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/images/summary.png
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/images/summary.png b/modules/web-console/frontend/public/images/summary.png
index ff88438..fda0abf 100644
Binary files a/modules/web-console/frontend/public/images/summary.png and b/modules/web-console/frontend/public/images/summary.png differ

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 15ee60c..bfa6c6c 100644
--- a/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
+++ b/modules/web-console/frontend/public/stylesheets/_font-awesome-custom.scss
@@ -47,4 +47,25 @@ $fa-font-path: '~font-awesome/fonts';
   @extend .fa-question-circle-o;
 
   cursor: default;
-}
\ No newline at end of file
+}
+
+.icon-note {
+  @extend .fa;
+  @extend .fa-info-circle;
+
+  cursor: default;
+}
+
+.icon-danger {
+  @extend .fa;
+  @extend .fa-exclamation-triangle;
+
+  cursor: default;
+}
+
+.icon-success {
+  @extend .fa;
+  @extend .fa-check-circle-o;
+
+  cursor: default;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/public/stylesheets/form-field.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/public/stylesheets/form-field.scss b/modules/web-console/frontend/public/stylesheets/form-field.scss
index f126786..ae33d75 100644
--- a/modules/web-console/frontend/public/stylesheets/form-field.scss
+++ b/modules/web-console/frontend/public/stylesheets/form-field.scss
@@ -106,3 +106,40 @@
         @include make-lg-column(8);
     }
 }
+
+.ignite-form-field {
+    &__btn {
+        overflow: hidden;
+
+        border-top-left-radius: 0;
+        border-bottom-left-radius: 0;
+
+        &.btn {
+            float: right;
+            margin-right: 0;
+
+            line-height: 20px;
+        }
+
+        input {
+            position: absolute;
+            left: 100px;
+        }
+
+        input:checked + span {
+            color: $brand-info;
+        }
+    }
+
+    &__btn ~ &__btn {
+        border-right: 0;
+        border-top-right-radius: 0;
+        border-bottom-right-radius: 0;  
+    }
+
+    &__btn ~ .input-tip input {
+        border-right: 0;
+        border-top-right-radius: 0;
+        border-bottom-right-radius: 0;  
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 172abf4..4318fc2 100644
--- a/modules/web-console/frontend/public/stylesheets/style.scss
+++ b/modules/web-console/frontend/public/stylesheets/style.scss
@@ -227,7 +227,7 @@ ul.navbar-nav, .sidebar-nav {
         overflow: hidden;
         white-space: nowrap;
         text-overflow: ellipsis;
-        
+
         &:hover,
         &:focus {
             text-decoration: none;
@@ -601,6 +601,10 @@ button.form-control {
 .theme-line .notebook-header {
     border-color: $gray-lighter;
 
+    button:last-child {
+        margin-right: 0;
+    }
+
     h1 {
         padding: 0;
         margin: 0;
@@ -611,7 +615,7 @@ button.form-control {
             overflow: hidden;
             text-overflow: ellipsis;
             white-space: nowrap;
-            margin-top: 5px;
+            height: 24px;
         }
 
         .btn-group {
@@ -637,7 +641,7 @@ button.form-control {
 }
 
 .theme-line .paragraphs {
-    .panel-group .panel + .panel {
+    .panel-group .panel-paragraph + .panel-paragraph {
         margin-top: 30px;
     }
 
@@ -679,8 +683,14 @@ button.form-control {
         line-height: 55px;
     }
 
-    .sql-controls {
+    .panel-collapse {
         border-top: 1px solid $ignite-border-color;
+    }
+
+    .sql-controls {
+        position: relative;
+        top: -1px;
+        border-top: 1px solid #ddd;
 
         padding: 10px 10px;
 
@@ -690,9 +700,50 @@ button.form-control {
         }
 
         label {
-            line-height: 20px !important;
+            line-height: 28px;
             vertical-align: middle;
         }
+
+        .btn {
+            line-height: 20px;
+        }
+
+        .ignite-form-field {
+            margin-right: 10px;
+
+            .ignite-form-field__label {
+                float: left;
+                width: auto;
+                margin-right: 5px;
+                line-height: 28px;
+            }
+
+            .ignite-form-field__label + div {
+                display: block;
+                float: none;
+                width: auto;
+            }
+        }
+
+        .tipLabel .btn {
+            float: right;
+        }
+
+        .pull-right {
+            margin-left: 10px;
+
+            .ignite-form-field {
+                margin-right: -24px;
+
+                label {
+                    margin-left: 5px;
+                }
+            }
+        }
+
+        .col-sm-3 + .tipLabel {
+            margin-left: 0;
+        }
     }
 
     .sql-result {
@@ -1243,6 +1294,17 @@ button.form-control {
     .clickable { cursor: pointer; }
 }
 
+
+.theme-line .summary {
+    .actions-note {
+        i {
+            margin-right: 5px;
+        }
+
+        margin: 15px 0;
+    }
+}
+
 .theme-line .popover.summary-project-structure {
     @extend .popover.settings;
 
@@ -1693,6 +1755,7 @@ th[st-sort] {
 }
 
 .chart-settings-link {
+    margin-top: -2px;
     padding-left: 10px;
     line-height: $input-height;
 
@@ -1991,6 +2054,10 @@ treecontrol.tree-classic {
         margin-right: 0
     }
 
+    .ui-grid-cell-actions {
+        line-height: 28px;
+    }
+
     .no-rows {
         .center-container {
             background: white;
@@ -2226,3 +2293,37 @@ html,body,.splash-screen {
         animation: none 0s;
     }
 }
+
+.admin-page {
+    .panel-heading {
+        border-bottom: 0;
+        padding-bottom: 0;
+
+        cursor: default;
+
+        i {
+            margin-right: 10px;
+        }
+
+        label {
+            cursor: default;
+            line-height: 24px;
+        }
+
+        sub {
+            bottom: 0;
+        }
+    }
+
+    .ui-grid-header-cell input {
+      font-weight: normal;
+    }
+
+    .ui-grid-header-cell input {
+      font-weight: normal;
+    }
+
+    .ui-grid-filter-select {
+        width: calc(100% - 10px);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/test/unit/JavaTypes.test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/test/unit/JavaTypes.test.js b/modules/web-console/frontend/test/unit/JavaTypes.test.js
index 2df8c6a..49e78cc 100644
--- a/modules/web-console/frontend/test/unit/JavaTypes.test.js
+++ b/modules/web-console/frontend/test/unit/JavaTypes.test.js
@@ -17,11 +17,11 @@
 
 import JavaTypes from '../../app/services/JavaTypes.service.js';
 
-import ClusterDflts from '../../app/modules/configuration/generator/defaults/cluster.provider';
-import CacheDflts from '../../app/modules/configuration/generator/defaults/cache.provider';
-import IgfsDflts from '../../app/modules/configuration/generator/defaults/igfs.provider';
+import ClusterDflts from '../../app/modules/configuration/generator/defaults/Cluster.service';
+import CacheDflts from '../../app/modules/configuration/generator/defaults/Cache.service';
+import IgfsDflts from '../../app/modules/configuration/generator/defaults/IGFS.service';
 
-const INSTANCE = new JavaTypes((new ClusterDflts()).$get[0](), (new CacheDflts()).$get[0](), (new IgfsDflts()).$get[0]());
+const INSTANCE = new JavaTypes(new ClusterDflts(), new CacheDflts(), new IgfsDflts());
 
 import { assert } from 'chai';
 
@@ -58,9 +58,14 @@ suite('JavaTypesTestsSuite', () => {
 
     test('shortClassName', () => {
         assert.equal(INSTANCE.shortClassName('java.math.BigDecimal'), 'BigDecimal');
+        assert.equal(INSTANCE.shortClassName('BigDecimal'), 'BigDecimal');
         assert.equal(INSTANCE.shortClassName('int'), 'int');
         assert.equal(INSTANCE.shortClassName('java.lang.Integer'), 'Integer');
+        assert.equal(INSTANCE.shortClassName('Integer'), 'Integer');
         assert.equal(INSTANCE.shortClassName('java.util.UUID'), 'UUID');
+        assert.equal(INSTANCE.shortClassName('java.sql.Date'), 'Date');
+        assert.equal(INSTANCE.shortClassName('Date'), 'Date');
+        assert.equal(INSTANCE.shortClassName('com.my.Abstract'), 'Abstract');
         assert.equal(INSTANCE.shortClassName('Abstract'), 'Abstract');
     });
 
@@ -113,8 +118,8 @@ suite('JavaTypesTestsSuite', () => {
         assert.equal(INSTANCE.isKeyword(' '), false);
     });
 
-    test('isJavaPrimitive', () => {
-        assert.equal(INSTANCE.isJavaPrimitive('boolean'), true);
+    test('isPrimitive', () => {
+        assert.equal(INSTANCE.isPrimitive('boolean'), true);
     });
 
     test('validUUID', () => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/test/unit/Version.test.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/test/unit/Version.test.js b/modules/web-console/frontend/test/unit/Version.test.js
index a67fde8..2d75ab5 100644
--- a/modules/web-console/frontend/test/unit/Version.test.js
+++ b/modules/web-console/frontend/test/unit/Version.test.js
@@ -39,7 +39,13 @@ suite('VersionServiceTestsSuite', () => {
     });
 
     test('Version a = b', () => {
-        assert.equal(INSTANCE.compare('1.7.0', '1.7.0'), 0);
+        assert.equal(INSTANCE.compare('1.0.0', '1.0.0'), 0);
+        assert.equal(INSTANCE.compare('1.2.0', '1.2.0'), 0);
+        assert.equal(INSTANCE.compare('1.2.3', '1.2.3'), 0);
+
+        assert.equal(INSTANCE.compare('1.0.0-1', '1.0.0-1'), 0);
+        assert.equal(INSTANCE.compare('1.2.0-1', '1.2.0-1'), 0);
+        assert.equal(INSTANCE.compare('1.2.3-1', '1.2.3-1'), 0);
     });
 
     test('Version a < b', () => {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 e4f95bc..bbcb391 100644
--- a/modules/web-console/frontend/views/configuration/domains-import.jade
+++ b/modules/web-console/frontend/views/configuration/domains-import.jade
@@ -62,7 +62,10 @@ mixin td-ellipses-lbl(w, lbl)
                             +ignite-form-field-dropdown('Driver JAR:', 'ui.selectedJdbcDriverJar', '"jdbcDriverJar"', false, true, false,
                                 'Choose JDBC driver', '', 'jdbcDriverJars',
                                 'Select appropriate JAR with JDBC driver<br> To add another driver you need to place it into "/jdbc-drivers" folder of Ignite Web Agent<br> Refer to Ignite Web Agent README.txt for for more information'
-                            )(data-container='.modal-domain-import')
+                            )(
+                                data-container='.modal-domain-import'
+                                data-ignite-form-field-input-autofocus='true'
+                            )
                         .settings-row.settings-row_small-label
                             +java-class('JDBC driver:', 'selectedPreset.jdbcDriverClass', '"jdbcDriverClass"', true, true, 'Fully qualified class name of JDBC driver that will be used to connect to database')
                         .settings-row.settings-row_small-label

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 9a6e553..a04f0db 100644
--- a/modules/web-console/frontend/views/configuration/summary.jade
+++ b/modules/web-console/frontend/views/configuration/summary.jade
@@ -21,7 +21,7 @@ mixin hard-link(ref, txt)
 
 .docs-header
     h1 Configurations Summary
-.docs-body
+.docs-body.summary
     ignite-information
         ul
             li Preview XML configurations for #[a(href='https://apacheignite.readme.io/docs/clients-vs-servers' target='_blank') server and client] nodes
@@ -29,7 +29,6 @@ mixin hard-link(ref, txt)
             li Preview #[a(href='https://apacheignite.readme.io/docs/docker-deployment' target='_blank') Docker file]
             li Preview POM dependencies
             li Download ready-to-use Maven project
-
     hr
     .padding-dflt(ng-if='ui.ready && (!clusters || clusters.length == 0)')
         | You have no clusters configured. Please configure them #[a(ui-sref='base.configuration.clusters') here].
@@ -37,13 +36,21 @@ mixin hard-link(ref, txt)
     div(ng-show='clusters && clusters.length > 0' ignite-loading='summaryPage' ignite-loading-text='Loading summary screen...' ignite-loading-position='top')
         +main-table('clusters', 'clustersView', 'clusterName', 'selectItem(row)', '{{$index + 1}}) {{row.name}}', 'name')
         div(ng-show='selectedItem && contentVisible(displayedRows, selectedItem)')
-            .padding-top-dflt(bs-affix)
-                button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom') Download project
-                .btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom')
-                    div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true')
-                        i.fa.fa-sitemap
-                        label.tipLabel Project structure
-                button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers
+            .actions.padding-top-dflt(bs-affix)
+                div
+                    button.btn.btn-primary(id='download' ng-click='downloadConfiguration()' bs-tooltip='' data-title='Download project' data-placement='bottom' ng-disabled='isPrepareDownloading')
+                        div
+                            i.fa.fa-fw.fa-download(ng-hide='isPrepareDownloading')
+                            i.fa.fa-fw.fa-refresh.fa-spin(ng-show='isPrepareDownloading')
+                            span.tipLabel Download project
+                    button.btn.btn-primary(bs-tooltip='' data-title='Preview generated project structure' data-placement='bottom')
+                        div(bs-popover data-template-url='/configuration/summary-project-structure.html', data-placement='bottom', data-trigger='click' data-auto-close='true')
+                            i.fa.fa-sitemap
+                            label.tipLabel Project structure
+                    button.btn.btn-primary(id='proprietary-jdbc-drivers' ng-if='downloadJdbcDriversVisible()' ng-click='downloadJdbcDrivers()' bs-tooltip='' data-title='Open proprietary JDBC drivers download pages' data-placement='bottom') Download JDBC drivers
+                .actions-note(ng-show='ui.isSafari')
+                    i.icon-note
+                    label "Download project" is not fully supported in Safari. Please rename downloaded file from "Unknown" to "&lt;project-name&gt;.zip"
                 hr
             .bs-affix-fix
             .panel-group(bs-collapse ng-init='ui.activePanels=[0,1]' ng-model='ui.activePanels' data-allow-multiple='true')

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/settings/admin.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/settings/admin.jade b/modules/web-console/frontend/views/settings/admin.jade
index 862d959..c985826 100644
--- a/modules/web-console/frontend/views/settings/admin.jade
+++ b/modules/web-console/frontend/views/settings/admin.jade
@@ -14,63 +14,38 @@
     See the License for the specific language governing permissions and
     limitations under the License.
 
-.row(ng-controller='adminController')
+mixin grid-settings()
+    i.fa.fa-bars(data-animation='am-flip-x' bs-dropdown='' aria-haspopup='true' aria-expanded='expanded' data-auto-close='1' data-trigger='click')
+    ul.select.dropdown-menu(role='menu')
+        li(ng-repeat='item in ctrl.gridOptions.categories|filter:{selectable:true}')
+            a(ng-click='ctrl.toggleColumns(item, !item.visible)')
+                i.fa.fa-check-square-o.pull-left(ng-if='item.visible')
+                i.fa.fa-square-o.pull-left(ng-if='!item.visible')
+                span {{::item.name}}
+        li.divider
+        li
+            a(ng-click='ctrl.selectAllColumns()') Select all
+        li
+            a(ng-click='ctrl.clearAllColumns()') Clear all
+        li.divider
+        li
+            a(ng-click='$hide()') Close
+
+.admin-page.row(ng-controller='adminController')
     .docs-content.greedy
         .docs-header
             h1 List of registered users
             hr
         .docs-body
-            .col-xs-12
-                table.table.table-striped.table-vertical-middle.admin(st-table='displayedUsers' st-safe-src='users')
-                    thead
-                        tr
-                            th.header(colspan='10')
-                                .col-xs-3
-                                    input.form-control(type='text' st-search='label' placeholder='Filter users...')
-                                .col-xs-9.admin-summary.text-right(colspan='10')
-                                    strong Total users: {{ users.length }}
-                                .col-xs-offset-6.col-xs-6.text-right
-                                    div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html')
-                        tr
-                            th(st-sort='userName') User
-                            th(st-sort='email') Email
-                            th(st-sort='company') Company
-                            th(st-sort='country') Country
-                            th.col-xs-2(st-sort='lastLogin' st-sort-default='reverse') Last login
-                            th.text-nowrap(st-sort='counters.clusters' st-descending-first bs-tooltip='"Clusters count"' data-placement='top')
-                                i.fa.fa-sitemap()
-                            th.text-nowrap(st-sort='counters.models' st-descending-first bs-tooltip='"Models count"' data-placement='top')
-                                i.fa.fa-object-group()
-                            th.text-nowrap(st-sort='counters.caches' st-descending-first bs-tooltip='"Caches count"' data-placement='top')
-                                i.fa.fa-database()
-                            th.text-nowrap(st-sort='counters.igfs' st-descending-first bs-tooltip='"IGFS count"' data-placement='top')
-                                i.fa.fa-folder-o()
-                            th(width='1%') Actions
-                    tbody
-                        tr(ng-repeat='row in displayedUsers track by row._id')
-                            td {{::row.userName}}
-                            td
-                                a(ng-href='mailto:{{::row.email}}') {{::row.email}}
-                            td {{::row.company}}
-                            td {{::row.countryCode}}
-                            td {{::row.lastLogin | date:'medium'}}
-                            td {{::row.counters.clusters}}
-                            td {{::row.counters.models}}
-                            td {{::row.counters.caches}}
-                            td {{::row.counters.igfs}}
-                            td.text-center
-                                a.btn.btn-default.dropdown-toggle(bs-dropdown='' ng-show='row._id != user._id' data-placement='bottom-right')
-                                    i.fa.fa-gear &nbsp;
-                                    span.caret
-                                ul.dropdown-menu(role='menu')
-                                    li
-                                        a(ng-click='becomeUser(row)') Become this user
-                                    li
-                                        a(ng-click='toggleAdmin(row)' ng-if='row.admin && row._id !== user._id') Revoke admin
-                                        a(ng-click='toggleAdmin(row)' ng-if='!row.admin && row._id !== user._id')  Grant admin
-                                    li
-                                        a(ng-click='removeUser(row)') Remove user
-                    tfoot
-                        tr
-                            td.text-right(colspan='10')
-                                div(st-pagination st-items-by-page='15' st-displayed-pages='5' st-template='../templates/pagination.html')
+            .row
+                .col-xs-12
+                    .panel.panel-default
+                        .panel-heading.ui-grid-settings
+                            +grid-settings
+                            label Total users: 
+                                strong {{ users.length }}&nbsp;&nbsp;&nbsp;
+                            label Showing users: 
+                                strong {{ ctrl.gridApi.grid.getVisibleRows().length }}
+                                sub(ng-show='users.length === ctrl.gridApi.grid.getVisibleRows().length') all
+                        .panel-collapse
+                            .grid(ui-grid='ctrl.gridOptions' ui-grid-resize-columns ui-grid-selection ui-grid-pinning)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/views/sql/notebook-new.jade
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/views/sql/notebook-new.jade b/modules/web-console/frontend/views/sql/notebook-new.jade
index 8d9e8c4..9585e92 100644
--- a/modules/web-console/frontend/views/sql/notebook-new.jade
+++ b/modules/web-console/frontend/views/sql/notebook-new.jade
@@ -21,7 +21,7 @@
                 button.close(ng-click='$hide()') &times;
                 h4.modal-title
                     i.fa.fa-file-o
-                    | New SQL notebook
+                    | New query notebook
             form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
                 div
                     .col-sm-2


[09/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1fe437c..0c90414 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -46,6 +46,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import javax.net.ssl.SSLEngine;
 import javax.net.ssl.SSLException;
 import org.apache.ignite.Ignite;
@@ -53,6 +54,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -103,6 +105,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -179,6 +182,7 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
  * <li>Node local IP address (see {@link #setLocalAddress(String)})</li>
  * <li>Node local port number (see {@link #setLocalPort(int)})</li>
  * <li>Local port range (see {@link #setLocalPortRange(int)}</li>
+ * <li>Connections per node (see {@link #setConnectionsPerNode(int)})</li>
  * <li>Connection buffer flush frequency (see {@link #setConnectionBufferFlushFrequency(long)})</li>
  * <li>Connection buffer size (see {@link #setConnectionBufferSize(int)})</li>
  * <li>Idle connection timeout (see {@link #setIdleConnectionTimeout(long)})</li>
@@ -238,6 +242,9 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
 @IgniteSpiConsistencyChecked(optional = false)
 public class TcpCommunicationSpi extends IgniteSpiAdapter
     implements CommunicationSpi<Message>, TcpCommunicationSpiMBean {
+    /** */
+    private static final IgniteProductVersion MULTIPLE_CONN_SINCE_VER = IgniteProductVersion.fromString("1.8.2");
+
     /** IPC error message. */
     public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
         "(switching to TCP, may be slower).";
@@ -257,11 +264,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Node attribute that is mapped to node's external addresses (value is <tt>comm.tcp.ext-addrs</tt>). */
     public static final String ATTR_EXT_ADDRS = "comm.tcp.ext-addrs";
 
+    /** */
+    public static final String ATTR_PAIRED_CONN = "comm.tcp.pairedConnection";
+
     /** Default port which node sets listener to (value is <tt>47100</tt>). */
     public static final int DFLT_PORT = 47100;
 
     /** Default port which node sets listener for shared memory connections (value is <tt>48100</tt>). */
-    public static final int DFLT_SHMEM_PORT = 48100;
+    public static final int DFLT_SHMEM_PORT = -1;
 
     /** Default idle connection timeout (value is <tt>30000</tt>ms). */
     public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
@@ -283,12 +293,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /**
      * Default count of selectors for TCP server equals to
-     * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
+     * {@code "Math.min(8, Runtime.getRuntime().availableProcessors())"}.
      */
-    public static final int DFLT_SELECTORS_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
+    public static final int DFLT_SELECTORS_CNT = Math.max(4, Runtime.getRuntime().availableProcessors() / 2);
 
-    /** Node ID meta for session. */
-    private static final int NODE_ID_META = GridNioSessionMetaKey.nextUniqueKey();
+    /** Connection index meta for session. */
+    private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
 
     /** Message tracker meta for session. */
     private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
@@ -303,11 +313,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     public static final boolean DFLT_TCP_NODELAY = true;
 
     /** Default received messages threshold for sending ack. */
-    public static final int DFLT_ACK_SND_THRESHOLD = 16;
+    public static final int DFLT_ACK_SND_THRESHOLD = 32;
 
     /** Default socket write timeout. */
     public static final long DFLT_SOCK_WRITE_TIMEOUT = 2000;
 
+    /** Default connections per node. */
+    public static final int DFLT_CONN_PER_NODE = 1;
+
     /** No-op runnable. */
     private static final IgniteRunnable NOOP = new IgniteRunnable() {
         @Override public void run() {
@@ -327,11 +340,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** */
     private ConnectGateway connectGate;
 
+    /** */
+    private ConnectionPolicy connPlc;
+
     /** Server listener. */
     private final GridNioServerListener<Message> srvLsnr =
         new GridNioServerListenerAdapter<Message>() {
             @Override public void onSessionWriteTimeout(GridNioSession ses) {
-                LT.warn(log, "Communication SPI Session write timed out (consider increasing " +
+                LT.warn(log,"Communication SPI session write timed out (consider increasing " +
                     "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
                     ", writeTimeout=" + sockWriteTimeout + ']');
 
@@ -347,46 +363,53 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     if (log.isDebugEnabled())
                         log.debug("Sending local node ID to newly accepted session: " + ses);
 
-                    ses.send(nodeIdMessage());
+                    try {
+                        ses.sendNoFuture(nodeIdMessage());
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send message: " + e, e);
+                    }
                 }
             }
 
             @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
-                UUID id = ses.meta(NODE_ID_META);
+                ConnectionKey connId = ses.meta(CONN_IDX_META);
 
-                if (id != null) {
-                    GridCommunicationClient client = clients.get(id);
+                if (connId != null) {
+                    UUID id = connId.nodeId();
 
-                    if (client instanceof GridTcpNioCommunicationClient &&
-                        ((GridTcpNioCommunicationClient) client).session() == ses) {
-                        client.close();
+                    GridCommunicationClient[] nodeClients = clients.get(id);
 
-                        clients.remove(id, client);
+                    if (nodeClients != null) {
+                        for (GridCommunicationClient client : nodeClients) {
+                            if (client instanceof GridTcpNioCommunicationClient &&
+                                ((GridTcpNioCommunicationClient)client).session() == ses) {
+                                client.close();
+
+                                removeNodeClient(id, client);
+                            }
+                        }
                     }
 
                     if (!stopping) {
-                        boolean reconnect = false;
-
-                        GridNioRecoveryDescriptor recoveryData = ses.recoveryDescriptor();
-
-                        if (recoveryData != null) {
-                            if (recoveryData.nodeAlive(getSpiContext().node(id))) {
-                                if (!recoveryData.messagesFutures().isEmpty()) {
-                                    reconnect = true;
+                        GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
 
+                        if (outDesc != null) {
+                            if (outDesc.nodeAlive(getSpiContext().node(id))) {
+                                if (!outDesc.messagesRequests().isEmpty()) {
                                     if (log.isDebugEnabled())
                                         log.debug("Session was closed but there are unacknowledged messages, " +
-                                            "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
+                                            "will try to reconnect [rmtNode=" + outDesc.node().id() + ']');
+
+                                    DisconnectedSessionInfo disconnectData =
+                                        new DisconnectedSessionInfo(outDesc, connId.connectionIndex());
+
+                                    commWorker.addProcessDisconnectRequest(disconnectData);
                                 }
                             }
                             else
-                                recoveryData.onNodeLeft();
+                                outDesc.onNodeLeft();
                         }
-
-                        DisconnectedSessionInfo disconnectData = new DisconnectedSessionInfo(recoveryData,
-                            reconnect);
-
-                        commWorker.addProcessDisconnectRequest(disconnectData);
                     }
 
                     CommunicationListener<Message> lsnr0 = lsnr;
@@ -403,21 +426,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             private void onFirstMessage(GridNioSession ses, Message msg) {
                 UUID sndId;
 
-                if (msg instanceof NodeIdMessage)
-                    sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+                ConnectionKey connKey;
+
+                if (msg instanceof NodeIdMessage) {
+                    sndId = U.bytesToUuid(((NodeIdMessage) msg).nodeIdBytes, 0);
+                    connKey = new ConnectionKey(sndId, 0, -1);
+                }
                 else {
                     assert msg instanceof HandshakeMessage : msg;
 
+                    HandshakeMessage msg0 = (HandshakeMessage)msg;
+
                     sndId = ((HandshakeMessage)msg).nodeId();
+                    connKey = new ConnectionKey(sndId, msg0.connectionIndex(), msg0.connectCount());
                 }
 
                 if (log.isDebugEnabled())
                     log.debug("Remote node ID received: " + sndId);
 
-                final UUID old = ses.addMeta(NODE_ID_META, sndId);
-
-                assert old == null;
-
                 final ClusterNode rmtNode = getSpiContext().node(sndId);
 
                 if (rmtNode == null) {
@@ -429,57 +455,65 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     return;
                 }
 
+                final ConnectionKey old = ses.addMeta(CONN_IDX_META, connKey);
+
+                assert old == null;
+
                 ClusterNode locNode = getSpiContext().localNode();
 
                 if (ses.remoteAddress() == null)
                     return;
 
-                GridCommunicationClient oldClient = clients.get(sndId);
+                assert msg instanceof HandshakeMessage : msg;
 
-                boolean hasShmemClient = false;
+                HandshakeMessage msg0 = (HandshakeMessage)msg;
 
-                if (oldClient != null) {
-                    if (oldClient instanceof GridTcpNioCommunicationClient) {
-                        if (log.isDebugEnabled())
-                            log.debug("Received incoming connection when already connected " +
-                                    "to this node, rejecting [locNode=" + locNode.id() +
-                                    ", rmtNode=" + sndId + ']');
+                if (usePairedConnections(rmtNode)) {
+                    final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
 
-                        ses.send(new RecoveryLastReceivedMessage(-1));
+                    ConnectClosureNew c = new ConnectClosureNew(ses, recoveryDesc, rmtNode);
 
-                        return;
-                    }
+                    boolean reserve = recoveryDesc.tryReserve(msg0.connectCount(), c);
+
+                    if (reserve)
+                        connectedNew(recoveryDesc, ses, true);
                     else {
-                        assert oldClient instanceof GridShmemCommunicationClient;
+                        if (c.failed) {
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+
+                            for (GridNioSession ses0 : nioSrvr.sessions()) {
+                                ConnectionKey key0 = ses0.meta(CONN_IDX_META);
 
-                        hasShmemClient = true;
+                                if (ses0.accepted() && key0 != null &&
+                                    key0.nodeId().equals(connKey.nodeId()) &&
+                                    key0.connectionIndex() == connKey.connectionIndex() &&
+                                    key0.connectCount() < connKey.connectCount())
+                                    ses0.close();
+                            }
+                        }
                     }
                 }
+                else {
+                    assert connKey.connectionIndex() >= 0 : connKey;
 
-                GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
-
-                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
-
-                assert msg instanceof HandshakeMessage : msg;
-
-                HandshakeMessage msg0 = (HandshakeMessage)msg;
+                    GridCommunicationClient[] curClients = clients.get(sndId);
 
-                final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
+                    GridCommunicationClient oldClient =
+                        curClients != null && connKey.connectionIndex() < curClients.length ?
+                            curClients[connKey.connectionIndex()] :
+                            null;
 
-                if (oldFut == null) {
-                    oldClient = clients.get(sndId);
+                    boolean hasShmemClient = false;
 
                     if (oldClient != null) {
                         if (oldClient instanceof GridTcpNioCommunicationClient) {
                             if (log.isDebugEnabled())
                                 log.debug("Received incoming connection when already connected " +
-                                        "to this node, rejecting [locNode=" + locNode.id() +
-                                        ", rmtNode=" + sndId + ']');
+                                    "to this node, rejecting [locNode=" + locNode.id() +
+                                    ", rmtNode=" + sndId + ']');
 
                             ses.send(new RecoveryLastReceivedMessage(-1));
 
-                            fut.onDone(oldClient);
-
                             return;
                         }
                         else {
@@ -489,51 +523,86 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         }
                     }
 
-                    boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                            new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+                    GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Received incoming connection from remote node " +
+                    GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
+
+                    final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
+
+                    if (oldFut == null) {
+                        curClients = clients.get(sndId);
+
+                        oldClient = curClients != null && connKey.connectionIndex() < curClients.length ?
+                            curClients[connKey.connectionIndex()] : null;
+
+                        if (oldClient != null) {
+                            if (oldClient instanceof GridTcpNioCommunicationClient) {
+                                assert oldClient.connectionIndex() == connKey.connectionIndex() : oldClient;
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Received incoming connection when already connected " +
+                                        "to this node, rejecting [locNode=" + locNode.id() +
+                                        ", rmtNode=" + sndId + ']');
+
+                                ses.send(new RecoveryLastReceivedMessage(-1));
+
+                                fut.onDone(oldClient);
+
+                                return;
+                            }
+                            else {
+                                assert oldClient instanceof GridShmemCommunicationClient;
+
+                                hasShmemClient = true;
+                            }
+                        }
+
+                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                            new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
+
+                        if (log.isDebugEnabled())
+                            log.debug("Received incoming connection from remote node " +
                                 "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ']');
 
-                    if (reserved) {
-                        try {
-                            GridTcpNioCommunicationClient client =
+                        if (reserved) {
+                            try {
+                                GridTcpNioCommunicationClient client =
                                     connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
-                            fut.onDone(client);
-                        }
-                        finally {
-                            clientFuts.remove(rmtNode.id(), fut);
+                                fut.onDone(client);
+                            }
+                            finally {
+                                clientFuts.remove(connKey, fut);
+                            }
                         }
                     }
-                }
-                else {
-                    if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
-                        if (log.isDebugEnabled()) {
-                            log.debug("Received incoming connection from remote node while " +
+                    else {
+                        if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
+                            if (log.isDebugEnabled()) {
+                                log.debug("Received incoming connection from remote node while " +
                                     "connecting to this node, rejecting [locNode=" + locNode.id() +
                                     ", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() +
                                     ", rmtNodeOrder=" + rmtNode.order() + ']');
-                        }
+                            }
 
-                        ses.send(new RecoveryLastReceivedMessage(-1));
-                    }
-                    else {
-                        // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
-                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                                new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+                        }
+                        else {
+                            // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
+                            boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                                new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
 
-                        if (reserved)
-                            connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+                            if (reserved)
+                                connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+                        }
                     }
                 }
             }
 
             @Override public void onMessage(GridNioSession ses, Message msg) {
-                UUID sndId = ses.meta(NODE_ID_META);
+                ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                if (sndId == null) {
+                if (connKey == null) {
                     assert ses.accepted() : ses;
 
                     if (!connectGate.tryEnter()) {
@@ -555,29 +624,37 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 else {
                     rcvdMsgsCnt.increment();
 
-                    GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+                    if (msg instanceof RecoveryLastReceivedMessage) {
+                        GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor();
 
-                    if (recovery != null) {
-                        if (msg instanceof RecoveryLastReceivedMessage) {
+                        if (recovery != null) {
                             RecoveryLastReceivedMessage msg0 = (RecoveryLastReceivedMessage)msg;
 
-                            if (log.isDebugEnabled())
-                                log.debug("Received recovery acknowledgement [rmtNode=" + sndId +
+                            if (log.isDebugEnabled()) {
+                                log.debug("Received recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+                                    ", connIdx=" + connKey.connectionIndex() +
                                     ", rcvCnt=" + msg0.received() + ']');
+                            }
 
                             recovery.ackReceived(msg0.received());
 
                             return;
                         }
-                        else {
+                    }
+                    else {
+                        GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+                        if (recovery != null) {
                             long rcvCnt = recovery.onReceived();
 
                             if (rcvCnt % ackSndThreshold == 0) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Send recovery acknowledgement [rmtNode=" + sndId +
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Send recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+                                        ", connIdx=" + connKey.connectionIndex() +
                                         ", rcvCnt=" + rcvCnt + ']');
+                                }
 
-                                nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(rcvCnt));
+                                ses.systemMessage(new RecoveryLastReceivedMessage(rcvCnt));
 
                                 recovery.lastAcknowledged(rcvCnt);
                             }
@@ -603,7 +680,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     else
                         c = NOOP;
 
-                    notifyListener(sndId, msg, c);
+                    notifyListener(connKey.nodeId(), msg, c);
                 }
             }
 
@@ -611,7 +688,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
              * @param recovery Recovery descriptor.
              * @param ses Session.
              * @param node Node.
-             * @param rcvCnt Number of received messages..
+             * @param rcvCnt Number of received messages.
              * @param sndRes If {@code true} sends response for recovery handshake.
              * @param createClient If {@code true} creates NIO communication client.
              * @return Client.
@@ -623,32 +700,128 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 long rcvCnt,
                 boolean sndRes,
                 boolean createClient) {
+                ConnectionKey connKey = ses.meta(CONN_IDX_META);
+
+                assert connKey != null && connKey.connectionIndex() >= 0 : connKey;
+                assert !usePairedConnections(node);
+
                 recovery.onHandshake(rcvCnt);
 
-                ses.recoveryDescriptor(recovery);
+                ses.inRecoveryDescriptor(recovery);
+                ses.outRecoveryDescriptor(recovery);
 
                 nioSrvr.resend(ses);
 
-                if (sndRes)
-                    nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+                try {
+                    if (sndRes)
+                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
+                }
 
-                recovery.connected();
+                recovery.onConnected();
 
                 GridTcpNioCommunicationClient client = null;
 
                 if (createClient) {
-                    client = new GridTcpNioCommunicationClient(ses, log);
-
-                    GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
+                    client = new GridTcpNioCommunicationClient(connKey.connectionIndex(), ses, log);
 
-                    assert oldClient == null : "Client already created [node=" + node + ", client=" + client +
-                        ", oldClient=" + oldClient + ", recoveryDesc=" + recovery + ']';
+                    addNodeClient(node, connKey.connectionIndex(), client);
                 }
 
                 return client;
             }
 
             /**
+             * @param recovery Recovery descriptor.
+             * @param ses Session.
+             * @param sndRes If {@code true} sends response for recovery handshake.
+             */
+            private void connectedNew(
+                GridNioRecoveryDescriptor recovery,
+                GridNioSession ses,
+                boolean sndRes) {
+                try {
+                    ses.inRecoveryDescriptor(recovery);
+
+                    if (sndRes)
+                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+
+                    recovery.onConnected();
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
+                }
+            }
+
+            /**
+             *
+             */
+            class ConnectClosureNew implements IgniteInClosure<Boolean> {
+                /** */
+                private static final long serialVersionUID = 0L;
+
+                /** */
+                private final GridNioSession ses;
+
+                /** */
+                private final GridNioRecoveryDescriptor recoveryDesc;
+
+                /** */
+                private final ClusterNode rmtNode;
+
+                /** */
+                private boolean failed;
+
+                /**
+                 * @param ses Incoming session.
+                 * @param recoveryDesc Recovery descriptor.
+                 * @param rmtNode Remote node.
+                 */
+                ConnectClosureNew(GridNioSession ses,
+                    GridNioRecoveryDescriptor recoveryDesc,
+                    ClusterNode rmtNode) {
+                    this.ses = ses;
+                    this.recoveryDesc = recoveryDesc;
+                    this.rmtNode = rmtNode;
+                }
+
+                /** {@inheritDoc} */
+                @Override public void apply(Boolean success) {
+                    try {
+                        failed = !success;
+
+                        if (success) {
+                            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> msgFut) {
+                                    try {
+                                        msgFut.get();
+
+                                        connectedNew(recoveryDesc, ses, false);
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Failed to send recovery handshake " +
+                                                    "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+
+                                        recoveryDesc.release();
+                                    }
+                                }
+                            };
+
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                        }
+                        else
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(-1));
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send message: " + e, e);
+                    }
+                }
+            }
+
+            /**
              *
              */
             @SuppressWarnings("PackageVisibleInnerClass")
@@ -674,10 +847,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 /** */
                 private final boolean createClient;
 
+                /** */
+                private final ConnectionKey connKey;
+
                 /**
                  * @param ses Incoming session.
                  * @param recoveryDesc Recovery descriptor.
                  * @param rmtNode Remote node.
+                 * @param connKey Connection key.
                  * @param msg Handshake message.
                  * @param createClient If {@code true} creates NIO communication client..
                  * @param fut Connect future.
@@ -685,12 +862,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ConnectClosure(GridNioSession ses,
                     GridNioRecoveryDescriptor recoveryDesc,
                     ClusterNode rmtNode,
+                    ConnectionKey connKey,
                     HandshakeMessage msg,
                     boolean createClient,
                     GridFutureAdapter<GridCommunicationClient> fut) {
                     this.ses = ses;
                     this.recoveryDesc = recoveryDesc;
                     this.rmtNode = rmtNode;
+                    this.connKey = connKey;
                     this.msg = msg;
                     this.createClient = createClient;
                     this.fut = fut;
@@ -699,39 +878,44 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 /** {@inheritDoc} */
                 @Override public void apply(Boolean success) {
                     if (success) {
-                        IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
-                            @Override public void apply(IgniteInternalFuture<?> msgFut) {
-                                try {
-                                    msgFut.get();
+                        try {
+                            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> msgFut) {
+                                    try {
+                                        msgFut.get();
 
-                                    GridTcpNioCommunicationClient client =
-                                        connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
+                                        GridTcpNioCommunicationClient client =
+                                                connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
 
-                                    fut.onDone(client);
-                                }
-                                catch (IgniteCheckedException e) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Failed to send recovery handshake " +
-                                            "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+                                        fut.onDone(client);
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Failed to send recovery handshake " +
+                                                    "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
 
-                                    recoveryDesc.release();
+                                        recoveryDesc.release();
 
-                                    fut.onDone();
-                                }
-                                finally {
-                                    clientFuts.remove(rmtNode.id(), fut);
+                                        fut.onDone();
+                                    }
+                                    finally {
+                                        clientFuts.remove(connKey, fut);
+                                    }
                                 }
-                            }
-                        };
+                            };
 
-                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to send message: " + e, e);
+                        }
                     }
                     else {
                         try {
                             fut.onDone();
                         }
                         finally {
-                            clientFuts.remove(rmtNode.id(), fut);
+                            clientFuts.remove(connKey, fut);
                         }
                     }
                 }
@@ -794,6 +978,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Shared memory server. */
     private IpcSharedMemoryServerEndpoint shmemSrv;
 
+    /** */
+    private boolean usePairedConnections = true;
+
+    /** */
+    private int connectionsPerNode = DFLT_CONN_PER_NODE;
+
     /** {@code TCP_NODELAY} option value for created sockets. */
     private boolean tcpNoDelay = DFLT_TCP_NODELAY;
 
@@ -816,7 +1006,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
 
     /** Clients. */
-    private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
+    private final ConcurrentMap<UUID, GridCommunicationClient[]> clients = GridConcurrentFactory.newMap();
 
     /** SPI listener. */
     private volatile CommunicationListener<Message> lsnr;
@@ -830,6 +1020,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Count of selectors to use in TCP server. */
     private int selectorsCnt = DFLT_SELECTORS_CNT;
 
+    /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     */
+    private long selectorSpins = IgniteSystemProperties.getLong("IGNITE_SELECTOR_SPINS", 0L);
+
     /** Address resolver. */
     private AddressResolver addrRslvr;
 
@@ -863,11 +1060,17 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     };
 
     /** Client connect futures. */
-    private final ConcurrentMap<UUID, GridFutureAdapter<GridCommunicationClient>> clientFuts =
+    private final ConcurrentMap<ConnectionKey, GridFutureAdapter<GridCommunicationClient>> clientFuts =
         GridConcurrentFactory.newMap();
 
     /** */
-    private final ConcurrentMap<ClientKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+
+    /** */
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> outRecDescs = GridConcurrentFactory.newMap();
+
+    /** */
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> inRecDescs = GridConcurrentFactory.newMap();
 
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
@@ -976,6 +1179,49 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return locPortRange;
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean isUsePairedConnections() {
+        return usePairedConnections;
+    }
+
+    /**
+     * Set this to {@code true} if {@code TcpCommunicationSpi} should
+     * maintain connection for outgoing and incoming messages separately.
+     * In this case total number of connections between local and each remote node
+     * is {@link #getConnectionsPerNode()} * 2.
+     * <p>
+     * Set this to {@code false} if each connection of {@link #getConnectionsPerNode()}
+     * should be used for outgoing and incoming messages. In this case total number
+     * of connections between local and each remote node is {@link #getConnectionsPerNode()}.
+     * In this case load NIO selectors load
+     * balancing of {@link GridNioServer} will be disabled.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @param usePairedConnections {@code true} to use paired connections and {@code false} otherwise.
+     * @see #getConnectionsPerNode()
+     */
+    public void setUsePairedConnections(boolean usePairedConnections) {
+        this.usePairedConnections = usePairedConnections;
+    }
+
+    /**
+     * Sets number of connections to each remote node. if {@link #isUsePairedConnections()}
+     * is {@code true} then number of connections is doubled and half is used for incoming and
+     * half for outgoing messages.
+     *
+     * @param maxConnectionsPerNode Number of connections per node.
+     * @see #isUsePairedConnections()
+     */
+    public void setConnectionsPerNode(int maxConnectionsPerNode) {
+        this.connectionsPerNode = maxConnectionsPerNode;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getConnectionsPerNode() {
+        return connectionsPerNode;
+    }
+
     /**
      * Sets local port to accept shared memory connections.
      * <p>
@@ -1222,6 +1468,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return selectorsCnt;
     }
 
+    /** {@inheritDoc} */
+    @Override public long getSelectorSpins() {
+        return selectorSpins;
+    }
+
+    /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     *
+     * @param selectorSpins Selector thread busy-loop iterations.
+     */
+    public void setSelectorSpins(long selectorSpins) {
+        this.selectorSpins = selectorSpins;
+    }
+
     /**
      * Sets value for {@code TCP_NODELAY} socket option. Each
      * socket will be opened using provided value.
@@ -1396,7 +1658,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (log != null) {
             StringBuilder sb = new StringBuilder("Communication SPI recovery descriptors: ").append(U.nl());
 
-            for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
                 GridNioRecoveryDescriptor desc = entry.getValue();
 
                 sb.append("    [key=").append(entry.getKey())
@@ -1409,14 +1671,48 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     .append(']').append(U.nl());
             }
 
-            sb.append("Communication SPI clients: ").append(U.nl());
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : outRecDescs.entrySet()) {
+                GridNioRecoveryDescriptor desc = entry.getValue();
+
+                sb.append("    [key=").append(entry.getKey())
+                    .append(", msgsSent=").append(desc.sent())
+                    .append(", msgsAckedByRmt=").append(desc.acked())
+                    .append(", reserveCnt=").append(desc.reserveCount())
+                    .append(", connected=").append(desc.connected())
+                    .append(", reserved=").append(desc.reserved())
+                    .append(", descIdHash=").append(System.identityHashCode(desc))
+                    .append(']').append(U.nl());
+            }
+
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : inRecDescs.entrySet()) {
+                GridNioRecoveryDescriptor desc = entry.getValue();
 
-            for (Map.Entry<UUID, GridCommunicationClient> entry : clients.entrySet()) {
-                sb.append("    [node=").append(entry.getKey())
-                    .append(", client=").append(entry.getValue())
+                sb.append("    [key=").append(entry.getKey())
+                    .append(", msgsRcvd=").append(desc.received())
+                    .append(", lastAcked=").append(desc.lastAcknowledged())
+                    .append(", reserveCnt=").append(desc.reserveCount())
+                    .append(", connected=").append(desc.connected())
+                    .append(", reserved=").append(desc.reserved())
+                    .append(", handshakeIdx=").append(desc.handshakeIndex())
+                    .append(", descIdHash=").append(System.identityHashCode(desc))
                     .append(']').append(U.nl());
             }
 
+            sb.append("Communication SPI clients: ").append(U.nl());
+
+            for (Map.Entry<UUID, GridCommunicationClient[]> entry : clients.entrySet()) {
+                UUID nodeId = entry.getKey();
+                GridCommunicationClient[] clients0 = entry.getValue();
+
+                for (GridCommunicationClient client : clients0) {
+                    if (client != null) {
+                        sb.append("    [node=").append(nodeId)
+                            .append(", client=").append(client)
+                            .append(']').append(U.nl());
+                    }
+                }
+            }
+
             U.warn(log, sb.toString());
         }
 
@@ -1426,6 +1722,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             nioSrvr.dumpStats();
     }
 
+    /** */
+    private final ThreadLocal<Integer> threadConnIdx = new ThreadLocal<>();
+
+    /** */
+    private final AtomicInteger connIdx = new AtomicInteger();
+
     /** {@inheritDoc} */
     @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
         initFailureDetectionTimeout();
@@ -1439,6 +1741,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
         assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
         assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
+        assertParameter(connectionsPerNode > 0, "connectionsPerNode > 0");
+        assertParameter(connectionsPerNode <= 1024, "connectionsPerNode <= 1024");
 
         if (!failureDetectionTimeoutEnabled()) {
             assertParameter(reconCnt > 0, "reconnectCnt > 0");
@@ -1458,6 +1762,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 "Specified 'unackedMsgsBufSize' is too low, it should be at least 'ackSndThreshold * 5'.");
         }
 
+        if (connectionsPerNode > 1) {
+            connPlc = new ConnectionPolicy() {
+                @Override public int connectionIndex() {
+                    return (int)(U.safeAbs(Thread.currentThread().getId()) % connectionsPerNode);
+                }
+            };
+        }
+        else {
+            connPlc = new ConnectionPolicy() {
+                @Override public int connectionIndex() {
+                    return 0;
+                }
+            };
+        }
+
         try {
             locHost = U.resolveLocalHost(locAddr);
         }
@@ -1495,6 +1814,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             res.put(createSpiAttributeName(ATTR_PORT), boundTcpPort);
             res.put(createSpiAttributeName(ATTR_SHMEM_PORT), boundTcpShmemPort >= 0 ? boundTcpShmemPort : null);
             res.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
+            res.put(createSpiAttributeName(ATTR_PAIRED_CONN), usePairedConnections);
 
             return res;
         }
@@ -1524,6 +1844,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
             log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
+            log.debug(configInfo("connectionsPerNode", connectionsPerNode));
 
             if (failureDetectionTimeoutEnabled()) {
                 log.debug(configInfo("connTimeout", connTimeout));
@@ -1548,6 +1869,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ", slowClientQueueLimit=" + slowClientQueueLimit + ']');
         }
 
+        if (msgQueueLimit == 0)
+            U.quietAndWarn(log, "Message queue limit is set to 0 which may lead to " +
+                "potential OOMEs when running cache operations in FULL_ASYNC or PRIMARY_SYNC modes " +
+                "due to message queues growth on sender and reciever sides.");
+
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
         connectGate = new ConnectGateway();
@@ -1642,9 +1968,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey key = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+                        return key != null ? formatter.reader(key.nodeId(), msgFactory) : null;
                     }
                 };
 
@@ -1657,9 +1983,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey key = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+                        return key != null ? formatter.writer(key.nodeId()) : null;
                     }
                 };
 
@@ -1716,6 +2042,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .logger(log)
                         .selectorCount(selectorsCnt)
                         .gridName(gridName)
+                        .serverName("tcp-comm")
                         .tcpNoDelay(tcpNoDelay)
                         .directBuffer(directBuf)
                         .byteOrder(ByteOrder.nativeOrder())
@@ -1725,18 +2052,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .directMode(true)
                         .metricsListener(metricsLsnr)
                         .writeTimeout(sockWriteTimeout)
+                        .selectorSpins(selectorSpins)
                         .filters(filters)
                         .writerFactory(writerFactory)
                         .skipRecoveryPredicate(skipRecoveryPred)
                         .messageQueueSizeListener(queueSizeMonitor)
+                        .balancing(usePairedConnections) // Current balancing logic assumes separate in/out connections.
                         .build();
 
                 boundTcpPort = port;
 
                 // Ack Port the TCP server was bound to.
                 if (log.isInfoEnabled())
-                    log.info("Successfully bound to TCP port [port=" + boundTcpPort +
-                        ", locHost=" + locHost + ']');
+                    log.info("Successfully bound communication NIO server to TCP port " +
+                        "[port=" + boundTcpPort + ", locHost=" + locHost + ", selectorsCnt=" + selectorsCnt +
+                        ", selectorSpins=" + srvr.selectorSpins() + ']');
 
                 srvr.idleTimeout(idleConnTimeout);
 
@@ -1837,8 +2167,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         shmemWorkers.clear();
 
         // Force closing on stop (safety).
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         // Clear resources.
         nioSrvr = null;
@@ -1863,8 +2197,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             connectGate.stopped();
 
         // Force closing.
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         getSpiContext().deregisterPorts();
 
@@ -1875,8 +2213,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) {
         connectGate.disconnected(reconnectFut);
 
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
             "Failed to connect client node disconnected.");
@@ -1885,6 +2227,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             clientFut.onDone(err);
 
         recoveryDescs.clear();
+        inRecDescs.clear();
+        outRecDescs.clear();
     }
 
     /** {@inheritDoc} */
@@ -1898,16 +2242,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     void onNodeLeft(UUID nodeId) {
         assert nodeId != null;
 
-        GridCommunicationClient client = clients.get(nodeId);
-
-        if (client != null) {
-            if (log.isDebugEnabled())
-                log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
-                    ", client=" + client + ']');
+        GridCommunicationClient[] clients0 = clients.remove(nodeId);
 
-            client.forceClose();
+        if (clients0 != null) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
+                            ", client=" + client + ']');
 
-            clients.remove(nodeId, client);
+                    client.forceClose();
+                }
+            }
         }
     }
 
@@ -1982,11 +2328,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         else {
             GridCommunicationClient client = null;
 
+            int connIdx = useMultipleConnections(node) ? connPlc.connectionIndex() : 0;
+
             try {
                 boolean retry;
 
                 do {
-                    client = reserveClient(node);
+                    client = reserveClient(node, connIdx);
 
                     UUID nodeId = null;
 
@@ -2000,7 +2348,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     if (!retry)
                         sentMsgsCnt.increment();
                     else {
-                        clients.remove(node.id(), client);
+                        removeNodeClient(node.id(), client);
 
                         ClusterNode node0 = getSpiContext().node(node.id());
 
@@ -2017,26 +2365,94 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 throw new IgniteSpiException("Failed to send message to remote node: " + node, e);
             }
             finally {
-                if (client != null && clients.remove(node.id(), client))
+                if (client != null && removeNodeClient(node.id(), client))
                     client.forceClose();
             }
         }
     }
 
     /**
+     * @param nodeId Node ID.
+     * @param rmvClient Client to remove.
+     * @return {@code True} if client was removed.
+     */
+    private boolean removeNodeClient(UUID nodeId, GridCommunicationClient rmvClient) {
+        for (;;) {
+            GridCommunicationClient[] curClients = clients.get(nodeId);
+
+            if (curClients == null || rmvClient.connectionIndex() >= curClients.length || curClients[rmvClient.connectionIndex()] != rmvClient)
+                return false;
+
+            GridCommunicationClient[] newClients = Arrays.copyOf(curClients, curClients.length);
+
+            newClients[rmvClient.connectionIndex()] = null;
+
+            if (clients.replace(nodeId, curClients, newClients))
+                return true;
+        }
+    }
+
+    /**
+     * @param node Node.
+     * @param connIdx Connection index.
+     * @param addClient Client to add.
+     */
+    private void addNodeClient(ClusterNode node, int connIdx, GridCommunicationClient addClient) {
+        assert connectionsPerNode > 0 : connectionsPerNode;
+        assert connIdx == addClient.connectionIndex() : addClient;
+
+        if (connIdx >= connectionsPerNode) {
+            assert !usePairedConnections(node);
+
+            return;
+        }
+
+        for (;;) {
+            GridCommunicationClient[] curClients = clients.get(node.id());
+
+            assert curClients == null || curClients[connIdx] == null : "Client already created [node=" + node.id() +
+                ", connIdx=" + connIdx +
+                ", client=" + addClient +
+                ", oldClient=" + curClients[connIdx] + ']';
+
+            GridCommunicationClient[] newClients;
+
+            if (curClients == null) {
+                newClients = new GridCommunicationClient[useMultipleConnections(node) ? connectionsPerNode : 1];
+                newClients[connIdx] = addClient;
+
+                if (clients.putIfAbsent(node.id(), newClients) == null)
+                    break;
+            }
+            else {
+                newClients = Arrays.copyOf(curClients, curClients.length);
+                newClients[connIdx] = addClient;
+
+                if (clients.replace(node.id(), curClients, newClients))
+                    break;
+            }
+        }
+    }
+
+    /**
      * Returns existing or just created client to node.
      *
      * @param node Node to which client should be open.
+     * @param connIdx Connection index.
      * @return The existing or just created client.
      * @throws IgniteCheckedException Thrown if any exception occurs.
      */
-    private GridCommunicationClient reserveClient(ClusterNode node) throws IgniteCheckedException {
+    private GridCommunicationClient reserveClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
         assert node != null;
+        assert (connIdx >= 0 && connIdx < connectionsPerNode) || !usePairedConnections(node) : connIdx;
 
         UUID nodeId = node.id();
 
         while (true) {
-            GridCommunicationClient client = clients.get(nodeId);
+            GridCommunicationClient[] curClients = clients.get(nodeId);
+
+            GridCommunicationClient client = curClients != null && connIdx < curClients.length ?
+                curClients[connIdx] : null;
 
             if (client == null) {
                 if (stopping)
@@ -2045,25 +2461,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 // Do not allow concurrent connects.
                 GridFutureAdapter<GridCommunicationClient> fut = new ConnectFuture();
 
-                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(nodeId, fut);
+                ConnectionKey connKey = new ConnectionKey(nodeId, connIdx, -1);
+
+                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
 
                 if (oldFut == null) {
                     try {
-                        GridCommunicationClient client0 = clients.get(nodeId);
+                        GridCommunicationClient[] curClients0 = clients.get(nodeId);
+
+                        GridCommunicationClient client0 = curClients0 != null && connIdx < curClients0.length ?
+                            curClients0[connIdx] : null;
 
                         if (client0 == null) {
-                            client0 = createNioClient(node);
+                            client0 = createNioClient(node, connIdx);
 
                             if (client0 != null) {
-                                GridCommunicationClient old = clients.put(nodeId, client0);
-
-                                assert old == null : "Client already created " +
-                                    "[node=" + node + ", client=" + client0 + ", oldClient=" + old + ']';
+                                addNodeClient(node, connIdx, client0);
 
                                 if (client0 instanceof GridTcpNioCommunicationClient) {
                                     GridTcpNioCommunicationClient tcpClient = ((GridTcpNioCommunicationClient)client0);
 
-                                    if (tcpClient.session().closeTime() > 0 && clients.remove(nodeId, client0)) {
+                                    if (tcpClient.session().closeTime() > 0 && removeNodeClient(nodeId, client0)) {
                                         if (log.isDebugEnabled())
                                             log.debug("Session was closed after client creation, will retry " +
                                                 "[node=" + node + ", client=" + client0 + ']');
@@ -2085,7 +2503,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             throw (Error)e;
                     }
                     finally {
-                        clientFuts.remove(nodeId, fut);
+                        clientFuts.remove(connKey, fut);
                     }
                 }
                 else
@@ -2097,27 +2515,31 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     continue;
 
                 if (getSpiContext().node(nodeId) == null) {
-                    if (clients.remove(nodeId, client))
+                    if (removeNodeClient(nodeId, client))
                         client.forceClose();
 
                     throw new IgniteSpiException("Destination node is not in topology: " + node.id());
                 }
             }
 
+            assert connIdx == client.connectionIndex() : client;
+
             if (client.reserve())
                 return client;
             else
                 // Client has just been closed by idle worker. Help it and try again.
-                clients.remove(nodeId, client);
+                removeNodeClient(nodeId, client);
         }
     }
 
     /**
      * @param node Node to create client for.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridCommunicationClient createNioClient(ClusterNode node) throws IgniteCheckedException {
+    @Nullable private GridCommunicationClient createNioClient(ClusterNode node, int connIdx)
+        throws IgniteCheckedException {
         assert node != null;
 
         Integer shmemPort = node.attribute(createSpiAttributeName(ATTR_SHMEM_PORT));
@@ -2136,6 +2558,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             try {
                 GridCommunicationClient client = createShmemClient(
                     node,
+                    connIdx,
                     shmemPort);
 
                 if (log.isDebugEnabled())
@@ -2158,7 +2581,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         connectGate.enter();
 
         try {
-            GridCommunicationClient client = createTcpClient(node);
+            GridCommunicationClient client = createTcpClient(node, connIdx);
 
             if (log.isDebugEnabled())
                 log.debug("TCP client created: " + client);
@@ -2173,10 +2596,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * @param node Node.
      * @param port Port.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridCommunicationClient createShmemClient(ClusterNode node,
+    @Nullable private GridCommunicationClient createShmemClient(ClusterNode node,
+        int connIdx,
         Integer port) throws IgniteCheckedException {
         int attempt = 1;
 
@@ -2190,7 +2615,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             GridCommunicationClient client;
 
             try {
-                client = new GridShmemCommunicationClient(metricsLsnr,
+                client = new GridShmemCommunicationClient(
+                    connIdx,
+                    metricsLsnr,
                     port,
                     timeoutHelper.nextTimeoutChunk(connTimeout),
                     log,
@@ -2211,7 +2638,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
 
             try {
-                safeHandshake(client, null, node.id(), timeoutHelper.nextTimeoutChunk(connTimeout0), null);
+                safeHandshake(client,
+                    null,
+                    node.id(),
+                    timeoutHelper.nextTimeoutChunk(connTimeout0),
+                    null,
+                    null);
             }
             catch (HandshakeTimeoutException | IgniteSpiOperationTimeoutException e) {
                 client.forceClose();
@@ -2270,10 +2702,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      */
     private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) {
         if (slowClientQueueLimit > 0 && msgQueueSize > slowClientQueueLimit) {
-            UUID id = ses.meta(NODE_ID_META);
+            ConnectionKey id = ses.meta(CONN_IDX_META);
 
             if (id != null) {
-                ClusterNode node = getSpiContext().node(id);
+                ClusterNode node = getSpiContext().node(id.nodeId);
 
                 if (node != null && node.isClient()) {
                     String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
@@ -2283,11 +2715,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
-                    U.quietAndWarn(
-                        log,
-                        msg);
+                    U.quietAndWarn(log, msg);
 
-                    getSpiContext().failNode(id, msg);
+                    getSpiContext().failNode(id.nodeId(), msg);
                 }
             }
         }
@@ -2297,10 +2727,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Establish TCP connection to remote node and returns client.
      *
      * @param node Remote node.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+    protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
         Collection<String> rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS));
         Collection<String> rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES));
         Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT));
@@ -2368,7 +2799,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             "(node left topology): " + node);
                     }
 
-                    GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(node);
+                    ConnectionKey connKey = new ConnectionKey(node.id(), connIdx, -1);
+
+                    GridNioRecoveryDescriptor recoveryDesc = outRecoveryDescriptor(node, connKey);
 
                     if (!recoveryDesc.reserve()) {
                         U.closeQuiet(ch);
@@ -2395,11 +2828,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             sslMeta.sslEngine(sslEngine);
                         }
 
+                        Integer handshakeConnIdx = useMultipleConnections(node) ? connIdx : null;
+
                         rcvCnt = safeHandshake(ch,
                             recoveryDesc,
                             node.id(),
                             timeoutHelper.nextTimeoutChunk(connTimeout0),
-                            sslMeta);
+                            sslMeta,
+                            handshakeConnIdx);
 
                         if (rcvCnt == -1)
                             return null;
@@ -2410,7 +2846,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
 
                     try {
-                        meta.put(NODE_ID_META, node.id());
+                        meta.put(CONN_IDX_META, connKey);
 
                         if (recoveryDesc != null) {
                             recoveryDesc.onHandshake(rcvCnt);
@@ -2420,7 +2856,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         GridNioSession ses = nioSrvr.createSession(ch, meta).get();
 
-                        client = new GridTcpNioCommunicationClient(ses, log);
+                        client = new GridTcpNioCommunicationClient(connIdx, ses, log);
 
                         conn = true;
                     }
@@ -2564,6 +3000,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * @param rmtNodeId Remote node.
      * @param timeout Timeout for handshake.
      * @param sslMeta Session meta.
+     * @param handshakeConnIdx Non null connection index if need send it in handshake.
      * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
      * @return Handshake response.
      */
@@ -2573,7 +3010,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         @Nullable GridNioRecoveryDescriptor recovery,
         UUID rmtNodeId,
         long timeout,
-        GridSslMeta sslMeta
+        GridSslMeta sslMeta,
+        @Nullable Integer handshakeConnIdx
     ) throws IgniteCheckedException {
         HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
 
@@ -2655,14 +3093,28 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             "fully initialized [isStopping=" + getSpiContext().isStopping() + ']');
 
                     if (recovery != null) {
-                        HandshakeMessage msg = new HandshakeMessage(locNode.id(),
-                            recovery.incrementConnectCount(),
-                            recovery.received());
+                        HandshakeMessage msg;
+
+                        int msgSize = 33;
+
+                        if (handshakeConnIdx != null) {
+                            msg = new HandshakeMessage2(locNode.id(),
+                                recovery.incrementConnectCount(),
+                                recovery.received(),
+                                handshakeConnIdx);
+
+                            msgSize += 4;
+                        }
+                        else {
+                            msg = new HandshakeMessage(locNode.id(),
+                                recovery.incrementConnectCount(),
+                                recovery.received());
+                        }
 
                         if (log.isDebugEnabled())
                             log.debug("Write handshake message [rmtNode=" + rmtNodeId + ", msg=" + msg + ']');
 
-                        buf = ByteBuffer.allocate(33);
+                        buf = ByteBuffer.allocate(msgSize);
 
                         buf.order(ByteOrder.nativeOrder());
 
@@ -2689,6 +3141,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         else
                             ch.write(ByteBuffer.wrap(nodeIdMessage().nodeIdBytesWithType));
                     }
+
                     if (recovery != null) {
                         if (log.isDebugEnabled())
                             log.debug("Waiting for handshake [rmtNode=" + rmtNodeId + ']');
@@ -2818,26 +3271,81 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         U.join(commWorker, log);
 
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
+    }
+
+    /**
+     * @param node Node.
+     * @param key Connection key.
+     * @return Recovery descriptor for outgoing connection.
+     */
+    private GridNioRecoveryDescriptor outRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+        if (usePairedConnections(node))
+            return recoveryDescriptor(outRecDescs, true, node, key);
+        else
+            return recoveryDescriptor(recoveryDescs, false, node, key);
     }
 
     /**
      * @param node Node.
-     * @return Recovery receive data for given node.
+     * @param key Connection key.
+     * @return Recovery descriptor for incoming connection.
+     */
+    private GridNioRecoveryDescriptor inRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+        if (usePairedConnections(node))
+            return recoveryDescriptor(inRecDescs, true, node, key);
+        else
+            return recoveryDescriptor(recoveryDescs, false, node, key);
+    }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if given node supports multiple connections per-node for communication.
+     */
+    private boolean useMultipleConnections(ClusterNode node) {
+        return node.version().compareToIgnoreTimestamp(MULTIPLE_CONN_SINCE_VER) >= 0;
+    }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if can use in/out connection pair for communication.
      */
-    private GridNioRecoveryDescriptor recoveryDescriptor(ClusterNode node) {
-        ClientKey id = new ClientKey(node.id(), node.order());
+    private boolean usePairedConnections(ClusterNode node) {
+        if (usePairedConnections) {
+            Boolean attr = node.attribute(createSpiAttributeName(ATTR_PAIRED_CONN));
+
+            return attr != null && attr;
+        }
+
+        return false;
+    }
 
-        GridNioRecoveryDescriptor recovery = recoveryDescs.get(id);
+    /**
+     * @param recoveryDescs Descriptors map.
+     * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
+     * @param node Node.
+     * @param key Connection key.
+     * @return Recovery receive data for given node.
+     */
+    private GridNioRecoveryDescriptor recoveryDescriptor(
+        ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs,
+        boolean pairedConnections,
+        ClusterNode node,
+        ConnectionKey key) {
+        GridNioRecoveryDescriptor recovery = recoveryDescs.get(key);
 
         if (recovery == null) {
             int maxSize = Math.max(msgQueueLimit, ackSndThreshold);
 
-            int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 5);
+            int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 128);
 
-            GridNioRecoveryDescriptor old =
-                recoveryDescs.putIfAbsent(id, recovery = new GridNioRecoveryDescriptor(queueLimit, node, log));
+            GridNioRecoveryDescriptor old = recoveryDescs.putIfAbsent(key,
+                recovery = new GridNioRecoveryDescriptor(pairedConnections, queueLimit, node, log));
 
             if (old != null)
                 recovery = old;
@@ -2879,54 +3387,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return S.toString(TcpCommunicationSpi.class, this);
     }
 
-    /**
-     *
-     */
-    private static class ClientKey {
-        /** */
-        private UUID nodeId;
-
-        /** */
-        private long order;
-
-        /**
-         * @param nodeId Node ID.
-         * @param order Node order.
-         */
-        private ClientKey(UUID nodeId, long order) {
-            this.nodeId = nodeId;
-            this.order = order;
-        }
-
-        /** {@inheritDoc} */
-        @Override public boolean equals(Object obj) {
-            if (this == obj)
-                return true;
-
-            if (obj == null || getClass() != obj.getClass())
-                return false;
-
-            ClientKey other = (ClientKey)obj;
-
-            return order == other.order && nodeId.equals(other.nodeId);
-
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            int res = nodeId.hashCode();
-
-            res = 31 * res + (int)(order ^ (order >>> 32));
-
-            return res;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(ClientKey.class, this);
-        }
-    }
-
     /** Internal exception class for proper timeout handling. */
     private static class HandshakeTimeoutException extends IgniteCheckedException {
         /** */
@@ -3026,9 +3486,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+                        return connKey != null ? formatter.writer(connKey.nodeId()) : null;
                     }
                 };
 
@@ -3042,9 +3502,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+                        return connKey != null ? formatter.reader(connKey.nodeId(), msgFactory) : null;
                     }
                 };
 
@@ -3125,62 +3585,108 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         private void processIdle() {
             cleanupRecovery();
 
-            for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
+            for (Map.Entry<UUID, GridCommunicationClient[]> e : clients.entrySet()) {
                 UUID nodeId = e.getKey();
 
-                GridCommunicationClient client = e.getValue();
+                for (GridCommunicationClient client : e.getValue()) {
+                    if (client == null)
+                        continue;
 
-                ClusterNode node = getSpiContext().node(nodeId);
+                    ClusterNode node = getSpiContext().node(nodeId);
 
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Forcing close of non-existent node connection: " + nodeId);
+                    if (node == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Forcing close of non-existent node connection: " + nodeId);
 
-                    client.forceClose();
+                        client.forceClose();
 
-                    clients.remove(nodeId, client);
+                        removeNodeClient(nodeId, client);
 
-                    continue;
-                }
+                        continue;
+                    }
 
-                GridNioRecoveryDescriptor recovery = null;
+                    GridNioRecoveryDescriptor recovery = null;
 
-                if (client instanceof GridTcpNioCommunicationClient) {
-                    recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
+                    if (!usePairedConnections(node) && client instanceof GridTcpNioCommunicationClient) {
+                        recovery = recoveryDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
 
-                    if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
-                        RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+                        if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+                            RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
 
-                        if (log.isDebugEnabled())
-                            log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
-                                ", rcvCnt=" + msg.received() + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
+                                    ", rcvCnt=" + msg.received() + ']');
 
-                        nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
+                            try {
+                                nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
 
-                        recovery.lastAcknowledged(msg.received());
+                                recovery.lastAcknowledged(msg.received());
+                            }
+                            catch (IgniteCheckedException err) {
+                                U.error(log, "Failed to send message: " + err, err);
+                            }
 
-                        continue;
+                            continue;
+                        }
                     }
-                }
 
-                long idleTime = client.getIdleTime();
+                    long idleTime = client.getIdleTime();
+
+                    if (idleTime >= idleConnTimeout) {
+                        if (recovery == null && usePairedConnections(node))
+                            recovery = outRecDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
+
+                        if (recovery != null &&
+                            recovery.nodeAlive(getSpiContext().node(nodeId)) &&
+                            !recovery.messagesRequests().isEmpty()) {
+                            if (log.isDebugEnabled())
+                                log.debug("Node connection is idle, but there are unacknowledged messages, " +
+                                    "will wait: " + nodeId);
+
+                            continue;
+                        }
 
-                if (idleTime >= idleConnTimeout) {
-                    if (recovery != null &&
-                        recovery.nodeAlive(getSpiContext().node(nodeId)) &&
-                        !recovery.messagesFutures().isEmpty()) {
                         if (log.isDebugEnabled())
-                            log.debug("Node connection is idle, but there are unacknowledged messages, " +
-                                "will wait: " + nodeId);
+                            log.debug("Closing idle node connection: " + nodeId);
 
-                        continue;
+                        if (client.close() || client.closed())
+                            removeNodeClient(nodeId, client);
                     }
+                }
+            }
 
-                    if (log.isDebugEnabled())
-                        log.debug("Closing idle node connection: " + nodeId);
+            for (GridNioSession ses : nioSrvr.sessions()) {
+                GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+                if (recovery != null && usePairedConnections(recovery.node())) {
+                    assert ses.accepted() : ses;
+
+                    sendAckOnTimeout(recovery, ses);
+                }
+            }
+        }
+
+        /**
+         * @param recovery Recovery descriptor.
+         * @param ses Session.
+         */
+        private void sendAckOnTimeout(GridNioRecoveryDescriptor recovery, GridNioSession ses) {
+            if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+                RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+
+                if (log.isDebugEnabled()) {
+                    log.debug("Send recovery acknowledgement on timeout [rmtNode=" + recovery.node().id() +
+                        ", rcvCnt=" + msg.received() +
+                        ", lastAcked=" + recovery.lastAcknowledged() + ']');
+                }
+
+                try {
+                    nioSrvr.sendSystem(ses, msg);
 
-                    if (client.close() || client.closed())
-                        clients.remove(nodeId, client);
+                    recovery.lastAcknowledged(msg.received());
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
                 }
             }
         }
@@ -3189,15 +3695,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
          *
          */
         private void cleanupRecovery() {
-            Set<ClientKey> left = null;
+            cleanupRecovery(recoveryDescs);
+            cleanupRecovery(inRecDescs);
+            cleanupRecovery(outRecDescs);
+        }
+
+        /**
+         * @param recoveryDescs Recovery descriptors to cleanup.
+         */
+        private void cleanupRecovery(ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs) {
+            Set<ConnectionKey> left = null;
 
-            for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
                 if (left != null && left.contains(e.getKey()))
                     continue;
 
-                GridNioRecoveryDescriptor recoverySnd = e.getValue();
+                GridNioRecoveryDescriptor recoveryDesc = e.getValue();
 
-                if (!recoverySnd.nodeAlive(getSpiContext().node(recoverySnd.node().id()))) {
+                if (!recoveryDesc.nodeAlive(getSpiContext().node(e.getKey().nodeId()))) {
                     if (left == null)
                         left = new HashSet<>();
 
@@ -3208,11 +3723,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             if (left != null) {
                 assert !left.isEmpty();
 
-                for (ClientKey id : left) {
-                    GridNioRecoveryDescriptor recoverySnd = recoveryDescs.get(id);
+                for (ConnectionKey id : left) {
+                    GridNioRecoveryDescriptor recoveryDesc = recoveryDescs.get(id);
 
-                    if (recoverySnd != null && recoverySnd.onNodeLeft())
-                        recoveryDescs.remove(id);
+                    if (recoveryDesc != null && recoveryDesc.onNodeLeft())
+                        recoveryDescs.remove(id, recoveryDesc);
                 }
             }
         }
@@ -3221,45 +3736,43 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
          * @param sesInfo Disconnected session information.
          */
         private void processDisconnect(DisconnectedSessionInfo sesInfo) {
-            if (sesInfo.reconnect) {
-                GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
-
-                ClusterNode node = recoveryDesc.node();
+            GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
 
-                if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
-                    return;
+            ClusterNode node = recoveryDesc.node();
 
-                try {
-                    if (log.isDebugEnabled())
-                        log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
+            if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+                return;
 
-                    GridCommunicationClient client = reserveClient(node);
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
 
-                    client.release();
-                }
-                catch (IgniteCheckedException | IgniteException e) {
-                    try {
-                        if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
-                            if (log.isDebugEnabled())
-                                log.debug("Recovery reconnect failed, will retry " +
-                                    "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+                GridCommunicationClient client = reserveClient(node, sesInfo.connIdx);
 
-                            addProcessDisconnectRequest(sesInfo);
-                        }
-                        else {
-                            if (log.isDebugEnabled())
-                                log.debug("Recovery reconnect failed, " +
-                                    "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+                client.release();
+            }
+            catch (IgniteCheckedException | IgniteException e) {
+                try {
+                    if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
+                        if (log.isDebugEnabled())
+                            log.debug("Recovery reconnect failed, will retry " +
+                                "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
 
-                            onException("Recovery reconnect failed, node left [rmtNode=" + recoveryDesc.node().id() + "]",
-                                e);
-                        }
+                        addProcessDisconnectRequest(sesInfo);
                     }
-                    catch (IgniteClientDisconnectedException e0) {
+                    else {
                         if (log.isDebugEnabled())
-                            log.debug("Failed to ping node, client disconnected.");
+                            log.debug("Recovery reconnect failed, " +
+                                "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+
+                        onE

<TRUNCATED>

[24/50] [abbrv] ignite git commit: TcpCommunicationSpi: fixed typo in warning.

Posted by vo...@apache.org.
TcpCommunicationSpi: fixed typo in warning.


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

Branch: refs/heads/ignite-2.0
Commit: ce46c10ea767254a60b80abecf45be8a2d9921b4
Parents: e8fd165
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 21 13:09:26 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 21 13:09:26 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ce46c10e/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 0c90414..c35b5ef 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -1872,7 +1872,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (msgQueueLimit == 0)
             U.quietAndWarn(log, "Message queue limit is set to 0 which may lead to " +
                 "potential OOMEs when running cache operations in FULL_ASYNC or PRIMARY_SYNC modes " +
-                "due to message queues growth on sender and reciever sides.");
+                "due to message queues growth on sender and receiver sides.");
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 


[28/50] [abbrv] ignite git commit: Merge ignite-1.8.2.

Posted by vo...@apache.org.
Merge ignite-1.8.2.


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

Branch: refs/heads/ignite-2.0
Commit: 1b2afbb8d4032d23ee9a78987aec90a42e886700
Parents: 41dddb8 8372e69
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 22 16:51:00 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 22 16:51:00 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |   6 +
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 ++--
 .../org/apache/ignite/internal/GridTopic.java   |   8 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../GridCachePartitionExchangeManager.java      |  23 +-
 .../CacheDataStructuresManager.java             |   6 +-
 .../cache/query/GridCacheQueryManager.java      |   4 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../internal/processors/job/GridJobWorker.java  |  76 ++-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../service/GridServiceProcessor.java           |  11 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../communication/tcp/TcpCommunicationSpi.java  |   4 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 +++++
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 -------------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 +++++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 ++++++
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../processors/query/h2/IgniteH2Indexing.java   |  75 +--
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 43 files changed, 2008 insertions(+), 779 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
index ea3ae8b,c6b41d2..2e0fd10
--- 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
@@@ -198,13 -198,19 +198,19 @@@ public class OdbcRequestHandler 
              qry.setDistributedJoins(distributedJoins);
              qry.setEnforceJoinOrder(enforceJoinOrder);
  
-             IgniteCache<Object, Object> cache = ctx.grid().cache(req.cacheName());
+             IgniteCache<Object, Object> cache0 = ctx.grid().cache(req.cacheName());
+ 
+             if (cache0 == null)
+                 return new OdbcResponse(OdbcResponse.STATUS_FAILED,
 -                        "Cache doesn't exist (did you configure it?): " + req.cacheName());
++                    "Cache doesn't exist (did you configure it?): " + req.cacheName());
+ 
+             IgniteCache<Object, Object> cache = cache0.withKeepBinary();
  
              if (cache == null)
-                 return new OdbcResponse(OdbcResponse.STATUS_FAILED, "Cache doesn't exist (did you configure it?): " +
-                     req.cacheName());
+                 return new OdbcResponse(OdbcResponse.STATUS_FAILED,
+                     "Can not get cache with keep binary: " + req.cacheName());
  
-             QueryCursor qryCur = cache.withKeepBinary().query(qry);
+             QueryCursor qryCur = cache.query(qry);
  
              qryCursors.put(qryId, new IgniteBiTuple<QueryCursor, Iterator>(qryCur, null));
  

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/dotnet/PlatformDotNetCacheStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index 22a7fa2,c266986..37727f5
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@@ -341,10 -340,10 +341,10 @@@ public class PlatformServices extends P
  
                  Object proxy = PlatformService.class.isAssignableFrom(d.serviceClass())
                      ? services.serviceProxy(name, PlatformService.class, sticky)
-                     : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky,
+                     : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky, 0,
                          platformCtx.kernalContext());
  
 -                return new ServiceProxyHolder(proxy, d.serviceClass());
 +                return new ServiceProxyHolder(proxy, d.serviceClass(), platformContext());
              }
          }
          return super.processInStreamOutObject(type, reader);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1b2afbb8/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformUtils.java
----------------------------------------------------------------------


[30/50] [abbrv] ignite git commit: IGNITE-4439 - Attribute based node filter

Posted by vo...@apache.org.
IGNITE-4439 - Attribute based node filter


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

Branch: refs/heads/ignite-2.0
Commit: 2591c160efc4251cb33854955970c93ec20d6b24
Parents: 7094c0f
Author: Valentin Kulichenko <va...@gmail.com>
Authored: Thu Dec 22 13:05:35 2016 -0800
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Thu Dec 22 13:05:35 2016 -0800

----------------------------------------------------------------------
 .../apache/ignite/util/AttributeNodeFilter.java | 105 +++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   3 +
 .../util/AttributeNodeFilterSelfTest.java       | 184 +++++++++++++++++++
 3 files changed, 292 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
new file mode 100644
index 0000000..e2b972b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/util/AttributeNodeFilter.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.util;
+
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterGroup;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.services.ServiceConfiguration;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Implementation of {@code IgnitePredicate<ClusterNode>} based on
+ * {@link IgniteConfiguration#getUserAttributes() user attributes}.
+ * This filter can be used in methods like {@link ClusterGroup#forPredicate(IgnitePredicate)},
+ * {@link CacheConfiguration#setNodeFilter(IgnitePredicate)},
+ * {@link ServiceConfiguration#setNodeFilter(IgnitePredicate)}, etc.
+ * <p>
+ * The filter will evaluate to true if a node has <b>all</b> provided attributes set to
+ * corresponding values. Here is an example of how you can configure node filter for a
+ * cache or a service so that it's deployed only on nodes that have {@code group}
+ * attribute set to value {@code data}:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;nodeFilter&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.util.ClusterAttributeNodeFilter&quot;&gt;
+ *         &lt;constructor-arg value="group"/&gt;
+ *         &lt;constructor-arg value="data"/&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * You can also specify multiple attributes for the filter:
+ * <pre name="code" class="xml">
+ * &lt;property name=&quot;nodeFilter&quot;&gt;
+ *     &lt;bean class=&quot;org.apache.ignite.util.ClusterAttributeNodeFilter&quot;&gt;
+ *         &lt;constructor-arg&gt;
+ *             &lt;map&gt;
+ *                 &lt;entry key=&quot;cpu-group&quot; value=&quot;high&quot;/&gt;
+ *                 &lt;entry key=&quot;memory-group&quot; value=&quot;high&quot;/&gt;
+ *             &lt;/map&gt;
+ *         &lt;/constructor-arg&gt;
+ *     &lt;/bean&gt;
+ * &lt;/property&gt;
+ * </pre>
+ * With this configuration a cache or a service will deploy only on nodes that have both
+ * {@code cpu-group} and {@code memory-group} attributes set to value {@code high}.
+ */
+public class AttributeNodeFilter implements IgnitePredicate<ClusterNode> {
+    /** Attributes. */
+    private final Map<String, Object> attrs;
+
+    /**
+     * Creates new node filter with a single attribute value.
+     *
+     * @param attrName Attribute name.
+     * @param attrVal Attribute value.
+     */
+    public AttributeNodeFilter(String attrName, @Nullable Object attrVal) {
+        A.notNull(attrName, "attrName");
+
+        attrs = Collections.singletonMap(attrName, attrVal);
+    }
+
+    /**
+     * Creates new node filter with a set of attributes.
+     *
+     * @param attrs Attributes.
+     */
+    public AttributeNodeFilter(Map<String, Object> attrs) {
+        A.notNull(attrs, "attrs");
+
+        this.attrs = attrs;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode node) {
+        Map<String, Object> nodeAttrs = node.attributes();
+
+        for (Map.Entry<String, Object> attr : attrs.entrySet()) {
+            if (!F.eq(nodeAttrs.get(attr.getKey()), attr.getValue()))
+                return false;
+        }
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index c6281df..8ccec34 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -20,6 +20,7 @@ package org.apache.ignite.testsuites;
 import java.util.Set;
 import junit.framework.TestSuite;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
+import org.apache.ignite.util.AttributeNodeFilterSelfTest;
 import org.apache.ignite.internal.ClusterGroupHostsSelfTest;
 import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
@@ -149,6 +150,8 @@ public class IgniteBasicTestSuite extends TestSuite {
 
         suite.addTestSuite(SecurityPermissionSetBuilderTest.class);
 
+        suite.addTestSuite(AttributeNodeFilterSelfTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/2591c160/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
new file mode 100644
index 0000000..ac3800f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/util/AttributeNodeFilterSelfTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.util;
+
+import java.lang.reflect.InvocationHandler;
+import java.lang.reflect.Method;
+import java.lang.reflect.Proxy;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgnitePredicate;
+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;
+
+/**
+ * Tests for {@link AttributeNodeFilter}.
+ */
+public class AttributeNodeFilterSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private Map<String, ?> attrs;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
+
+        if (attrs != null)
+            cfg.setUserAttributes(attrs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        attrs = null;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSingleAttribute() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter("attr", "value");
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr", "value"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", null))));
+        assertFalse(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertFalse(filter.apply(nodeProxy(F.asMap("wrong", "value"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("null", "value"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSingleAttributeNullValue() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter("attr", null);
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr", null))));
+        assertTrue(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertTrue(filter.apply(nodeProxy(F.asMap("wrong", "value"))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("wrong", null))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr", "value"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleAttributes() throws Exception {
+        IgnitePredicate<ClusterNode> filter =
+            new AttributeNodeFilter(F.<String, Object>asMap("attr1", "value1", "attr2", "value2"));
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "wrong", "attr2", "wrong"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultipleAttributesNullValues() throws Exception {
+        IgnitePredicate<ClusterNode> filter = new AttributeNodeFilter(F.asMap("attr1", null, "attr2", null));
+
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null, "attr2", null))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr1", null))));
+        assertTrue(filter.apply(nodeProxy(F.asMap("attr2", null))));
+        assertTrue(filter.apply(nodeProxy(Collections.<String, Object>emptyMap())));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr2", "value2"))));
+        assertFalse(filter.apply(nodeProxy(F.asMap("attr1", "value1", "attr2", "value2"))));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClusterGroup() throws Exception {
+        Ignite group1 = startGridsMultiThreaded(3);
+
+        attrs = F.asMap("group", "data");
+
+        Ignite group2 = startGridsMultiThreaded(3, 2);
+
+        assertEquals(2, group1.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size());
+        assertEquals(2, group2.cluster().forPredicate(new AttributeNodeFilter("group", "data")).nodes().size());
+
+        assertEquals(3, group1.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size());
+        assertEquals(3, group2.cluster().forPredicate(new AttributeNodeFilter("group", null)).nodes().size());
+
+        assertEquals(0, group1.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size());
+        assertEquals(0, group2.cluster().forPredicate(new AttributeNodeFilter("group", "wrong")).nodes().size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheFilter() throws Exception {
+        Ignite group1 = startGridsMultiThreaded(3);
+
+        attrs = F.asMap("group", "data");
+
+        Ignite group2 = startGridsMultiThreaded(3, 2);
+
+        group1.createCache(new CacheConfiguration<>("test-cache").
+            setNodeFilter(new AttributeNodeFilter("group", "data")));
+
+        assertEquals(2, group1.cluster().forDataNodes("test-cache").nodes().size());
+        assertEquals(2, group2.cluster().forDataNodes("test-cache").nodes().size());
+
+        assertEquals(0, group1.cluster().forDataNodes("wrong").nodes().size());
+        assertEquals(0, group2.cluster().forDataNodes("wrong").nodes().size());
+    }
+
+    /**
+     * @param attrs Attributes.
+     * @return Node proxy.
+     */
+    private static ClusterNode nodeProxy(final Map<String, ?> attrs) {
+        return (ClusterNode)Proxy.newProxyInstance(
+            ClusterNode.class.getClassLoader(),
+            new Class[] { ClusterNode.class },
+            new InvocationHandler() {
+                @SuppressWarnings("SuspiciousMethodCalls")
+                @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
+                    if ("attributes".equals(mtd.getName()))
+                        return attrs;
+
+                    throw new UnsupportedOperationException();
+                }
+            });
+    }
+}


[21/50] [abbrv] ignite git commit: Null argument check for cache.putAll.

Posted by vo...@apache.org.
Null argument check for cache.putAll.

(cherry picked from commit c0f1a21)


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

Branch: refs/heads/ignite-2.0
Commit: e82eefe7a7d7f541e6d2fc4b166843b87ec9ef87
Parents: 09c634e
Author: sboikov <sb...@gridgain.com>
Authored: Tue Dec 20 12:35:51 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Dec 20 13:21:28 2016 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e82eefe7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 8187e8f..f235f6a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -2661,6 +2661,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public void putAll(@Nullable final Map<? extends K, ? extends V> m) throws IgniteCheckedException {
+        A.notNull(m, "map");
+
         if (F.isEmpty(m))
             return;
 


[03/50] [abbrv] ignite git commit: IGNITE-4426: Hadoop: tasks can share the same classloader. This closes #1344.

Posted by vo...@apache.org.
IGNITE-4426: Hadoop: tasks can share the same classloader. This closes #1344.


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

Branch: refs/heads/ignite-2.0
Commit: 30b869ddd32db637ee9ea8f13a115dd4bacc52fe
Parents: b44baf1
Author: devozerov <vo...@gridgain.com>
Authored: Wed Dec 14 14:35:29 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:34 2016 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopClassLoader.java    | 10 +++++++++
 .../processors/hadoop/HadoopJobProperty.java    |  8 +++++++
 .../processors/hadoop/impl/v2/HadoopV2Job.java  | 22 ++++++++++++++++++--
 3 files changed, 38 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/30b869dd/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index cd94c89..f6c2fa9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -101,6 +101,16 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
     }
 
     /**
+     * Classloader name for job.
+     *
+     * @param jobId Job ID.
+     * @return Name.
+     */
+    public static String nameForJob(HadoopJobId jobId) {
+        return "hadoop-job-" + jobId;
+    }
+
+    /**
      * Gets name for the task class loader. Task class loader
      * @param info The task info.
      * @param prefix Get only prefix (without task type and number)

http://git-wip-us.apache.org/repos/asf/ignite/blob/30b869dd/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 1f0ef1b..9e1dede 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -57,6 +57,13 @@ public enum HadoopJobProperty {
     JOB_STATUS_POLL_DELAY("ignite.job.status.poll.delay"),
 
     /**
+     * Whether job classloader can be shared between all tasks.
+     * <p>
+     * Defaults to {@code true}.
+     */
+    JOB_SHARED_CLASSLOADER("ignite.job.shared.classloader"),
+
+    /**
      * Size in bytes of single memory page which will be allocated for data structures in shuffle.
      * <p>
      * By default is {@code 32 * 1024}.
@@ -105,6 +112,7 @@ public enum HadoopJobProperty {
      */
     SHUFFLE_JOB_THROTTLE("ignite.shuffle.job.throttle");
 
+
     /** Property name. */
     private final String propName;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/30b869dd/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
index 36da410..a24e581 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Job.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskType;
@@ -73,6 +74,7 @@ import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 
+import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.JOB_SHARED_CLASSLOADER;
 import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.jobLocalDir;
 import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.taskLocalDir;
 import static org.apache.ignite.internal.processors.hadoop.impl.HadoopUtils.transformException;
@@ -121,6 +123,9 @@ public class HadoopV2Job implements HadoopJob {
     /** File system cache map. */
     private final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap = createHadoopLazyConcurrentMap();
 
+    /** Shared class loader. */
+    private volatile HadoopClassLoader sharedClsLdr;
+
     /** Local node ID */
     private volatile UUID locNodeId;
 
@@ -261,8 +266,8 @@ public class HadoopV2Job implements HadoopJob {
                 // If there is no pooled class, then load new one.
                 // Note that the classloader identified by the task it was initially created for,
                 // but later it may be reused for other tasks.
-                HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(),
-                    HadoopClassLoader.nameForTask(info, false), libNames, helper);
+                HadoopClassLoader ldr = sharedClsLdr != null ?
+                    sharedClsLdr : createClassLoader(HadoopClassLoader.nameForTask(info, false));
 
                 cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 
@@ -312,6 +317,9 @@ public class HadoopV2Job implements HadoopJob {
 
         try {
             rsrcMgr.prepareJobEnvironment(!external, jobLocalDir(igniteWorkDirectory(), locNodeId, jobId));
+
+            if (HadoopJobProperty.get(jobInfo, JOB_SHARED_CLASSLOADER, true))
+                sharedClsLdr = createClassLoader(HadoopClassLoader.nameForJob(jobId));
         }
         finally {
             HadoopCommonUtils.restoreContextClassLoader(oldLdr);
@@ -454,4 +462,14 @@ public class HadoopV2Job implements HadoopJob {
     public FileSystem fileSystem(@Nullable URI uri, Configuration cfg) throws IOException {
         return fileSystemForMrUserWithCaching(uri, cfg, fsMap);
     }
+
+    /**
+     * Create class loader with the given name.
+     *
+     * @param name Name.
+     * @return Class loader.
+     */
+    private HadoopClassLoader createClassLoader(String name) {
+        return new HadoopClassLoader(rsrcMgr.classPath(), name, libNames, helper);
+    }
 }
\ No newline at end of file


[29/50] [abbrv] ignite git commit: IGNITE-1443 CPP: Implement cache continuous queries - minor fix

Posted by vo...@apache.org.
IGNITE-1443 CPP: Implement cache continuous queries - minor fix

This closes #1378


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

Branch: refs/heads/ignite-2.0
Commit: 848bb66311365e5f8e3bde8310ad064d3449bf13
Parents: 1b2afbb
Author: Igor Sapego <is...@gridgain.com>
Authored: Thu Dec 22 19:14:10 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 22 19:14:10 2016 +0300

----------------------------------------------------------------------
 .../core/include/ignite/cache/query/continuous/continuous_query.h  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/848bb663/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
index 563b11a..4bd9c46 100644
--- a/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
+++ b/modules/platforms/cpp/core/include/ignite/cache/query/continuous/continuous_query.h
@@ -204,7 +204,7 @@ namespace ignite
                      */
                     void SetListener(Reference<event::CacheEntryEventListener<K, V>> lsnr)
                     {
-                        impl.Get()->SetListener(val);
+                        impl.Get()->SetListener(lsnr);
                     }
 
                     /**


[25/50] [abbrv] ignite git commit: IGNITE-3886 .NET: Build script added

Posted by vo...@apache.org.
IGNITE-3886 .NET: Build script added

This closes #1298


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

Branch: refs/heads/ignite-2.0
Commit: c864fe443284508c7eb1d2341d9fdc2a5a844c45
Parents: 0a33644
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Dec 21 15:50:36 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Dec 21 15:50:36 2016 +0300

----------------------------------------------------------------------
 modules/platforms/dotnet/Apache.Ignite.sln |   2 +
 modules/platforms/dotnet/DEVNOTES.txt      |  12 +-
 modules/platforms/dotnet/build.bat         |  14 ++
 modules/platforms/dotnet/build.ps1         | 211 ++++++++++++++++++++++++
 4 files changed, 236 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/Apache.Ignite.sln
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.sln b/modules/platforms/dotnet/Apache.Ignite.sln
index ef953b1..bcaa753 100644
--- a/modules/platforms/dotnet/Apache.Ignite.sln
+++ b/modules/platforms/dotnet/Apache.Ignite.sln
@@ -30,6 +30,8 @@ Project("{2150E333-8FDC-42A3-9474-1A3956D46DE8}") = "Solution Items", "Solution
 		Apache.Ignite.FxCop = Apache.Ignite.FxCop
 		Apache.Ignite.sln.DotSettings = Apache.Ignite.sln.DotSettings
 		Apache.Ignite.sln.TeamCity.DotSettings = Apache.Ignite.sln.TeamCity.DotSettings
+		build.bat = build.bat
+		build.ps1 = build.ps1
 		DEVNOTES.txt = DEVNOTES.txt
 		README.txt = README.txt
 		README.md = README.md

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/DEVNOTES.txt b/modules/platforms/dotnet/DEVNOTES.txt
index 630960e..df0ebb5 100644
--- a/modules/platforms/dotnet/DEVNOTES.txt
+++ b/modules/platforms/dotnet/DEVNOTES.txt
@@ -5,12 +5,18 @@ Requirements:
 * Windows (XP and up), Windows Server (2008 and up)
 * Oracle JDK 7 and above
 * .NET Framework 4.0
+* PowerShell 3.0+
 * Visual Studio 2010 (later versions require upgrading "common" C++ project, see below)
-* JAVA_HOME environment variable set to the corresponding JDK (x64 or x86).
+* JAVA_HOME environment variable set to the corresponding JDK (x64 or x86)
+* Apache Maven bin directory in PATH, or MAVEN_HOME environment variable
 
 Building binaries:
-  msbuild Apache.Ignite.sln /p:Configuration=Release
-Resulting binaries will be in Apache.Ignite\bin folder
+  build.bat -skipCodeAnalysis
+Resulting binaries will be in bin folder, and NuGet packages in nupkg folder.
+
+Running built binaries: resulting "bin" folder in self contained, you can copy it anywhere and run
+  bin\Apache.Ignite.exe  
+
 NOTE: 
 * x86 solution platform requires x86 Oracle JDK.
 * x64 solution platform requires x64 Oracle JDK.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/build.bat
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.bat b/modules/platforms/dotnet/build.bat
new file mode 100644
index 0000000..ef0a96f
--- /dev/null
+++ b/modules/platforms/dotnet/build.bat
@@ -0,0 +1,14 @@
+:: Licensed 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.
+
+:: Apache Ignite.NET build script runner, see build.ps1 for more details.
+powershell -executionpolicy remotesigned -file build.ps1 %*
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c864fe44/modules/platforms/dotnet/build.ps1
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/build.ps1 b/modules/platforms/dotnet/build.ps1
new file mode 100644
index 0000000..be7e638
--- /dev/null
+++ b/modules/platforms/dotnet/build.ps1
@@ -0,0 +1,211 @@
+<#
+
+  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.
+
+ #>
+
+ <#
+
+.SYNOPSIS
+Apache Ignite.NET build script.
+
+.DESCRIPTION
+Builds all parts of Apache Ignite.NET: Java, .NET, NuGet. Copies results to 'bin' and 'nupkg' folders.
+
+Requirements:
+* PowerShell 3
+* JDK 7+
+* MAVEN_HOME environment variable or mvn.bat in PATH
+
+.PARAMETER skipJava
+Skip Java build.
+
+.PARAMETER skipNuGet
+Skip NuGet packaging.
+
+.PARAMETER skipCodeAnalysis
+Skip code analysis.
+
+.PARAMETER clean
+Perform a clean rebuild.
+
+.PARAMETER platform
+Build platform ("Any CPU", "x86", "x64").
+
+.PARAMETER configuration
+Build configuration ("Release", "Debug").
+
+.PARAMETER mavenOpts
+Custom Maven options, default is "-U -P-lgpl,-scala,-examples,-test,-benchmarks -Dmaven.javadoc.skip=true".
+
+.EXAMPLE
+.\build.ps1 -clean  
+# Full rebuild of Java, .NET and NuGet packages.
+
+.EXAMPLE
+.\build.ps1 -skipJava -skipCodeAnalysis -skipNuGet -configuration Debug -platform x64
+# Quick build of .NET code only.
+
+#>
+
+param (
+    [switch]$skipJava,
+    [switch]$skipNuGet,
+    [switch]$skipCodeAnalysis,  
+    [switch]$clean,
+    [ValidateSet("Any CPU", "x64", "x86")]
+    [string]$platform="Any CPU",
+    [ValidateSet("Release", "Debug")]
+    [string]$configuration="Release",
+    [string]$mavenOpts="-U -P-lgpl,-scala,-examples,-test,-benchmarks -Dmaven.javadoc.skip=true"
+ )
+
+# 1) Build Java (Maven)
+if (!$skipJava) {
+    # Detect Ignite root directory
+    cd $PSScriptRoot\..
+
+    while (!((Test-Path bin) -and (Test-Path examples) -and ((Test-Path modules) -or (Test-Path platforms))))
+    { cd .. }
+
+    echo "Ignite home detected at '$pwd'."
+
+    # Detect Maven
+    $mv = "mvn"
+    if ((Get-Command $mv -ErrorAction SilentlyContinue) -eq $null) { 
+        $mvHome = ($env:MAVEN_HOME, $env:M2_HOME, $env:M3_HOME, $env:MVN_HOME -ne $null)[0]
+
+        if ($mvHome -eq $null) {
+            echo "Maven not found. Make sure to update PATH variable or set MAVEN_HOME, M2_HOME, M3_HOME, or MVN_HOME."
+            exit -1
+        }
+
+        $mv = "`"" + (join-path $mvHome "bin\mvn.bat") + "`""
+        echo "Maven detected at $mv."
+    }
+
+    # Run Maven
+    echo "Starting Java (Maven) build..."
+    
+    $mvnTargets = if ($clean)  { "clean package" } else { "package" }
+    cmd /c "$mv $mvnTargets -DskipTests $mavenOpts"
+
+    # Check result
+    if ($LastExitCode -ne 0) {
+        echo "Java (Maven) build failed."; exit -1
+    }
+
+    # Copy (relevant) jars
+    $libsDir = "$PSScriptRoot\bin\Libs"
+    mkdir -Force $libsDir; del -Force $libsDir\*.*
+    
+    copy -Force target\release-package\libs\*.jar $libsDir
+    copy -Force target\release-package\libs\ignite-spring\*.jar $libsDir
+    copy -Force target\release-package\libs\ignite-indexing\*.jar $libsDir
+    copy -Force target\release-package\libs\licenses\*.jar $libsDir
+
+    # Restore directory
+    cd $PSScriptRoot
+}
+else {
+    echo "Java (Maven) build skipped."
+}
+
+
+# 2) Build .NET
+# Detect MSBuild 4.0+
+for ($i=4; $i -le 20; $i++) {
+    $regKey = "HKLM:\software\Microsoft\MSBuild\ToolsVersions\$i.0"
+    if (Test-Path $regKey) { break }
+}
+
+if (!(Test-Path $regKey)) {
+    echo "Failed to detect MSBuild path, exiting."
+    exit -1
+}
+
+$msbuildExe = (join-path -path (Get-ItemProperty $regKey)."MSBuildToolsPath" -childpath "msbuild.exe")
+echo "MSBuild detected at '$msbuildExe'."
+
+# Detect NuGet
+$ng = "nuget"
+if ((Get-Command $ng -ErrorAction SilentlyContinue) -eq $null) { 
+    echo "Downloading NuGet..."
+    (New-Object System.Net.WebClient).DownloadFile("https://dist.nuget.org/win-x86-commandline/v3.3.0/nuget.exe", "nuget.exe");    
+    $ng = ".\nuget.exe"
+}
+
+# Restore NuGet packages
+echo "Restoring NuGet..."
+& $ng restore
+
+# Build
+echo "Starting MsBuild..."
+$targets = if ($clean) {"Clean;Rebuild"} else {"Build"}
+$codeAnalysis = if ($skipCodeAnalysis) {"/p:RunCodeAnalysis=false"} else {""}
+& $msbuildExe Apache.Ignite.sln /target:$targets /p:Configuration=$configuration /p:Platform=`"$platform`" $codeAnalysis /p:UseSharedCompilation=false
+
+# Check result
+if ($LastExitCode -ne 0) {
+    echo ".NET build failed."
+    exit -1
+}
+
+# Copy binaries
+mkdir -Force bin; del -Force bin\*.*
+
+ls *.csproj -Recurse | where Name -NotLike "*Examples*" `
+                     | where Name -NotLike "*Tests*" `
+                     | where Name -NotLike "*Benchmarks*" | % {
+    $binDir = if (($configuration -eq "Any CPU") -or ($_.Name -ne "Apache.Ignite.Core.csproj")) `
+                {"bin\$configuration"} else {"bin\$platform\$configuration"}
+    $dir = join-path (split-path -parent $_) $binDir    
+    xcopy /s /y $dir\*.* bin
+}
+
+
+# 3) Pack NuGet
+if (!$skipNuGet) {
+    # Check parameters
+    if (($platform -ne "Any CPU") -or ($configuration -ne "Release")) {
+        echo "NuGet can only package 'Release' 'Any CPU' builds; you have specified '$configuration' '$platform'."
+        exit -1
+    }
+
+    $nupkgDir = "nupkg"
+    mkdir -Force $nupkgDir; del -Force $nupkgDir\*.*
+
+    # Detect version
+    $ver = (gi Apache.Ignite.Core\bin\Release\Apache.Ignite.Core.dll).VersionInfo.ProductVersion
+
+    # Find all nuspec files and run 'nuget pack' either directly, or on corresponding csproj files (if present)
+    ls *.nuspec -Recurse  `
+        | % { 
+            If (Test-Path ([io.path]::ChangeExtension($_.FullName, ".csproj"))){
+                [io.path]::ChangeExtension($_.FullName, ".csproj")
+            } Else { $_.FullName }
+        } | % { 
+            & $ng pack $_ -Prop Configuration=Release -Prop Platform=AnyCPU -Version $ver -OutputDirectory $nupkgDir
+
+            # check result
+            if ($LastExitCode -ne 0)
+            {
+                echo "NuGet pack failed."; exit -1
+            }
+        }
+
+    echo "NuGet packages created in '$pwd\$nupkgDir'."
+}
\ No newline at end of file


[36/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
deleted file mode 100644
index 61de1a2..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-optional.js
+++ /dev/null
@@ -1,25 +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.
- */
-
-// Optional content generation entry point.
-const $generatorOptional = {};
-
-$generatorOptional.optionalContent = function(zip, cluster) { // eslint-disable-line no-unused-vars
-    // No-op.
-};
-
-export default $generatorOptional;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js b/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
deleted file mode 100644
index f70c66f..0000000
--- a/modules/web-console/frontend/app/modules/configuration/generator/generator-spring.js
+++ /dev/null
@@ -1,2111 +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.
- */
-
-// XML generation entry point.
-const $generatorSpring = {};
-
-// Do XML escape.
-$generatorSpring.escape = function(s) {
-    if (typeof (s) !== 'string')
-        return s;
-
-    return s.replace(/&/g, '&amp;').replace(/</g, '&lt;').replace(/>/g, '&gt;').replace(/"/g, '&quot;');
-};
-
-// Add constructor argument
-$generatorSpring.constructorArg = function(res, ix, obj, propName, dflt, opt) {
-    const v = (obj ? obj[propName] : null) || dflt;
-
-    if ($generatorCommon.isDefinedAndNotEmpty(v))
-        res.line('<constructor-arg ' + (ix >= 0 ? 'index="' + ix + '" ' : '') + 'value="' + v + '"/>');
-    else if (!opt) {
-        res.startBlock('<constructor-arg ' + (ix >= 0 ? 'index="' + ix + '"' : '') + '>');
-        res.line('<null/>');
-        res.endBlock('</constructor-arg>');
-    }
-};
-
-// Add XML element.
-$generatorSpring.element = function(res, tag, attr1, val1, attr2, val2) {
-    let elem = '<' + tag;
-
-    if (attr1)
-        elem += ' ' + attr1 + '="' + val1 + '"';
-
-    if (attr2)
-        elem += ' ' + attr2 + '="' + val2 + '"';
-
-    elem += '/>';
-
-    res.emptyLineIfNeeded();
-    res.line(elem);
-};
-
-// Add property.
-$generatorSpring.property = function(res, obj, propName, setterName, dflt) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            const missDflt = _.isNil(dflt);
-
-            // Add to result if no default provided or value not equals to default.
-            if (missDflt || (!missDflt && val !== dflt)) {
-                $generatorSpring.element(res, 'property', 'name', setterName ? setterName : propName, 'value', $generatorSpring.escape(val));
-
-                return true;
-            }
-        }
-    }
-
-    return false;
-};
-
-// Add property for class name.
-$generatorSpring.classNameProperty = function(res, obj, propName) {
-    const val = obj[propName];
-
-    if (!_.isNil(val))
-        $generatorSpring.element(res, 'property', 'name', propName, 'value', $generatorCommon.JavaTypes.fullClassName(val));
-};
-
-// Add list property.
-$generatorSpring.listProperty = function(res, obj, propName, listType, rowFactory) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        if (!listType)
-            listType = 'list';
-
-        if (!rowFactory)
-            rowFactory = (v) => '<value>' + $generatorSpring.escape(v) + '</value>';
-
-        res.startBlock('<property name="' + propName + '">');
-        res.startBlock('<' + listType + '>');
-
-        _.forEach(val, (v) => res.line(rowFactory(v)));
-
-        res.endBlock('</' + listType + '>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Add array property
-$generatorSpring.arrayProperty = function(res, obj, propName, descr, rowFactory) {
-    const val = obj[propName];
-
-    if (val && val.length > 0) {
-        res.emptyLineIfNeeded();
-
-        if (!rowFactory)
-            rowFactory = (v) => '<bean class="' + v + '"/>';
-
-        res.startBlock('<property name="' + propName + '">');
-        res.startBlock('<list>');
-
-        _.forEach(val, (v) => res.append(rowFactory(v)));
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-};
-
-/**
- * Add bean property with internal content.
- *
- * @param res Optional configuration presentation builder object.
- * @param bean Bean object for code generation.
- * @param beanPropName Name of property to set generated bean as value.
- * @param desc Bean metadata object.
- * @param createBeanAlthoughNoProps Always generate bean even it has no properties defined.
- */
-$generatorSpring.beanProperty = function(res, bean, beanPropName, desc, createBeanAlthoughNoProps) {
-    const props = desc.fields;
-
-    if (bean && $generatorCommon.hasProperty(bean, props)) {
-        if (!createBeanAlthoughNoProps)
-            res.startSafeBlock();
-
-        res.emptyLineIfNeeded();
-        res.startBlock('<property name="' + beanPropName + '">');
-
-        if (createBeanAlthoughNoProps)
-            res.startSafeBlock();
-
-        res.startBlock('<bean class="' + desc.className + '">');
-
-        let hasData = false;
-
-        _.forIn(props, function(descr, propName) {
-            if (props.hasOwnProperty(propName)) {
-                if (descr) {
-                    switch (descr.type) {
-                        case 'list':
-                            $generatorSpring.listProperty(res, bean, propName, descr.setterName);
-
-                            break;
-
-                        case 'array':
-                            $generatorSpring.arrayProperty(res, bean, propName, descr);
-
-                            break;
-
-                        case 'propertiesAsList':
-                            const val = bean[propName];
-
-                            if (val && val.length > 0) {
-                                res.startBlock('<property name="' + propName + '">');
-                                res.startBlock('<props>');
-
-                                _.forEach(val, function(nameAndValue) {
-                                    const eqIndex = nameAndValue.indexOf('=');
-                                    if (eqIndex >= 0) {
-                                        res.line('<prop key="' + $generatorSpring.escape(nameAndValue.substring(0, eqIndex)) + '">' +
-                                            $generatorSpring.escape(nameAndValue.substr(eqIndex + 1)) + '</prop>');
-                                    }
-                                });
-
-                                res.endBlock('</props>');
-                                res.endBlock('</property>');
-
-                                hasData = true;
-                            }
-
-                            break;
-
-                        case 'bean':
-                            if ($generatorCommon.isDefinedAndNotEmpty(bean[propName])) {
-                                res.startBlock('<property name="' + propName + '">');
-                                res.line('<bean class="' + bean[propName] + '"/>');
-                                res.endBlock('</property>');
-
-                                hasData = true;
-                            }
-
-                            break;
-
-                        default:
-                            if ($generatorSpring.property(res, bean, propName, descr.setterName, descr.dflt))
-                                hasData = true;
-                    }
-                }
-                else
-                    if ($generatorSpring.property(res, bean, propName))
-                        hasData = true;
-            }
-        });
-
-        res.endBlock('</bean>');
-
-        if (createBeanAlthoughNoProps && !hasData) {
-            res.rollbackSafeBlock();
-
-            res.line('<bean class="' + desc.className + '"/>');
-        }
-
-        res.endBlock('</property>');
-
-        if (!createBeanAlthoughNoProps && !hasData)
-            res.rollbackSafeBlock();
-    }
-    else if (createBeanAlthoughNoProps) {
-        res.emptyLineIfNeeded();
-        res.startBlock('<property name="' + beanPropName + '">');
-        res.line('<bean class="' + desc.className + '"/>');
-        res.endBlock('</property>');
-    }
-};
-
-/**
- * Add bean property without internal content.
- *
- * @param res Optional configuration presentation builder object.
- * @param obj Object to take bean class name.
- * @param propName Property name.
- */
-$generatorSpring.simpleBeanProperty = function(res, obj, propName) {
-    if (!_.isNil(obj)) {
-        const val = obj[propName];
-
-        if ($generatorCommon.isDefinedAndNotEmpty(val)) {
-            res.startBlock('<property name="' + propName + '">');
-            res.line('<bean class="' + val + '"/>');
-            res.endBlock('</property>');
-        }
-    }
-
-    return false;
-};
-
-// Generate eviction policy.
-$generatorSpring.evictionPolicy = function(res, evtPlc, propName) {
-    if (evtPlc && evtPlc.kind) {
-        $generatorSpring.beanProperty(res, evtPlc[evtPlc.kind.toUpperCase()], propName,
-            $generatorCommon.EVICTION_POLICIES[evtPlc.kind], true);
-    }
-};
-
-// Generate discovery.
-$generatorSpring.clusterGeneral = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'name', 'gridName');
-    $generatorSpring.property(res, cluster, 'localHost');
-
-    if (cluster.discovery) {
-        res.startBlock('<property name="discoverySpi">');
-        res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">');
-        res.startBlock('<property name="ipFinder">');
-
-        const d = cluster.discovery;
-
-        switch (d.kind) {
-            case 'Multicast':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder">');
-
-                if (d.Multicast) {
-                    $generatorSpring.property(res, d.Multicast, 'multicastGroup');
-                    $generatorSpring.property(res, d.Multicast, 'multicastPort');
-                    $generatorSpring.property(res, d.Multicast, 'responseWaitTime');
-                    $generatorSpring.property(res, d.Multicast, 'addressRequestAttempts');
-                    $generatorSpring.property(res, d.Multicast, 'localAddress');
-                    $generatorSpring.listProperty(res, d.Multicast, 'addresses');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Vm':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">');
-
-                if (d.Vm)
-                    $generatorSpring.listProperty(res, d.Vm, 'addresses');
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'S3':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder">');
-
-                if (d.S3) {
-                    if (d.S3.bucketName)
-                        res.line('<property name="bucketName" value="' + $generatorSpring.escape(d.S3.bucketName) + '"/>');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Cloud':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.cloud.TcpDiscoveryCloudIpFinder">');
-
-                if (d.Cloud) {
-                    $generatorSpring.property(res, d.Cloud, 'credential');
-                    $generatorSpring.property(res, d.Cloud, 'credentialPath');
-                    $generatorSpring.property(res, d.Cloud, 'identity');
-                    $generatorSpring.property(res, d.Cloud, 'provider');
-                    $generatorSpring.listProperty(res, d.Cloud, 'regions');
-                    $generatorSpring.listProperty(res, d.Cloud, 'zones');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'GoogleStorage':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.gce.TcpDiscoveryGoogleStorageIpFinder">');
-
-                if (d.GoogleStorage) {
-                    $generatorSpring.property(res, d.GoogleStorage, 'projectName');
-                    $generatorSpring.property(res, d.GoogleStorage, 'bucketName');
-                    $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountP12FilePath');
-                    $generatorSpring.property(res, d.GoogleStorage, 'serviceAccountId');
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Jdbc':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder">');
-
-                if (d.Jdbc) {
-                    const datasource = d.Jdbc;
-
-                    res.line('<property name="initSchema" value="' + (!_.isNil(datasource.initSchema) && datasource.initSchema) + '"/>');
-
-                    if (datasource.dataSourceBean && datasource.dialect) {
-                        res.line('<property name="dataSource" ref="' + datasource.dataSourceBean + '"/>');
-
-                        if (!_.find(res.datasources, { dataSourceBean: datasource.dataSourceBean })) {
-                            res.datasources.push({
-                                dataSourceBean: datasource.dataSourceBean,
-                                dialect: datasource.dialect
-                            });
-                        }
-                    }
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'SharedFs':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder">');
-
-                if (d.SharedFs)
-                    $generatorSpring.property(res, d.SharedFs, 'path');
-
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'ZooKeeper':
-                res.startBlock('<bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.zk.TcpDiscoveryZookeeperIpFinder">');
-
-                if (d.ZooKeeper) {
-                    if ($generatorCommon.isDefinedAndNotEmpty(d.ZooKeeper.curator)) {
-                        res.startBlock('<property name="curator">');
-                        res.line('<bean class="' + d.ZooKeeper.curator + '"/>');
-                        res.endBlock('</property>');
-                    }
-
-                    $generatorSpring.property(res, d.ZooKeeper, 'zkConnectionString');
-
-                    if (d.ZooKeeper.retryPolicy && d.ZooKeeper.retryPolicy.kind) {
-                        const kind = d.ZooKeeper.retryPolicy.kind;
-                        const retryPolicy = d.ZooKeeper.retryPolicy[kind];
-                        const customClassDefined = retryPolicy && $generatorCommon.isDefinedAndNotEmpty(retryPolicy.className);
-
-                        if (kind !== 'Custom' || customClassDefined)
-                            res.startBlock('<property name="retryPolicy">');
-
-                        switch (kind) {
-                            case 'ExponentialBackoff':
-                                res.startBlock('<bean class="org.apache.curator.retry.ExponentialBackoffRetry">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxRetries', 10);
-                                $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxSleepMs', null, true);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'BoundedExponentialBackoff':
-                                res.startBlock('<bean class="org.apache.curator.retry.BoundedExponentialBackoffRetry">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'baseSleepTimeMs', 1000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'maxSleepTimeMs', 2147483647);
-                                $generatorSpring.constructorArg(res, 2, retryPolicy, 'maxRetries', 10);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'UntilElapsed':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryUntilElapsed">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'maxElapsedTimeMs', 60000);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'NTimes':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryNTimes">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'n', 10);
-                                $generatorSpring.constructorArg(res, 1, retryPolicy, 'sleepMsBetweenRetries', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'OneTime':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryOneTime">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'sleepMsBetweenRetry', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'Forever':
-                                res.startBlock('<bean class="org.apache.curator.retry.RetryForever">');
-                                $generatorSpring.constructorArg(res, 0, retryPolicy, 'retryIntervalMs', 1000);
-                                res.endBlock('</bean>');
-
-                                break;
-
-                            case 'Custom':
-                                if (customClassDefined)
-                                    res.line('<bean class="' + retryPolicy.className + '"/>');
-
-                                break;
-
-                            default:
-                        }
-
-                        if (kind !== 'Custom' || customClassDefined)
-                            res.endBlock('</property>');
-                    }
-
-                    $generatorSpring.property(res, d.ZooKeeper, 'basePath', null, '/services');
-                    $generatorSpring.property(res, d.ZooKeeper, 'serviceName', null, 'ignite');
-                    $generatorSpring.property(res, d.ZooKeeper, 'allowDuplicateRegistrations', null, false);
-                }
-
-                res.endBlock('</bean>');
-
-                break;
-
-            default:
-                res.line('Unknown discovery kind: ' + d.kind);
-        }
-
-        res.endBlock('</property>');
-
-        $generatorSpring.clusterDiscovery(d, res);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate atomics group.
-$generatorSpring.clusterAtomics = function(atomics, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.hasAtLeastOneProperty(atomics, ['cacheMode', 'atomicSequenceReserveSize', 'backups'])) {
-        res.startSafeBlock();
-
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="atomicConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.AtomicConfiguration">');
-
-        const cacheMode = atomics.cacheMode ? atomics.cacheMode : 'PARTITIONED';
-
-        let hasData = cacheMode !== 'PARTITIONED';
-
-        $generatorSpring.property(res, atomics, 'cacheMode', null, 'PARTITIONED');
-
-        hasData = $generatorSpring.property(res, atomics, 'atomicSequenceReserveSize', null, 1000) || hasData;
-
-        if (cacheMode === 'PARTITIONED')
-            hasData = $generatorSpring.property(res, atomics, 'backups', null, 0) || hasData;
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-
-        if (!hasData)
-            res.rollbackSafeBlock();
-    }
-
-    return res;
-};
-
-// Generate binary group.
-$generatorSpring.clusterBinary = function(binary, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.binaryIsDefined(binary)) {
-        res.startBlock('<property name="binaryConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.BinaryConfiguration">');
-
-        $generatorSpring.simpleBeanProperty(res, binary, 'idMapper');
-        $generatorSpring.simpleBeanProperty(res, binary, 'nameMapper');
-        $generatorSpring.simpleBeanProperty(res, binary, 'serializer');
-
-        if ($generatorCommon.isDefinedAndNotEmpty(binary.typeConfigurations)) {
-            res.startBlock('<property name="typeConfigurations">');
-            res.startBlock('<list>');
-
-            _.forEach(binary.typeConfigurations, function(type) {
-                res.startBlock('<bean class="org.apache.ignite.binary.BinaryTypeConfiguration">');
-
-                $generatorSpring.property(res, type, 'typeName');
-                $generatorSpring.simpleBeanProperty(res, type, 'idMapper');
-                $generatorSpring.simpleBeanProperty(res, type, 'nameMapper');
-                $generatorSpring.simpleBeanProperty(res, type, 'serializer');
-                $generatorSpring.property(res, type, 'enum', null, false);
-
-                res.endBlock('</bean>');
-            });
-
-            res.endBlock('</list>');
-            res.endBlock('</property>');
-        }
-
-        $generatorSpring.property(res, binary, 'compactFooter', null, true);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache key configurations.
-$generatorSpring.clusterCacheKeyConfiguration = function(keyCfgs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    keyCfgs = _.filter(keyCfgs, (cfg) => cfg.typeName && cfg.affinityKeyFieldName);
-
-    if (_.isEmpty(keyCfgs))
-        return res;
-
-    res.startBlock('<property name="cacheKeyConfiguration">');
-    res.startBlock('<array>');
-
-    _.forEach(keyCfgs, (cfg) => {
-        res.startBlock('<bean class="org.apache.ignite.cache.CacheKeyConfiguration">');
-
-        $generatorSpring.constructorArg(res, -1, cfg, 'typeName');
-        $generatorSpring.constructorArg(res, -1, cfg, 'affinityKeyFieldName');
-
-        res.endBlock('</bean>');
-    });
-
-    res.endBlock('</array>');
-    res.endBlock('</property>');
-
-    return res;
-};
-
-// Generate collision group.
-$generatorSpring.clusterCollision = function(collision, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (collision && collision.kind && collision.kind !== 'Noop') {
-        const spi = collision[collision.kind];
-
-        if (collision.kind !== 'Custom' || (spi && $generatorCommon.isDefinedAndNotEmpty(spi.class))) {
-            res.startBlock('<property name="collisionSpi">');
-
-            switch (collision.kind) {
-                case 'JobStealing':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.jobstealing.JobStealingCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'activeJobsThreshold', null, 95);
-                    $generatorSpring.property(res, spi, 'waitJobsThreshold', null, 0);
-                    $generatorSpring.property(res, spi, 'messageExpireTime', null, 1000);
-                    $generatorSpring.property(res, spi, 'maximumStealingAttempts', null, 5);
-                    $generatorSpring.property(res, spi, 'stealingEnabled', null, true);
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.externalCollisionListener)) {
-                        res.needEmptyLine = true;
-
-                        res.startBlock('<property name="externalCollisionListener">');
-                        res.line('<bean class="' + spi.externalCollisionListener + ' "/>');
-                        res.endBlock('</property>');
-                    }
-
-                    if ($generatorCommon.isDefinedAndNotEmpty(spi.stealingAttributes)) {
-                        res.needEmptyLine = true;
-
-                        res.startBlock('<property name="stealingAttributes">');
-                        res.startBlock('<map>');
-
-                        _.forEach(spi.stealingAttributes, function(attr) {
-                            $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value);
-                        });
-
-                        res.endBlock('</map>');
-                        res.endBlock('</property>');
-                    }
-
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'FifoQueue':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.fifoqueue.FifoQueueCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'parallelJobsNumber');
-                    $generatorSpring.property(res, spi, 'waitingJobsNumber');
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'PriorityQueue':
-                    res.startBlock('<bean class="org.apache.ignite.spi.collision.priorityqueue.PriorityQueueCollisionSpi">');
-                    $generatorSpring.property(res, spi, 'parallelJobsNumber');
-                    $generatorSpring.property(res, spi, 'waitingJobsNumber');
-                    $generatorSpring.property(res, spi, 'priorityAttributeKey', null, 'grid.task.priority');
-                    $generatorSpring.property(res, spi, 'jobPriorityAttributeKey', null, 'grid.job.priority');
-                    $generatorSpring.property(res, spi, 'defaultPriority', null, 0);
-                    $generatorSpring.property(res, spi, 'starvationIncrement', null, 1);
-                    $generatorSpring.property(res, spi, 'starvationPreventionEnabled', null, true);
-                    res.endBlock('</bean>');
-
-                    break;
-
-                case 'Custom':
-                    res.line('<bean class="' + spi.class + '"/>');
-
-                    break;
-
-                default:
-            }
-
-            res.endBlock('</property>');
-        }
-    }
-
-    return res;
-};
-
-// Generate communication group.
-$generatorSpring.clusterCommunication = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.beanProperty(res, cluster.communication, 'communicationSpi', $generatorCommon.COMMUNICATION_CONFIGURATION);
-
-    $generatorSpring.property(res, cluster, 'networkTimeout', null, 5000);
-    $generatorSpring.property(res, cluster, 'networkSendRetryDelay', null, 1000);
-    $generatorSpring.property(res, cluster, 'networkSendRetryCount', null, 3);
-    $generatorSpring.property(res, cluster, 'segmentCheckFrequency');
-    $generatorSpring.property(res, cluster, 'waitForSegmentOnStart', null, false);
-    $generatorSpring.property(res, cluster, 'discoveryStartupDelay', null, 60000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * XML generator for cluster's REST access configuration.
- *
- * @param connector Cluster REST connector configuration.
- * @param res Optional configuration presentation builder object.
- * @returns Configuration presentation builder object
- */
-$generatorSpring.clusterConnector = function(connector, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (!_.isNil(connector) && connector.enabled) {
-        const cfg = _.cloneDeep($generatorCommon.CONNECTOR_CONFIGURATION);
-
-        if (connector.sslEnabled) {
-            cfg.fields.sslClientAuth = {dflt: false};
-            cfg.fields.sslFactory = {type: 'bean'};
-        }
-
-        $generatorSpring.beanProperty(res, connector, 'connectorConfiguration', cfg, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate deployment group.
-$generatorSpring.clusterDeployment = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorSpring.property(res, cluster, 'deploymentMode', null, 'SHARED'))
-        res.needEmptyLine = true;
-
-    const p2pEnabled = cluster.peerClassLoadingEnabled;
-
-    if (!_.isNil(p2pEnabled)) {
-        $generatorSpring.property(res, cluster, 'peerClassLoadingEnabled', null, false);
-
-        if (p2pEnabled) {
-            $generatorSpring.property(res, cluster, 'peerClassLoadingMissedResourcesCacheSize', null, 100);
-            $generatorSpring.property(res, cluster, 'peerClassLoadingThreadPoolSize', null, 2);
-            $generatorSpring.listProperty(res, cluster, 'peerClassLoadingLocalClassPathExclude');
-        }
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate discovery group.
-$generatorSpring.clusterDiscovery = function(disco, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (disco) {
-        $generatorSpring.property(res, disco, 'localAddress');
-        $generatorSpring.property(res, disco, 'localPort', null, 47500);
-        $generatorSpring.property(res, disco, 'localPortRange', null, 100);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.addressResolver))
-            $generatorSpring.beanProperty(res, disco, 'addressResolver', {className: disco.addressResolver}, true);
-        $generatorSpring.property(res, disco, 'socketTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'ackTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'maxAckTimeout', null, 600000);
-        $generatorSpring.property(res, disco, 'networkTimeout', null, 5000);
-        $generatorSpring.property(res, disco, 'joinTimeout', null, 0);
-        $generatorSpring.property(res, disco, 'threadPriority', null, 10);
-        $generatorSpring.property(res, disco, 'heartbeatFrequency', null, 2000);
-        $generatorSpring.property(res, disco, 'maxMissedHeartbeats', null, 1);
-        $generatorSpring.property(res, disco, 'maxMissedClientHeartbeats', null, 5);
-        $generatorSpring.property(res, disco, 'topHistorySize', null, 1000);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.listener))
-            $generatorSpring.beanProperty(res, disco, 'listener', {className: disco.listener}, true);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.dataExchange))
-            $generatorSpring.beanProperty(res, disco, 'dataExchange', {className: disco.dataExchange}, true);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.metricsProvider))
-            $generatorSpring.beanProperty(res, disco, 'metricsProvider', {className: disco.metricsProvider}, true);
-        $generatorSpring.property(res, disco, 'reconnectCount', null, 10);
-        $generatorSpring.property(res, disco, 'statisticsPrintFrequency', null, 0);
-        $generatorSpring.property(res, disco, 'ipFinderCleanFrequency', null, 60000);
-        if ($generatorCommon.isDefinedAndNotEmpty(disco.authenticator))
-            $generatorSpring.beanProperty(res, disco, 'authenticator', {className: disco.authenticator}, true);
-        $generatorSpring.property(res, disco, 'forceServerMode', null, false);
-        $generatorSpring.property(res, disco, 'clientReconnectDisabled', null, false);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate events group.
-$generatorSpring.clusterEvents = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.includeEventTypes && cluster.includeEventTypes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="includeEventTypes">');
-
-        const evtGrps = angular.element(document.getElementById('app')).injector().get('igniteEventGroups');
-
-        if (cluster.includeEventTypes.length === 1) {
-            const evtGrp = _.find(evtGrps, {value: cluster.includeEventTypes[0]});
-
-            if (evtGrp)
-                res.line('<util:constant static-field="' + evtGrp.class + '.' + evtGrp.value + '"/>');
-        }
-        else {
-            res.startBlock('<list>');
-
-            _.forEach(cluster.includeEventTypes, (item, ix) => {
-                ix > 0 && res.line();
-
-                const evtGrp = _.find(evtGrps, {value: item});
-
-                if (evtGrp) {
-                    res.line('<!-- EventType.' + item + ' -->');
-
-                    _.forEach(evtGrp.events, (event) => res.line('<util:constant static-field="' + evtGrp.class + '.' + event + '"/>'));
-                }
-            });
-
-            res.endBlock('</list>');
-        }
-
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate failover group.
-$generatorSpring.clusterFailover = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.failoverSpi) && _.findIndex(cluster.failoverSpi, function(spi) {
-        return $generatorCommon.isDefinedAndNotEmpty(spi.kind) && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')));
-    }) >= 0) {
-        res.startBlock('<property name="failoverSpi">');
-        res.startBlock('<list>');
-
-        _.forEach(cluster.failoverSpi, function(spi) {
-            if (spi.kind && (spi.kind !== 'Custom' || $generatorCommon.isDefinedAndNotEmpty(_.get(spi, spi.kind + '.class')))) {
-                const maxAttempts = _.get(spi, spi.kind + '.maximumFailoverAttempts');
-
-                if ((spi.kind === 'JobStealing' || spi.kind === 'Always') && $generatorCommon.isDefinedAndNotEmpty(maxAttempts) && maxAttempts !== 5) {
-                    res.startBlock('<bean class="' + $generatorCommon.failoverSpiClass(spi) + '">');
-
-                    $generatorSpring.property(res, spi[spi.kind], 'maximumFailoverAttempts', null, 5);
-
-                    res.endBlock('</bean>');
-                }
-                else
-                    res.line('<bean class="' + $generatorCommon.failoverSpiClass(spi) + '"/>');
-
-                res.needEmptyLine = true;
-            }
-        });
-
-        res.needEmptyLine = true;
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorSpring.clusterLogger = function(logger, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.loggerConfigured(logger)) {
-        res.startBlock('<property name="gridLogger">');
-
-        const log = logger[logger.kind];
-
-        switch (logger.kind) {
-            case 'Log4j2':
-                res.startBlock('<bean class="org.apache.ignite.logger.log4j2.Log4J2Logger">');
-                res.line('<constructor-arg value="' + $generatorSpring.escape(log.path) + '"/>');
-                $generatorSpring.property(res, log, 'level');
-                res.endBlock('</bean>');
-
-                break;
-
-            case 'Null':
-                res.line('<bean class="org.apache.ignite.logger.NullLogger"/>');
-
-                break;
-
-            case 'Java':
-                res.line('<bean class="org.apache.ignite.logger.java.JavaLogger"/>');
-
-                break;
-
-            case 'JCL':
-                res.line('<bean class="org.apache.ignite.logger.jcl.JclLogger"/>');
-
-                break;
-
-            case 'SLF4J':
-                res.line('<bean class="org.apache.ignite.logger.slf4j.Slf4jLogger"/>');
-
-                break;
-
-            case 'Log4j':
-                if (log.mode === 'Default' && !$generatorCommon.isDefinedAndNotEmpty(log.level))
-                    res.line('<bean class="org.apache.ignite.logger.log4j.Log4JLogger"/>');
-                else {
-                    res.startBlock('<bean class="org.apache.ignite.logger.log4j.Log4JLogger">');
-
-                    if (log.mode === 'Path')
-                        res.line('<constructor-arg value="' + $generatorSpring.escape(log.path) + '"/>');
-
-                    $generatorSpring.property(res, log, 'level');
-                    res.endBlock('</bean>');
-                }
-
-                break;
-
-            case 'Custom':
-                res.line('<bean class="' + log.class + '"/>');
-
-                break;
-
-            default:
-        }
-
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate marshaller group.
-$generatorSpring.clusterMarshaller = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const marshaller = cluster.marshaller;
-
-    if (marshaller && marshaller.kind)
-        $generatorSpring.beanProperty(res, marshaller[marshaller.kind], 'marshaller', $generatorCommon.MARSHALLERS[marshaller.kind], true);
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cluster, 'marshalLocalJobs', null, false);
-    $generatorSpring.property(res, cluster, 'marshallerCacheKeepAliveTime', null, 10000);
-    $generatorSpring.property(res, cluster, 'marshallerCacheThreadPoolSize', 'marshallerCachePoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate metrics group.
-$generatorSpring.clusterMetrics = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'metricsExpireTime');
-    $generatorSpring.property(res, cluster, 'metricsHistorySize', null, 10000);
-    $generatorSpring.property(res, cluster, 'metricsLogFrequency', null, 60000);
-    $generatorSpring.property(res, cluster, 'metricsUpdateFrequency', null, 2000);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate swap group.
-$generatorSpring.clusterSwap = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.swapSpaceSpi && cluster.swapSpaceSpi.kind === 'FileSwapSpaceSpi') {
-        $generatorSpring.beanProperty(res, cluster.swapSpaceSpi.FileSwapSpaceSpi, 'swapSpaceSpi',
-            $generatorCommon.SWAP_SPACE_SPI, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate time group.
-$generatorSpring.clusterTime = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'clockSyncSamples', null, 8);
-    $generatorSpring.property(res, cluster, 'clockSyncFrequency', null, 120000);
-    $generatorSpring.property(res, cluster, 'timeServerPortBase', null, 31100);
-    $generatorSpring.property(res, cluster, 'timeServerPortRange', null, 100);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate OBC configuration group.
-$generatorSpring.clusterODBC = function(odbc, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (odbc && odbc.odbcEnabled)
-        $generatorSpring.beanProperty(res, odbc, 'odbcConfiguration', $generatorCommon.ODBC_CONFIGURATION, true);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate thread pools group.
-$generatorSpring.clusterPools = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cluster, 'publicThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'systemThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'managementThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'igfsThreadPoolSize');
-    $generatorSpring.property(res, cluster, 'rebalanceThreadPoolSize');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate transactions group.
-$generatorSpring.clusterTransactions = function(transactionConfiguration, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.beanProperty(res, transactionConfiguration, 'transactionConfiguration', $generatorCommon.TRANSACTION_CONFIGURATION, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate user attributes group.
-$generatorSpring.clusterUserAttributes = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(cluster.attributes)) {
-        res.startBlock('<property name="userAttributes">');
-        res.startBlock('<map>');
-
-        _.forEach(cluster.attributes, function(attr) {
-            $generatorSpring.element(res, 'entry', 'key', attr.name, 'value', attr.value);
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-/**
- * XML generator for cluster's SSL configuration.
- *
- * @param cluster Cluster to get SSL configuration.
- * @param res Optional configuration presentation builder object.
- * @returns Configuration presentation builder object
- */
-$generatorSpring.clusterSsl = function(cluster, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cluster.sslEnabled && !_.isNil(cluster.sslContextFactory)) {
-        let sslFactory;
-
-        if (_.isEmpty(cluster.sslContextFactory.keyStoreFilePath) && _.isEmpty(cluster.sslContextFactory.trustStoreFilePath))
-            sslFactory = cluster.sslContextFactory;
-        else {
-            sslFactory = _.clone(cluster.sslContextFactory);
-
-            sslFactory.keyStorePassword = _.isEmpty(cluster.sslContextFactory.keyStoreFilePath) ? null : '${ssl.key.storage.password}';
-            sslFactory.trustStorePassword = _.isEmpty(cluster.sslContextFactory.trustStoreFilePath) ? null : '${ssl.trust.storage.password}';
-        }
-
-        const propsDesc = $generatorCommon.isDefinedAndNotEmpty(cluster.sslContextFactory.trustManagers) ?
-            $generatorCommon.SSL_CONFIGURATION_TRUST_MANAGER_FACTORY :
-            $generatorCommon.SSL_CONFIGURATION_TRUST_FILE_FACTORY;
-
-        $generatorSpring.beanProperty(res, sslFactory, 'sslContextFactory', propsDesc, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate cache general group.
-$generatorSpring.cacheGeneral = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'name');
-
-    $generatorSpring.property(res, cache, 'cacheMode');
-    $generatorSpring.property(res, cache, 'atomicityMode');
-
-    if (cache.cacheMode === 'PARTITIONED' && $generatorSpring.property(res, cache, 'backups'))
-        $generatorSpring.property(res, cache, 'readFromBackup');
-
-    $generatorSpring.property(res, cache, 'copyOnRead');
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.atomicityMode === 'TRANSACTIONAL')
-        $generatorSpring.property(res, cache, 'invalidate');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache memory group.
-$generatorSpring.cacheMemory = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'memoryMode', null, 'ONHEAP_TIERED');
-
-    if (cache.memoryMode !== 'OFFHEAP_VALUES')
-        $generatorSpring.property(res, cache, 'offHeapMaxMemory', null, -1);
-
-    res.softEmptyLine();
-
-    $generatorSpring.evictionPolicy(res, cache.evictionPolicy, 'evictionPolicy');
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'startSize', null, 1500000);
-    $generatorSpring.property(res, cache, 'swapEnabled', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache query & indexing group.
-$generatorSpring.cacheQuery = function(cache, domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'sqlSchema');
-    $generatorSpring.property(res, cache, 'sqlOnheapRowCacheSize', null, 10240);
-    $generatorSpring.property(res, cache, 'longQueryWarningTimeout', null, 3000);
-
-    const indexedTypes = _.filter(domains, (domain) => domain.queryMetadata === 'Annotations');
-
-    if (indexedTypes.length > 0) {
-        res.softEmptyLine();
-
-        res.startBlock('<property name="indexedTypes">');
-        res.startBlock('<list>');
-
-        _.forEach(indexedTypes, function(domain) {
-            res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + '</value>');
-            res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + '</value>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    res.softEmptyLine();
-
-    $generatorSpring.listProperty(res, cache, 'sqlFunctionClasses');
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'snapshotableIndex', null, false);
-    $generatorSpring.property(res, cache, 'sqlEscapeAll', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache store group.
-$generatorSpring.cacheStore = function(cache, domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheStoreFactory && cache.cacheStoreFactory.kind) {
-        const factoryKind = cache.cacheStoreFactory.kind;
-
-        const storeFactory = cache.cacheStoreFactory[factoryKind];
-
-        if (storeFactory) {
-            if (factoryKind === 'CacheJdbcPojoStoreFactory') {
-                res.startBlock('<property name="cacheStoreFactory">');
-                res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">');
-
-                $generatorSpring.property(res, storeFactory, 'dataSourceBean');
-
-                res.startBlock('<property name="dialect">');
-                res.line('<bean class="' + $generatorCommon.jdbcDialectClassName(storeFactory.dialect) + '"/>');
-                res.endBlock('</property>');
-
-                if (storeFactory.sqlEscapeAll)
-                    $generatorSpring.property(res, storeFactory, 'sqlEscapeAll');
-
-                const domainConfigs = _.filter(domains, function(domain) {
-                    return $generatorCommon.isDefinedAndNotEmpty(domain.databaseTable);
-                });
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-                    res.startBlock('<property name="types">');
-                    res.startBlock('<list>');
-
-                    _.forEach(domainConfigs, function(domain) {
-                        res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.JdbcType">');
-
-                        $generatorSpring.property(res, cache, 'name', 'cacheName');
-
-                        $generatorSpring.classNameProperty(res, domain, 'keyType');
-                        $generatorSpring.property(res, domain, 'valueType');
-
-                        $generatorSpring.domainStore(domain, res);
-
-                        res.endBlock('</bean>');
-                    });
-
-                    res.endBlock('</list>');
-                    res.endBlock('</property>');
-                }
-
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-            else if (factoryKind === 'CacheJdbcBlobStoreFactory') {
-                res.startBlock('<property name="cacheStoreFactory">');
-                res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcBlobStoreFactory">');
-
-                if (storeFactory.connectVia === 'DataSource')
-                    $generatorSpring.property(res, storeFactory, 'dataSourceBean');
-                else {
-                    $generatorSpring.property(res, storeFactory, 'connectionUrl');
-                    $generatorSpring.property(res, storeFactory, 'user');
-                    res.line('<property name="password" value="${ds.' + storeFactory.user + '.password}"/>');
-                }
-
-                $generatorSpring.property(res, storeFactory, 'initSchema');
-                $generatorSpring.property(res, storeFactory, 'createTableQuery');
-                $generatorSpring.property(res, storeFactory, 'loadQuery');
-                $generatorSpring.property(res, storeFactory, 'insertQuery');
-                $generatorSpring.property(res, storeFactory, 'updateQuery');
-                $generatorSpring.property(res, storeFactory, 'deleteQuery');
-
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-            else
-                $generatorSpring.beanProperty(res, storeFactory, 'cacheStoreFactory', $generatorCommon.STORE_FACTORIES[factoryKind], true);
-
-            if (storeFactory.dataSourceBean && (storeFactory.connectVia ? (storeFactory.connectVia === 'DataSource' ? storeFactory.dialect : null) : storeFactory.dialect)) {
-                if (!_.find(res.datasources, { dataSourceBean: storeFactory.dataSourceBean})) {
-                    res.datasources.push({
-                        dataSourceBean: storeFactory.dataSourceBean,
-                        dialect: storeFactory.dialect
-                    });
-                }
-            }
-        }
-    }
-
-    res.softEmptyLine();
-
-    $generatorSpring.property(res, cache, 'storeKeepBinary', null, false);
-    $generatorSpring.property(res, cache, 'loadPreviousValue', null, false);
-    $generatorSpring.property(res, cache, 'readThrough', null, false);
-    $generatorSpring.property(res, cache, 'writeThrough', null, false);
-
-    res.softEmptyLine();
-
-    if (cache.writeBehindEnabled) {
-        $generatorSpring.property(res, cache, 'writeBehindEnabled', null, false);
-        $generatorSpring.property(res, cache, 'writeBehindBatchSize', null, 512);
-        $generatorSpring.property(res, cache, 'writeBehindFlushSize', null, 10240);
-        $generatorSpring.property(res, cache, 'writeBehindFlushFrequency', null, 5000);
-        $generatorSpring.property(res, cache, 'writeBehindFlushThreadCount', null, 1);
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache node filter group.
-$generatorSpring.cacheNodeFilter = function(cache, igfss, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const kind = _.get(cache, 'nodeFilter.kind');
-
-    if (_.isNil(kind) || _.isNil(cache.nodeFilter[kind]))
-        return res;
-
-    switch (kind) {
-        case 'IGFS':
-            const foundIgfs = _.find(igfss, (igfs) => igfs._id === cache.nodeFilter.IGFS.igfs);
-
-            if (foundIgfs) {
-                res.startBlock('<property name="nodeFilter">');
-                res.startBlock('<bean class="org.apache.ignite.internal.processors.igfs.IgfsNodePredicate">');
-                res.line('<constructor-arg value="' + foundIgfs.name + '"/>');
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'OnNodes':
-            const nodes = cache.nodeFilter.OnNodes.nodeIds;
-
-            if ($generatorCommon.isDefinedAndNotEmpty(nodes)) {
-                res.startBlock('<property name="nodeFilter">');
-                res.startBlock('<bean class="org.apache.ignite.internal.util.lang.GridNodePredicate">');
-                res.startBlock('<constructor-arg>');
-                res.startBlock('<list>');
-
-                _.forEach(nodes, (nodeId) => {
-                    res.startBlock('<bean class="java.util.UUID" factory-method="fromString">');
-                    res.line('<constructor-arg value="' + nodeId + '"/>');
-                    res.endBlock('</bean>');
-                });
-
-                res.endBlock('</list>');
-                res.endBlock('</constructor-arg>');
-                res.endBlock('</bean>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'Custom':
-            res.startBlock('<property name="nodeFilter">');
-            res.line('<bean class="' + cache.nodeFilter.Custom.className + '"/>');
-            res.endBlock('</property>');
-
-            break;
-
-        default: break;
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache concurrency group.
-$generatorSpring.cacheConcurrency = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'maxConcurrentAsyncOperations', null, 500);
-    $generatorSpring.property(res, cache, 'defaultLockTimeout', null, 0);
-    $generatorSpring.property(res, cache, 'atomicWriteOrderMode');
-    $generatorSpring.property(res, cache, 'writeSynchronizationMode', null, 'PRIMARY_SYNC');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache rebalance group.
-$generatorSpring.cacheRebalance = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheMode !== 'LOCAL') {
-        $generatorSpring.property(res, cache, 'rebalanceMode', null, 'ASYNC');
-        $generatorSpring.property(res, cache, 'rebalanceThreadPoolSize', null, 1);
-        $generatorSpring.property(res, cache, 'rebalanceBatchSize', null, 524288);
-        $generatorSpring.property(res, cache, 'rebalanceBatchesPrefetchCount', null, 2);
-        $generatorSpring.property(res, cache, 'rebalanceOrder', null, 0);
-        $generatorSpring.property(res, cache, 'rebalanceDelay', null, 0);
-        $generatorSpring.property(res, cache, 'rebalanceTimeout', null, 10000);
-        $generatorSpring.property(res, cache, 'rebalanceThrottle', null, 0);
-    }
-
-    res.softEmptyLine();
-
-    if (cache.igfsAffinnityGroupSize) {
-        res.startBlock('<property name="affinityMapper">');
-        res.startBlock('<bean class="org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper">');
-        $generatorSpring.constructorArg(res, -1, cache, 'igfsAffinnityGroupSize');
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate cache server near cache group.
-$generatorSpring.cacheServerNearCache = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (cache.cacheMode === 'PARTITIONED' && cache.nearCacheEnabled) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="nearConfiguration">');
-        res.startBlock('<bean class="org.apache.ignite.configuration.NearCacheConfiguration">');
-
-        if (cache.nearConfiguration) {
-            if (cache.nearConfiguration.nearStartSize)
-                $generatorSpring.property(res, cache.nearConfiguration, 'nearStartSize', null, 375000);
-
-            $generatorSpring.evictionPolicy(res, cache.nearConfiguration.nearEvictionPolicy, 'nearEvictionPolicy');
-        }
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate cache statistics group.
-$generatorSpring.cacheStatistics = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, cache, 'statisticsEnabled', null, false);
-    $generatorSpring.property(res, cache, 'managementEnabled', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model query fields.
-$generatorSpring.domainModelQueryFields = function(res, domain) {
-    const fields = domain.fields;
-
-    if (fields && fields.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="fields">');
-        res.startBlock('<map>');
-
-        _.forEach(fields, function(field) {
-            $generatorSpring.element(res, 'entry', 'key', field.name, 'value', $generatorCommon.JavaTypes.fullClassName(field.className));
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model query fields.
-$generatorSpring.domainModelQueryAliases = function(res, domain) {
-    const aliases = domain.aliases;
-
-    if (aliases && aliases.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="aliases">');
-        res.startBlock('<map>');
-
-        _.forEach(aliases, function(alias) {
-            $generatorSpring.element(res, 'entry', 'key', alias.field, 'value', alias.alias);
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model indexes.
-$generatorSpring.domainModelQueryIndexes = function(res, domain) {
-    const indexes = domain.indexes;
-
-    if (indexes && indexes.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="indexes">');
-        res.startBlock('<list>');
-
-        _.forEach(indexes, function(index) {
-            res.startBlock('<bean class="org.apache.ignite.cache.QueryIndex">');
-
-            $generatorSpring.property(res, index, 'name');
-            $generatorSpring.property(res, index, 'indexType');
-
-            const fields = index.fields;
-
-            if (fields && fields.length > 0) {
-                res.startBlock('<property name="fields">');
-                res.startBlock('<map>');
-
-                _.forEach(fields, function(field) {
-                    $generatorSpring.element(res, 'entry', 'key', field.name, 'value', field.direction);
-                });
-
-                res.endBlock('</map>');
-                res.endBlock('</property>');
-            }
-
-            res.endBlock('</bean>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model db fields.
-$generatorSpring.domainModelDatabaseFields = function(res, domain, fieldProp) {
-    const fields = domain[fieldProp];
-
-    if (fields && fields.length > 0) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="' + fieldProp + '">');
-
-        res.startBlock('<list>');
-
-        _.forEach(fields, function(field) {
-            res.startBlock('<bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">');
-
-            $generatorSpring.property(res, field, 'databaseFieldName');
-
-            res.startBlock('<property name="databaseFieldType">');
-            res.line('<util:constant static-field="java.sql.Types.' + field.databaseFieldType + '"/>');
-            res.endBlock('</property>');
-
-            $generatorSpring.property(res, field, 'javaFieldName');
-
-            $generatorSpring.classNameProperty(res, field, 'javaFieldType');
-
-            res.endBlock('</bean>');
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-};
-
-// Generate domain model general group.
-$generatorSpring.domainModelGeneral = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    switch ($generatorCommon.domainQueryMetadata(domain)) {
-        case 'Annotations':
-            if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType) || $generatorCommon.isDefinedAndNotEmpty(domain.valueType)) {
-                res.startBlock('<property name="indexedTypes">');
-                res.startBlock('<list>');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.keyType))
-                    res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.keyType) + '</value>');
-                else
-                    res.line('<value>???</value>');
-
-                if ($generatorCommon.isDefinedAndNotEmpty(domain.valueType))
-                    res.line('<value>' + $generatorCommon.JavaTypes.fullClassName(domain.valueType) + '</value>');
-                else
-                    res.line('<value>>???</value>');
-
-                res.endBlock('</list>');
-                res.endBlock('</property>');
-            }
-
-            break;
-
-        case 'Configuration':
-            $generatorSpring.classNameProperty(res, domain, 'keyType');
-            $generatorSpring.property(res, domain, 'valueType');
-
-            break;
-
-        default:
-    }
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain model for query group.
-$generatorSpring.domainModelQuery = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.domainQueryMetadata(domain) === 'Configuration') {
-        $generatorSpring.domainModelQueryFields(res, domain);
-        $generatorSpring.domainModelQueryAliases(res, domain);
-        $generatorSpring.domainModelQueryIndexes(res, domain);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate domain model for store group.
-$generatorSpring.domainStore = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, domain, 'databaseSchema');
-    $generatorSpring.property(res, domain, 'databaseTable');
-
-    res.softEmptyLine();
-
-    $generatorSpring.domainModelDatabaseFields(res, domain, 'keyFields');
-    $generatorSpring.domainModelDatabaseFields(res, domain, 'valueFields');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-$generatorSpring.cacheQueryMetadata = function(domain, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    res.startBlock('<bean class="org.apache.ignite.cache.QueryEntity">');
-
-    $generatorSpring.classNameProperty(res, domain, 'keyType');
-    $generatorSpring.property(res, domain, 'valueType');
-
-    $generatorSpring.domainModelQuery(domain, res);
-
-    res.endBlock('</bean>');
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-// Generate domain models configs.
-$generatorSpring.cacheDomains = function(domains, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    const domainConfigs = _.filter(domains, function(domain) {
-        return $generatorCommon.domainQueryMetadata(domain) === 'Configuration' &&
-            $generatorCommon.isDefinedAndNotEmpty(domain.fields);
-    });
-
-    if ($generatorCommon.isDefinedAndNotEmpty(domainConfigs)) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="queryEntities">');
-        res.startBlock('<list>');
-
-        _.forEach(domainConfigs, function(domain) {
-            $generatorSpring.cacheQueryMetadata(domain, res);
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorSpring.cache = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    res.startBlock('<bean class="org.apache.ignite.configuration.CacheConfiguration">');
-
-    $generatorSpring.cacheConfiguration(cache, res);
-
-    res.endBlock('</bean>');
-
-    return res;
-};
-
-// Generate cache configs.
-$generatorSpring.cacheConfiguration = function(cache, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.cacheGeneral(cache, res);
-    $generatorSpring.cacheMemory(cache, res);
-    $generatorSpring.cacheQuery(cache, cache.domains, res);
-    $generatorSpring.cacheStore(cache, cache.domains, res);
-
-    const igfs = _.get(cache, 'nodeFilter.IGFS.instance');
-
-    $generatorSpring.cacheNodeFilter(cache, igfs ? [igfs] : [], res);
-    $generatorSpring.cacheConcurrency(cache, res);
-    $generatorSpring.cacheRebalance(cache, res);
-    $generatorSpring.cacheServerNearCache(cache, res);
-    $generatorSpring.cacheStatistics(cache, res);
-    $generatorSpring.cacheDomains(cache.domains, res);
-
-    return res;
-};
-
-// Generate caches configs.
-$generatorSpring.clusterCaches = function(caches, igfss, isSrvCfg, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(caches) || (isSrvCfg && $generatorCommon.isDefinedAndNotEmpty(igfss))) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="cacheConfiguration">');
-        res.startBlock('<list>');
-
-        _.forEach(caches, function(cache) {
-            $generatorSpring.cache(cache, res);
-
-            res.needEmptyLine = true;
-        });
-
-        if (isSrvCfg) {
-            _.forEach(igfss, (igfs) => {
-                $generatorSpring.cache($generatorCommon.igfsDataCache(igfs), res);
-
-                res.needEmptyLine = true;
-
-                $generatorSpring.cache($generatorCommon.igfsMetaCache(igfs), res);
-
-                res.needEmptyLine = true;
-            });
-        }
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFSs configs.
-$generatorSpring.igfss = function(igfss, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfss)) {
-        res.emptyLineIfNeeded();
-
-        res.startBlock('<property name="fileSystemConfiguration">');
-        res.startBlock('<list>');
-
-        _.forEach(igfss, function(igfs) {
-            res.startBlock('<bean class="org.apache.ignite.configuration.FileSystemConfiguration">');
-
-            $generatorSpring.igfsGeneral(igfs, res);
-            $generatorSpring.igfsIPC(igfs, res);
-            $generatorSpring.igfsFragmentizer(igfs, res);
-            $generatorSpring.igfsDualMode(igfs, res);
-            $generatorSpring.igfsSecondFS(igfs, res);
-            $generatorSpring.igfsMisc(igfs, res);
-
-            res.endBlock('</bean>');
-
-            res.needEmptyLine = true;
-        });
-
-        res.endBlock('</list>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS IPC configuration.
-$generatorSpring.igfsIPC = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.ipcEndpointEnabled) {
-        $generatorSpring.beanProperty(res, igfs.ipcEndpointConfiguration, 'ipcEndpointConfiguration', $generatorCommon.IGFS_IPC_CONFIGURATION, true);
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS fragmentizer configuration.
-$generatorSpring.igfsFragmentizer = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.fragmentizerEnabled) {
-        $generatorSpring.property(res, igfs, 'fragmentizerConcurrentFiles', null, 0);
-        $generatorSpring.property(res, igfs, 'fragmentizerThrottlingBlockLength', null, 16777216);
-        $generatorSpring.property(res, igfs, 'fragmentizerThrottlingDelay', null, 200);
-
-        res.needEmptyLine = true;
-    }
-    else
-        $generatorSpring.property(res, igfs, 'fragmentizerEnabled');
-
-    return res;
-};
-
-// Generate IGFS dual mode configuration.
-$generatorSpring.igfsDualMode = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, igfs, 'dualModeMaxPendingPutsSize', null, 0);
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfs.dualModePutExecutorService)) {
-        res.startBlock('<property name="dualModePutExecutorService">');
-        res.line('<bean class="' + igfs.dualModePutExecutorService + '"/>');
-        res.endBlock('</property>');
-    }
-
-    $generatorSpring.property(res, igfs, 'dualModePutExecutorServiceShutdown', null, false);
-
-    res.needEmptyLine = true;
-
-    return res;
-};
-
-$generatorSpring.igfsSecondFS = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (igfs.secondaryFileSystemEnabled) {
-        const secondFs = igfs.secondaryFileSystem || {};
-
-        res.startBlock('<property name="secondaryFileSystem">');
-
-        res.startBlock('<bean class="org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem">');
-
-        const nameDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.userName);
-        const cfgDefined = $generatorCommon.isDefinedAndNotEmpty(secondFs.cfgPath);
-
-        $generatorSpring.constructorArg(res, 0, secondFs, 'uri');
-
-        if (cfgDefined || nameDefined)
-            $generatorSpring.constructorArg(res, 1, secondFs, 'cfgPath');
-
-        $generatorSpring.constructorArg(res, 2, secondFs, 'userName', null, true);
-
-        res.endBlock('</bean>');
-        res.endBlock('</property>');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS general configuration.
-$generatorSpring.igfsGeneral = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if ($generatorCommon.isDefinedAndNotEmpty(igfs.name)) {
-        igfs.dataCacheName = $generatorCommon.igfsDataCache(igfs).name;
-        igfs.metaCacheName = $generatorCommon.igfsMetaCache(igfs).name;
-
-        $generatorSpring.property(res, igfs, 'name');
-        $generatorSpring.property(res, igfs, 'dataCacheName');
-        $generatorSpring.property(res, igfs, 'metaCacheName');
-        $generatorSpring.property(res, igfs, 'defaultMode', null, 'DUAL_ASYNC');
-
-        res.needEmptyLine = true;
-    }
-
-    return res;
-};
-
-// Generate IGFS misc configuration.
-$generatorSpring.igfsMisc = function(igfs, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    $generatorSpring.property(res, igfs, 'blockSize', null, 65536);
-    $generatorSpring.property(res, igfs, 'streamBufferSize', null, 65536);
-    $generatorSpring.property(res, igfs, 'maxSpaceSize', null, 0);
-    $generatorSpring.property(res, igfs, 'maximumTaskRangeLength', null, 0);
-    $generatorSpring.property(res, igfs, 'managementPort', null, 11400);
-    $generatorSpring.property(res, igfs, 'perNodeBatchSize', null, 100);
-    $generatorSpring.property(res, igfs, 'perNodeParallelBatchCount', null, 8);
-    $generatorSpring.property(res, igfs, 'prefetchBlocks', null, 0);
-    $generatorSpring.property(res, igfs, 'sequentialReadsBeforePrefetch', null, 0);
-    $generatorSpring.property(res, igfs, 'trashPurgeTimeout', null, 1000);
-    $generatorSpring.property(res, igfs, 'colocateMetadata', null, true);
-    $generatorSpring.property(res, igfs, 'relaxedConsistency', null, true);
-
-    res.softEmptyLine();
-
-    if (igfs.pathModes && igfs.pathModes.length > 0) {
-        res.startBlock('<property name="pathModes">');
-        res.startBlock('<map>');
-
-        _.forEach(igfs.pathModes, function(pair) {
-            res.line('<entry key="' + pair.path + '" value="' + pair.mode + '"/>');
-        });
-
-        res.endBlock('</map>');
-        res.endBlock('</property>');
-    }
-
-    return res;
-};
-
-// Generate DataSource beans.
-$generatorSpring.generateDataSources = function(datasources, res) {
-    if (!res)
-        res = $generatorCommon.builder();
-
-    if (datasources.length > 0) {
-        res.line('<!-- Data source beans will be initialized from external properties file. -->');
-
-        _.forEach(datasources, (datasource) => $generatorSpring.generateDataSource(datasource, res));
-
-        res.needEmptyLine = true;
-
-        res.emptyLineIfNeeded();
-    }
-
-    return res;
-};
-
-$generatorSpring.generateDataSource = function(datasource, res) {
-    const beanId = datasource.dataSourceBean;
-
-    res.startBlock('<bean id="' + beanId + '" class="' + $generatorCommon.DATA_SOURCES[datasource.dialect] + '">');
-
-    switch (datasource.dialect) {
-        case 'Generic':
-            res.line('<property name="jdbcUrl" value="${' + beanId + '.jdbc.url}"/>');
-
-            break;
-
-        case 'DB2':
-            res.line('<property name="serverName" value="${' + beanId + '.jdbc.server_name}"/>');
-            res.line('<property name="portNumber" value="${' + beanId + '.jdbc.port_number}"/>');
-            res.line('<property name="databaseName" value="${' + beanId + '.jdbc.database_name}"/>');
-            res.line('<property name="driverType" value="${' + beanId + '.jdbc.driver_type}"/>');
-
-            break;
-
-        case 'PostgreSQL':
-            res.line('<property name="url" value="${' + beanId + '.jdbc.url}"/>');
-
-            break;
-
-        default:
-            res.line('<property name="URL" value="${' + beanId + '.jdbc.url}"/>');
-    }
-
-    res.line('<property name="user" value="${' + beanId + '.jdbc.username}"/>');
-    res.line('<property name="password" value="${' + beanId + '.jdbc.password}"/>');
-
-    res.endBlock('</bean>');
-
-    res.needEmptyLine = true;
-
-    res.emptyLineIfNeeded();
-};
-
-$generatorSpring.clusterConfiguration = function(cluster, clientNearCfg, res) {
-    const isSrvCfg = _.isNil(clientNearCfg);
-
-    if (!isSrvCfg) {
-        res.line('<property name="clientMode" value="true"/>');
-
-        res.needEmptyLine = true;
-    }
-
-    $generatorSpring.clusterGeneral(cluster, res);
-
-    $generatorSpring.clusterAtomics(cluster.atomicConfiguration, res);
-
-    $generatorSpring.clusterBinary(cluster.binaryConfiguration, res);
-
-    $generatorSpring.clusterCacheKeyConfiguration(cluster.cacheKeyConfiguration, res);
-
-    $generatorSpring.clusterCollision(cluster.collision, res);
-
-    $generatorSpring.clusterCommunication(cluster, res);
-
-    $generatorSpring.clusterConnector(cluster.connector, res);
-
-    $generatorSpring.clusterDeployment(cluster, res);
-
-    $generatorSpring.clusterEvents(cluster, res);
-
-    $generatorSpring.clusterFailover(cluster, res);
-
-    $generatorSpring.clusterLogger(cluster.logger, res);
-
-    $generatorSpring.clusterODBC(cluster.odbc, res);
-
-    $generatorSpring.clusterMarshaller(cluster, res);
-
-    $generatorSpring.clusterMetrics(cluster, res);
-
-    $generatorSpring.clusterSwap(cluster, res);
-
-    $generatorSpring.clusterTime(cluster, res);
-
-    $generatorSpring.clusterPools(cluster, res);
-
-    $generatorSpring.clusterTransactions(cluster.transactionConfiguration, res);
-
-    $generatorSpring.clusterCaches(cluster.caches, cluster.igfss, isSrvCfg, res);
-
-    $generatorSpring.clusterSsl(cluster, res);
-
-    if (isSrvCfg)
-        $generatorSpring.igfss(cluster.igfss, res);
-
-    $generatorSpring.clusterUserAttributes(cluster, res);
-
-    return res;
-};
-
-$generatorSpring.cluster = function(cluster, clientNearCfg) {
-    if (cluster) {
-        const res = $generatorCommon.builder(1);
-
-        if (clientNearCfg) {
-            res.startBlock('<bean id="nearCacheBean" class="org.apache.ignite.configuration.NearCacheConfiguration">');
-
-            if (clientNearCfg.nearStartSize)
-                $generatorSpring.property(res, clientNearCfg, 'nearStartSize');
-
-            if (clientNearCfg.nearEvictionPolicy && clientNearCfg.nearEvictionPolicy.kind)
-                $generatorSpring.evictionPolicy(res, clientNearCfg.nearEvictionPolicy, 'nearEvictionPolicy');
-
-            res.endBlock('</bean>');
-
-            res.needEmptyLine = true;
-
-            res.emptyLineIfNeeded();
-        }
-
-        // Generate Ignite Configuration.
-        res.startBlock('<bean class="org.apache.ignite.configuration.IgniteConfiguration">');
-
-        $generatorSpring.clusterConfiguration(cluster, clientNearCfg, res);
-
-        res.endBlock('</bean>');
-
-        // Build final XML:
-        // 1. Add header.
-        let xml = '<?xml version="1.0" encoding="UTF-8"?>\n\n';
-
-        xml += '<!-- ' + $generatorCommon.mainComment('configuration') + ' -->\n\n';
-        xml += '<beans xmlns="http://www.springframework.org/schema/beans"\n';
-        xml += '       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"\n';
-        xml += '       xmlns:util="http://www.springframework.org/schema/util"\n';
-        xml += '       xsi:schemaLocation="http://www.springframework.org/schema/beans\n';
-        xml += '                           http://www.springframework.org/schema/beans/spring-beans.xsd\n';
-        xml += '                           http://www.springframework.org/schema/util\n';
-        xml += '                           http://www.springframework.org/schema/util/spring-util.xsd">\n';
-
-        // 2. Add external property file
-        if ($generatorCommon.secretPropertiesNeeded(cluster)) {
-            xml += '    <!-- Load external properties file. -->\n';
-            xml += '    <bean id="placeholderConfig" class="org.springframework.beans.factory.config.PropertyPlaceholderConfigurer">\n';
-            xml += '        <property name="location" value="classpath:secret.properties"/>\n';
-            xml += '    </bean>\n\n';
-        }
-
-        // 3. Add data sources.
-        xml += $generatorSpring.generateDataSources(res.datasources, $generatorCommon.builder(1)).asString();
-
-        // 3. Add main content.
-        xml += res.asString();
-
-        // 4. Add footer.
-        xml += '\n</beans>';
-
-        return xml;
-    }
-
-    return '';
-};
-
-export default $generatorSpring;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.data.js b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
index f66faba..3f98bed 100644
--- a/modules/web-console/frontend/app/modules/sql/Notebook.data.js
+++ b/modules/web-console/frontend/app/modules/sql/Notebook.data.js
@@ -21,7 +21,8 @@ const DEMO_NOTEBOOK = {
         {
             name: 'Query with refresh rate',
             cacheName: 'CarCache',
-            pageSize: 50,
+            pageSize: 100,
+            limit: 0,
             query: [
                 'SELECT count(*)',
                 'FROM "CarCache".Car'
@@ -37,7 +38,8 @@ const DEMO_NOTEBOOK = {
         {
             name: 'Simple query',
             cacheName: 'CarCache',
-            pageSize: 50,
+            pageSize: 100,
+            limit: 0,
             query: 'SELECT * FROM "CarCache".Car',
             result: 'table',
             timeLineSpan: '1',
@@ -49,8 +51,9 @@ const DEMO_NOTEBOOK = {
         },
         {
             name: 'Query with aggregates',
-            cacheName: 'CarCache',
-            pageSize: 50,
+            cacheName: 'ParkingCache',
+            pageSize: 100,
+            limit: 0,
             query: [
                 'SELECT p.name, count(*) AS cnt',
                 'FROM "ParkingCache".Parking p',

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/Notebook.service.js b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
index 12730be..f32d26f 100644
--- a/modules/web-console/frontend/app/modules/sql/Notebook.service.js
+++ b/modules/web-console/frontend/app/modules/sql/Notebook.service.js
@@ -60,7 +60,7 @@ export default class Notebook {
     }
 
     remove(notebook) {
-        return this.confirmModal.confirm(`Are you sure you want to remove: "${notebook.name}"?`)
+        return this.confirmModal.confirm(`Are you sure you want to remove notebook: "${notebook.name}"?`)
             .then(() => this.NotebookData.findIndex(notebook))
             .then((idx) => {
                 this.NotebookData.remove(notebook)

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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
deleted file mode 100644
index 0396727..0000000
--- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.jade
+++ /dev/null
@@ -1,39 +0,0 @@
-//-
-    Licensed to the Apache Software Foundation (ASF) under one or more
-    contributor license agreements.  See the NOTICE file distributed with
-    this work for additional information regarding copyright ownership.
-    The ASF licenses this file to You under the Apache License, Version 2.0
-    (the "License"); you may not use this file except in compliance with
-    the License.  You may obtain a copy of the License at
-
-         http://www.apache.org/licenses/LICENSE-2.0
-
-    Unless required by applicable law or agreed to in writing, software
-    distributed under the License is distributed on an "AS IS" BASIS,
-    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-    See the License for the specific language governing permissions and
-    limitations under the License.
-
-include /app/helpers/jade/mixins.jade
-
-.modal(tabindex='-1' role='dialog')
-    .modal-dialog
-        .modal-content
-            .modal-header
-                button.close(ng-click='$hide()') &times;
-                h4.modal-title Scan filter
-            form.form-horizontal.modal-body.row(name='ui.inputForm' novalidate)
-                .settings-row
-                    .col-sm-2
-                        label.required.labelFormField Filter:&nbsp;
-                    .col-sm-10
-                        .input-tip
-                            +ignite-form-field-input('"filter"', 'ui.filter', false, 'true', 'Enter filter')(
-                                data-ignite-form-field-input-autofocus='true'
-                                ignite-on-enter='form.$valid && ok()'
-                            )
-                .settings-row
-                    +checkbox('Case sensitive search', 'ui.caseSensitive', '"caseSensitive"', 'Select this checkbox for case sensitive search')
-            .modal-footer
-                button.btn.btn-default(id='btn-cancel' ng-click='$hide()') Cancel
-                button.btn.btn-primary(id='btn-scan' ng-disabled='ui.inputForm.$invalid' ng-click='ok()') Scan

http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js b/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
deleted file mode 100644
index 18ba3ba..0000000
--- a/modules/web-console/frontend/app/modules/sql/scan-filter-input.service.js
+++ /dev/null
@@ -1,51 +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.
- */
-
-export default class ScanFilter {
-    static $inject = ['$rootScope', '$q', '$modal'];
-
-    constructor($root, $q, $modal) {
-        this.deferred = null;
-        this.$q = $q;
-
-        const scope = $root.$new();
-
-        scope.ui = {};
-
-        scope.ok = () => {
-            this.deferred.resolve({filter: scope.ui.filter, caseSensitive: !!scope.ui.caseSensitive});
-
-            this.modal.hide();
-        };
-
-        scope.$hide = () => {
-            this.modal.hide();
-
-            this.deferred.reject();
-        };
-
-        this.modal = $modal({templateUrl: '/scan-filter-input.html', scope, placement: 'center', show: false});
-    }
-
-    open() {
-        this.deferred = this.$q.defer();
-
-        this.modal.$promise.then(this.modal.show);
-
-        return this.deferred.promise;
-    }
-}


[50/50] [abbrv] ignite git commit: Merge branch 'master' into ignite-2.0

Posted by vo...@apache.org.
Merge branch 'master' into ignite-2.0


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

Branch: refs/heads/ignite-2.0
Commit: c5882a85f4e3a1f61723ac54fd92f087684df6da
Parents: c893da7 c040c37
Author: devozerov <vo...@gridgain.com>
Authored: Mon Dec 26 14:15:42 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Dec 26 14:15:42 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |    6 +
 .../store/cassandra/datasource/DataSource.java  |    9 +
 .../java/org/apache/ignite/IgniteLogger.java    |    6 +-
 .../java/org/apache/ignite/IgniteServices.java  |   16 +
 .../ignite/cache/store/CacheStoreAdapter.java   |    6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java |  112 +-
 .../org/apache/ignite/internal/GridTopic.java   |    8 +-
 .../ignite/internal/IgniteServicesImpl.java     |    9 +-
 .../internal/binary/BinaryClassDescriptor.java  |   12 +-
 .../ignite/internal/binary/BinaryUtils.java     |   10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |   11 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../cache/CacheStoreBalancingWrapper.java       |    6 +
 .../processors/cache/GridCacheAdapter.java      |    8 +-
 .../cache/GridCacheLoaderWriterStore.java       |    6 +
 .../GridCachePartitionExchangeManager.java      |   23 +-
 .../CacheDataStructuresManager.java             |    6 +-
 .../GridDhtPartitionsExchangeFuture.java        |    6 +-
 .../cache/query/GridCacheQueryManager.java      |    4 +-
 .../closure/GridClosureProcessor.java           |   31 +-
 .../internal/processors/job/GridJobWorker.java  |   76 +-
 .../processors/odbc/OdbcRequestHandler.java     |   16 +-
 .../internal/processors/odbc/OdbcTableMeta.java |   15 +-
 .../platform/PlatformContextImpl.java           |    2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |   11 +
 .../platform/services/PlatformServices.java     |    2 +-
 .../platform/utils/PlatformUtils.java           |   28 +
 .../service/GridServiceProcessor.java           |   11 +-
 .../processors/service/GridServiceProxy.java    |   18 +-
 .../processors/task/GridTaskWorker.java         |    7 +
 .../visor/service/VisorCancelServiceTask.java   |   70 +
 .../visor/service/VisorServiceDescriptor.java   |  132 +
 .../visor/service/VisorServiceTask.java         |   75 +
 .../internal/visor/util/VisorTaskUtils.java     |   15 +-
 .../apache/ignite/logger/java/JavaLogger.java   |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    4 +-
 .../apache/ignite/util/AttributeNodeFilter.java |  108 +
 .../resources/META-INF/classnames.properties    |   65 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   |  283 ++
 .../IgniteComputeTopologyExceptionTest.java     |    5 +-
 .../binary/BinaryMarshallerSelfTest.java        |   66 +
 ...heapCacheMetricsForClusterGroupSelfTest.java |  141 +
 .../CacheLockReleaseNodeLeaveTest.java          |  135 +
 .../CacheOffHeapAndSwapMetricsSelfTest.java     |  621 ---
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |  621 +++
 .../closure/GridClosureSerializationTest.java   |  177 +
 ...gniteServiceProxyTimeoutInitializedTest.java |  284 ++
 .../junits/logger/GridTestLog4jLogger.java      |    4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    5 +
 .../IgniteCacheMetricsSelfTestSuite.java        |    6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    2 +
 .../util/AttributeNodeFilterSelfTest.java       |  184 +
 .../processors/query/h2/IgniteH2Indexing.java   |   75 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |   25 +
 .../apache/ignite/logger/log4j/Log4JLogger.java |    4 +-
 .../cache/query/continuous/continuous_query.h   |    2 +-
 .../cpp/odbc-test/config/queries-default.xml    |  145 +
 .../odbc-test/config/queries-test-noodbc.xml    |   84 +-
 .../cpp/odbc-test/config/queries-test.xml       |  122 +-
 .../cpp/odbc-test/src/queries_test.cpp          |   76 +
 .../cpp/odbc-test/src/utility_test.cpp          |   27 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   10 +
 .../Cache/Query/CacheLinqTest.cs                |   51 +-
 .../EntityFrameworkCacheTest.cs                 |   54 +-
 .../Impl/DbCommandInfo.cs                       |   21 +-
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |    8 +
 modules/platforms/dotnet/Apache.Ignite.sln      |    2 +
 modules/platforms/dotnet/DEVNOTES.txt           |   12 +-
 modules/platforms/dotnet/build.bat              |   19 +
 modules/platforms/dotnet/build.ps1              |  211 +
 .../spi/deployment/uri/UriDeploymentSpi.java    |    2 +-
 modules/web-console/backend/app/agent.js        |   15 +
 modules/web-console/backend/app/browser.js      |   13 +
 modules/web-console/backend/app/mongo.js        |   24 +-
 modules/web-console/backend/routes/demo.js      |   17 +-
 modules/web-console/backend/routes/profile.js   |    3 +-
 .../web-console/backend/services/notebooks.js   |   14 +-
 .../web-console/backend/services/sessions.js    |    6 +-
 modules/web-console/backend/services/spaces.js  |   15 +
 modules/web-console/frontend/app/app.js         |    5 -
 .../controllers/reset-password.controller.js    |   14 +-
 .../frontend/app/data/event-groups.json         |  169 +
 .../frontend/app/data/event-types.json          |  169 -
 .../frontend/app/data/pom-dependencies.json     |   12 +-
 .../ui-ace-docker/ui-ace-docker.controller.js   |    2 +-
 .../directives/ui-ace-docker/ui-ace-docker.jade |    2 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |   12 +-
 .../ui-ace-pom/ui-ace-pom.controller.js         |    4 +-
 .../helpers/jade/form/form-field-dropdown.jade  |    5 +-
 .../helpers/jade/form/form-field-number.jade    |    3 +-
 .../app/helpers/jade/form/form-field-text.jade  |   19 +-
 .../frontend/app/helpers/jade/mixins.jade       |   52 +-
 .../frontend/app/modules/Demo/Demo.module.js    |    6 +-
 .../configuration/EventGroups.provider.js       |   30 -
 .../modules/configuration/Version.service.js    |    6 +-
 .../configuration/configuration.module.js       |   63 +-
 .../generator/AbstractTransformer.js            |   17 +
 .../modules/configuration/generator/Beans.js    |    5 +
 .../generator/ConfigurationGenerator.js         | 2795 +++++++-------
 .../configuration/generator/Custom.service.js   |   23 +
 .../configuration/generator/Docker.service.js   |    4 +-
 .../generator/JavaTransformer.service.js        | 2318 +++++------
 .../configuration/generator/Maven.service.js    |  234 ++
 .../configuration/generator/Pom.service.js      |  233 --
 .../generator/Properties.service.js             |    2 +-
 .../configuration/generator/Readme.service.js   |    2 +-
 .../generator/SharpTransformer.service.js       |  437 ++-
 .../generator/SpringTransformer.service.js      |  497 ++-
 .../defaults/Cache.platform.service.js          |   56 +
 .../generator/defaults/Cache.service.js         |  131 +
 .../defaults/Cluster.platform.service.js        |   43 +
 .../generator/defaults/Cluster.service.js       |  289 ++
 .../generator/defaults/Event-groups.service.js  |   27 +
 .../generator/defaults/IGFS.service.js          |   64 +
 .../defaults/cache.platform.provider.js         |   60 -
 .../generator/defaults/cache.provider.js        |  137 -
 .../defaults/cluster.platform.provider.js       |   49 -
 .../generator/defaults/cluster.provider.js      |  293 --
 .../generator/defaults/igfs.provider.js         |   68 -
 .../configuration/generator/generator-common.js |  625 ---
 .../configuration/generator/generator-java.js   | 3617 ------------------
 .../generator/generator-optional.js             |   25 -
 .../configuration/generator/generator-spring.js | 2111 ----------
 .../frontend/app/modules/sql/Notebook.data.js   |   11 +-
 .../app/modules/sql/Notebook.service.js         |    2 +-
 .../app/modules/sql/scan-filter-input.jade      |   39 -
 .../modules/sql/scan-filter-input.service.js    |   51 -
 .../frontend/app/modules/sql/sql.controller.js  |  211 +-
 .../frontend/app/modules/sql/sql.module.js      |    2 -
 .../app/modules/states/configuration.state.js   |    2 +
 .../configuration/caches/node-filter.jade       |    2 +-
 .../states/configuration/caches/query.jade      |    3 +
 .../states/configuration/caches/store.jade      |    4 +-
 .../configuration/clusters/checkpoint.jade      |   11 +-
 .../configuration/clusters/checkpoint/fs.jade   |    8 +-
 .../configuration/clusters/checkpoint/jdbc.jade |    8 +-
 .../configuration/clusters/checkpoint/s3.jade   |   25 +-
 .../clusters/collision/custom.jade              |    2 +-
 .../clusters/collision/job-stealing.jade        |    2 +-
 .../configuration/clusters/deployment.jade      |  129 +-
 .../states/configuration/clusters/events.jade   |    4 +-
 .../states/configuration/clusters/failover.jade |    4 +-
 .../clusters/general/discovery/zookeeper.jade   |    2 +-
 .../discovery/zookeeper/retrypolicy/custom.jade |    2 +-
 .../configuration/clusters/load-balancing.jade  |   23 +-
 .../configuration/clusters/logger/custom.jade   |    2 +-
 .../states/configuration/clusters/ssl.jade      |    2 +-
 .../summary/summary-zipper.service.js           |   37 +
 .../configuration/summary/summary.controller.js |  103 +-
 .../configuration/summary/summary.worker.js     |  123 +
 .../frontend/app/modules/user/Auth.service.js   |   11 +-
 .../frontend/app/services/JavaTypes.service.js  |   13 +-
 .../frontend/app/services/Messages.service.js   |   17 +-
 .../frontend/controllers/admin-controller.js    |  211 +-
 .../frontend/controllers/caches-controller.js   |   22 +-
 .../frontend/controllers/clusters-controller.js |   42 +-
 .../frontend/controllers/domains-controller.js  |   32 +-
 .../frontend/controllers/igfs-controller.js     |   20 +-
 .../frontend/controllers/profile-controller.js  |    3 +-
 .../gulpfile.babel.js/webpack/common.js         |   17 +-
 .../webpack/environments/development.js         |   14 +-
 .../webpack/environments/production.js          |    3 +-
 .../webpack/plugins/progress.js                 |   82 -
 modules/web-console/frontend/package.json       |  178 +-
 .../frontend/public/images/cache.png            |  Bin 23700 -> 24791 bytes
 .../frontend/public/images/domains.png          |  Bin 23828 -> 22131 bytes
 .../web-console/frontend/public/images/igfs.png |  Bin 14307 -> 14139 bytes
 .../frontend/public/images/query-chart.png      |  Bin 16637 -> 17142 bytes
 .../frontend/public/images/query-metadata.png   |  Bin 32298 -> 39361 bytes
 .../frontend/public/images/query-table.png      |  Bin 29189 -> 28065 bytes
 .../frontend/public/images/summary.png          |  Bin 31997 -> 33650 bytes
 .../stylesheets/_font-awesome-custom.scss       |   23 +-
 .../frontend/public/stylesheets/form-field.scss |   37 +
 .../frontend/public/stylesheets/style.scss      |  111 +-
 .../frontend/test/unit/JavaTypes.test.js        |   17 +-
 .../frontend/test/unit/Version.test.js          |    8 +-
 .../views/configuration/domains-import.jade     |    5 +-
 .../frontend/views/configuration/summary.jade   |   25 +-
 .../frontend/views/settings/admin.jade          |   85 +-
 .../frontend/views/sql/notebook-new.jade        |    2 +-
 modules/web-console/frontend/views/sql/sql.jade |  235 +-
 .../frontend/views/templates/alert.jade         |    2 +-
 .../frontend/views/templates/select.jade        |    2 +-
 185 files changed, 9012 insertions(+), 12322 deletions(-)
----------------------------------------------------------------------



[05/50] [abbrv] ignite git commit: IGNITE-4277: Hadoop: better property naming for "partially raw" comparator.

Posted by vo...@apache.org.
IGNITE-4277: Hadoop: better property naming for "partially raw" comparator.


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

Branch: refs/heads/ignite-2.0
Commit: f8ac0f14986169ee25159734d0a97b08976c5751
Parents: c1ddf21
Author: devozerov <vo...@gridgain.com>
Authored: Thu Dec 15 12:13:10 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 15 13:46:46 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/hadoop/HadoopJobProperty.java     | 4 ++--
 .../internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java  | 2 +-
 .../internal/processors/hadoop/impl/HadoopTeraSortTest.java      | 2 +-
 3 files changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f8ac0f14/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index 4122eef..9dd430b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -64,9 +64,9 @@ public enum HadoopJobProperty {
     JOB_SHARED_CLASSLOADER("ignite.job.shared.classloader"),
 
     /**
-     * Fully qualified name of partially-raw comparator which should be used on sorting phase.
+     * Fully qualified name of partially raw comparator which should be used on sorting phase.
      */
-    JOB_PARTIAL_RAW_COMPARATOR("ignite.job.partial.raw.comparator"),
+    JOB_PARTIALLY_RAW_COMPARATOR("ignite.job.partially.raw.comparator"),
 
     /**
      * Size in bytes of single memory page which will be allocated for data structures in shuffle.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8ac0f14/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
index 42bbec5..e9cae1c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2TaskContext.java
@@ -433,7 +433,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public PartiallyOffheapRawComparatorEx<Object> partialRawSortComparator() {
-        Class cls = jobCtx.getJobConf().getClass(HadoopJobProperty.JOB_PARTIAL_RAW_COMPARATOR.propertyName(), null);
+        Class cls = jobCtx.getJobConf().getClass(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(), null);
 
         if (cls == null)
             return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/f8ac0f14/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
index a016506..b1fa91f 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopTeraSortTest.java
@@ -165,7 +165,7 @@ public class HadoopTeraSortTest extends HadoopAbstractSelfTest {
 
         jobConf.setBoolean(HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT.propertyName(), true);
 
-        jobConf.set(HadoopJobProperty.JOB_PARTIAL_RAW_COMPARATOR.propertyName(),
+        jobConf.set(HadoopJobProperty.JOB_PARTIALLY_RAW_COMPARATOR.propertyName(),
             TextPartiallyRawComparator.class.getName());
 
         Job job = setupConfig(jobConf);


[08/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 482e2ef..c7a1a53 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import org.apache.ignite.internal.util.nio.GridNioServer;
 import org.apache.ignite.mxbean.MXBeanDescription;
 import org.apache.ignite.spi.IgniteSpiManagementMBean;
 
@@ -44,6 +45,35 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getLocalPort();
 
     /**
+     * Returns {@code true} if {@code TcpCommunicationSpi} should
+     * maintain connection for outgoing and incoming messages separately.
+     * In this case total number of connections between local and some remote node
+     * is {@link #getConnectionsPerNode()} * 2.
+     * <p>
+     * Returns {@code false} if each connection of {@link #getConnectionsPerNode()}
+     * should be used for outgoing and incoming messages. In this case load NIO selectors load
+     * balancing of {@link GridNioServer} will be disabled.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @return {@code true} to use paired connections and {@code false} otherwise.
+     * @see #getConnectionsPerNode()
+     */
+    @MXBeanDescription("Paired connections used.")
+    public boolean isUsePairedConnections();
+
+    /**
+     * Gets number of connections to each remote node. if {@link #isUsePairedConnections()}
+     * is {@code true} then number of connections is doubled and half is used for incoming and
+     * half for outgoing messages.
+     *
+     * @return Number of connections per node.
+     * @see #isUsePairedConnections()
+     */
+    @MXBeanDescription("Connections per node.")
+    public int getConnectionsPerNode();
+
+    /**
      * Gets local port for shared memory communication.
      *
      * @return Port number.
@@ -153,6 +183,16 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getReconnectCount();
 
     /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     *
+     * @return Selector thread busy-loop iterations.
+     */
+    @MXBeanDescription("Selector thread busy-loop iterations.")
+    public long getSelectorSpins();
+
+    /**
      * Gets value for {@code TCP_NODELAY} socket option.
      *
      * @return {@code True} if TCP delay is disabled.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 9a36f1a..8a9f1c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3408,7 +3408,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     if (log.isDebugEnabled())
                         log.debug("Node validation failed [res=" + err + ", node=" + node + ']');
 
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 boolean ping = node.id().equals(err.nodeId()) ? pingNode(node) : pingNode(err.nodeId());
@@ -3453,7 +3453,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final String rmtMarsh = node.attribute(ATTR_MARSHALLER);
 
                 if (!F.eq(locMarsh, rmtMarsh)) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's marshaller differs from remote node's marshaller " +
@@ -3510,7 +3510,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 boolean locLateAssignBool = locLateAssign != null ? locLateAssign : false;
 
                 if (locMarshUseDfltSuidBool != rmtMarshUseDfltSuidBool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's " + IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID +
@@ -3552,7 +3552,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final boolean rmtMarshCompactFooterBool = rmtMarshCompactFooter != null ? rmtMarshCompactFooter : false;
 
                 if (locMarshCompactFooterBool != rmtMarshCompactFooterBool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's binary marshaller \"compactFooter\" property differs from " +
@@ -3590,7 +3590,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final boolean rmtMarshStrSerialVer2Bool = rmtMarshStrSerialVer2 != null ? rmtMarshStrSerialVer2 : false;
 
                 if (locMarshStrSerialVer2Bool != rmtMarshStrSerialVer2Bool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's " + IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2 +
@@ -3663,7 +3663,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     final Boolean rmtSrvcCompatibilityEnabled = node.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
 
                     if (!F.eq(locSrvcCompatibilityEnabled, rmtSrvcCompatibilityEnabled)) {
-                        utilityPool.submit(
+                        utilityPool.execute(
                             new Runnable() {
                                 @Override public void run() {
                                     String errMsg = "Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
@@ -3698,7 +3698,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
                 else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Remote node doesn't support lazy services configuration and " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
index d1c8d19..127778b 100644
--- a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
@@ -184,6 +184,7 @@ public class SocketStreamer<T, K, V> extends StreamAdapter<T, K, V> {
         try {
             srv = new GridNioServer.Builder<byte[]>()
                 .address(addr == null ? InetAddress.getLocalHost() : addr)
+                .serverName("sock-streamer")
                 .port(port)
                 .listener(lsnr)
                 .logger(log)

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
index 55557dd..d173594 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
@@ -20,6 +20,7 @@ package org.apache.ignite.thread;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -62,4 +63,9 @@ public class IgniteThreadFactory implements ThreadFactory {
     @Override public Thread newThread(@NotNull Runnable r) {
         return new IgniteThread(gridName, threadName, r, idxGen.incrementAndGet());
     }
-}
\ No newline at end of file
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteThreadFactory.class, this, super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 760313b..5721887 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -75,6 +75,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         commSpi.setSlowClientQueueLimit(50);
         commSpi.setSharedMemoryPort(-1);
         commSpi.setIdleConnectionTimeout(300_000);
+        commSpi.setConnectionsPerNode(1);
 
         cfg.setCommunicationSpi(commSpi);
 

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
new file mode 100644
index 0000000..e142aef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private int selectors;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpCommunicationSpi commSpi = ((TcpCommunicationSpi)cfg.getCommunicationSpi());
+
+        commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(connectionsPerNode());
+
+        if (selectors > 0)
+            commSpi.setSelectorsCount(selectors);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @return Connections per node.
+     */
+    protected int connectionsPerNode() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBalance1() throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "5000");
+
+        try {
+            selectors = 4;
+
+            final int SRVS = 4;
+
+            startGridsMultiThreaded(SRVS);
+
+            client = true;
+
+            final Ignite client = startGrid(SRVS);
+
+            for (int i = 0; i < 4; i++) {
+                ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+                client.compute(client.cluster().forNode(node)).call(new DummyCallable(null));
+            }
+
+            waitNioBalanceStop(Collections.singletonList(client), 10_000);
+
+            final GridNioServer srv = GridTestUtils.getFieldValue(client.configuration().getCommunicationSpi(), "nioSrvr");
+
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            long readMoveCnt1 = srv.readerMoveCount();
+            long writeMoveCnt1 = srv.writerMoveCount();
+
+            int prevNodeIdx = -1;
+
+            for (int iter = 0; iter < 10; iter++) {
+                int nodeIdx = rnd.nextInt(SRVS);
+
+                while (prevNodeIdx == nodeIdx)
+                    nodeIdx = rnd.nextInt(SRVS);
+
+                prevNodeIdx = nodeIdx;
+
+                log.info("Iteration [iter=" + iter + ", node=" + nodeIdx + ']');
+
+                final long readMoveCnt = readMoveCnt1;
+                final long writeMoveCnt = writeMoveCnt1;
+
+                final int nodeIdx0 = nodeIdx;
+
+                GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                    @Override public boolean apply() {
+                        byte[] data = new byte[100_000];
+
+                        for (int j = 0; j < 10; j++) {
+                            for (int i = 0; i < SRVS; i++) {
+                                ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+                                IgniteCompute compute = client.compute(client.cluster().forNode(node));
+
+                                compute.call(new DummyCallable(i == nodeIdx0 ? data : null));
+                            }
+                        }
+
+                        return srv.readerMoveCount() > readMoveCnt && srv.writerMoveCount() > writeMoveCnt;
+                    }
+                }, 30_000);
+
+                waitNioBalanceStop(Collections.singletonList(client), 30_000);
+
+                long readMoveCnt2 = srv.readerMoveCount();
+                long writeMoveCnt2 = srv.writerMoveCount();
+
+                log.info("Move counts [rc1=" + readMoveCnt1 +
+                    ", wc1=" + writeMoveCnt1 +
+                    ", rc2=" + readMoveCnt2 +
+                    ", wc2=" + writeMoveCnt2 + ']');
+
+                assertTrue(readMoveCnt2 > readMoveCnt1);
+                assertTrue(writeMoveCnt2 > writeMoveCnt1);
+
+                readMoveCnt1 = readMoveCnt2;
+                writeMoveCnt1 = writeMoveCnt2;
+            }
+
+            waitNioBalanceStop(G.allGrids(), 10_000);
+        }
+        finally {
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBalance2() throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "1000");
+
+        try {
+            startGridsMultiThreaded(5);
+
+            client = true;
+
+            startGridsMultiThreaded(5, 5);
+
+            for (int i = 0; i < 5; i++) {
+                log.info("Iteration: " + i);
+
+                final AtomicInteger idx = new AtomicInteger();
+
+                GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        Ignite node = ignite(idx.incrementAndGet() % 10);
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        int msgs = rnd.nextInt(500, 600);
+
+                        for (int i = 0; i < msgs; i++) {
+                            int sndTo = rnd.nextInt(10);
+
+                            ClusterNode sntToNode = node.cluster().node(ignite(sndTo).cluster().localNode().id());
+
+                            IgniteCompute compute = node.compute(node.cluster().forNode(sntToNode));
+
+                            compute.call(new DummyCallable(new byte[rnd.nextInt(rnd.nextInt(256, 1024))]));
+                        }
+
+                        return null;
+                    }
+                }, 30, "test-thread");
+
+                waitNioBalanceStop(G.allGrids(), 10_000);
+            }
+        }
+        finally {
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     * @param nodes Node.
+     * @param timeout Timeout.
+     * @throws Exception If failed.
+     */
+    private void waitNioBalanceStop(List<Ignite> nodes, long timeout) throws Exception {
+        final List<GridNioServer> srvs = new ArrayList<>();
+
+        for (Ignite node : nodes) {
+            TcpCommunicationSpi spi = (TcpCommunicationSpi) node.configuration().getCommunicationSpi();
+
+            GridNioServer srv = GridTestUtils.getFieldValue(spi, "nioSrvr");
+
+            srvs.add(srv);
+        }
+
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicateX() {
+            @Override public boolean applyx() throws IgniteCheckedException {
+                List<Long> rCnts = new ArrayList<>();
+                List<Long> wCnts = new ArrayList<>();
+
+                for (GridNioServer srv : srvs) {
+                    long readerMovCnt1 = srv.readerMoveCount();
+                    long writerMovCnt1 = srv.writerMoveCount();
+
+                    rCnts.add(readerMovCnt1);
+                    wCnts.add(writerMovCnt1);
+                }
+
+                U.sleep(2000);
+
+                for (int i = 0; i < srvs.size(); i++) {
+                    GridNioServer srv = srvs.get(i);
+
+                    long readerMovCnt1 = rCnts.get(i);
+                    long writerMovCnt1 = wCnts.get(i);
+
+                    long readerMovCnt2 = srv.readerMoveCount();
+                    long writerMovCnt2 = srv.writerMoveCount();
+
+                    if (readerMovCnt1 != readerMovCnt2) {
+                        log.info("Readers balance is in progress [node=" + i + ", cnt1=" + readerMovCnt1 +
+                            ", cnt2=" + readerMovCnt2 + ']');
+
+                        return false;
+                    }
+                    if (writerMovCnt1 != writerMovCnt2) {
+                        log.info("Writers balance is in progress [node=" + i + ", cnt1=" + writerMovCnt1 +
+                            ", cnt2=" + writerMovCnt2 + ']');
+
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        }, timeout));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomBalance() throws Exception {
+        System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "true");
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "500");
+
+        try {
+            final int NODES = 10;
+
+            startGridsMultiThreaded(NODES);
+
+            final long stopTime = System.currentTimeMillis() + 60_000;
+
+            GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    while (System.currentTimeMillis() < stopTime)
+                        ignite(rnd.nextInt(NODES)).compute().broadcast(new DummyCallable(null));
+
+                    return null;
+                }
+            }, 20, "test-thread");
+        }
+        finally {
+            System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "");
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class DummyCallable implements IgniteCallable<Object> {
+        /** */
+        private byte[] data;
+
+        /**
+         * @param data Data.
+         */
+        DummyCallable(byte[] data) {
+            this.data = data;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            return data;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
new file mode 100644
index 0000000..b644878
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.G;
+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;
+
+/**
+ *
+ */
+public class IgniteIoTestMessagesTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(3);
+
+        client = true;
+
+        startGrid(3);
+
+        startGrid(4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIoTestMessages() throws Exception {
+        for (Ignite node : G.allGrids()) {
+            IgniteKernal ignite = (IgniteKernal)node;
+
+            List<ClusterNode> rmts = new ArrayList<>(ignite.cluster().forRemotes().nodes());
+
+            assertEquals(4, rmts.size());
+
+            for (ClusterNode rmt : rmts) {
+                ignite.sendIoTest(rmt, new byte[1024], false);
+
+                ignite.sendIoTest(rmt, new byte[1024], true);
+
+                ignite.sendIoTest(rmts, new byte[1024], false);
+
+                ignite.sendIoTest(rmts, new byte[1024], true);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
new file mode 100644
index 0000000..510751e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+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.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteVariousConnectionNumberTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 6;
+
+    /** */
+    private static Random rnd = new Random();
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        int connections = rnd.nextInt(10) + 1;
+
+        log.info("Node connections [name=" + gridName + ", connections=" + connections + ']');
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(connections);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setUsePairedConnections(rnd.nextBoolean());
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        long seed = U.currentTimeMillis();
+
+        rnd.setSeed(seed);
+
+        log.info("Random seed: " + seed);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testVariousConnectionNumber() throws Exception {
+        startGridsMultiThreaded(3);
+
+        client = true;
+
+        startGridsMultiThreaded(3, 3);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        ignite(0).createCache(ccfg);
+
+        for (int i = 0; i < 10; i++) {
+            log.info("Iteration: " + i);
+
+            runOperations(5000);
+
+            awaitPartitionMapExchange();
+
+            int idx = ThreadLocalRandom.current().nextInt(NODES);
+
+            Ignite node = ignite(idx);
+
+            client = node.configuration().isClientMode();
+
+            stopGrid(idx);
+
+            startGrid(idx);
+        }
+    }
+
+    /**
+     * @param time Execution time.
+     * @throws Exception If failed.
+     */
+    private void runOperations(final long time) throws Exception {
+        final AtomicInteger idx = new AtomicInteger();
+
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Ignite node = ignite(idx.getAndIncrement() % NODES);
+
+                IgniteCache cache = node.cache(null);
+
+                long stopTime = U.currentTimeMillis() + time;
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                while (U.currentTimeMillis() < stopTime) {
+                    cache.put(rnd.nextInt(10_000), 0);
+
+                    node.compute().broadcast(new DummyJob());
+                }
+
+                return null;
+            }
+        }, NODES * 10, "test-thread");
+    }
+
+    /**
+     *
+     */
+    private static class DummyJob implements IgniteRunnable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index 67ec371..eaa9923 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -86,6 +86,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 6 * 60 * 1000;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
@@ -170,9 +175,17 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param cacheMode Cache mode.
+     * @param writeSync Write synchronization mode.
+     * @param fairAff Fair affinity flag.
+     * @param ignite Node to use.
+     * @param name Cache name.
      */
-    protected void createCache(CacheMode cacheMode, CacheWriteSynchronizationMode writeSync, boolean fairAff,
-        Ignite ignite, String name) {
+    protected void createCache(CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSync,
+        boolean fairAff,
+        Ignite ignite,
+        String name) {
         ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff));
     }
 
@@ -269,9 +282,18 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
 
             boolean checkData = fullSync && !optimistic;
 
+            long stopTime = System.currentTimeMillis() + 10_000;
+
             for (int i = 0; i < 10_000; i++) {
-                if (i % 100 == 0)
+                if (i % 100 == 0) {
+                    if (System.currentTimeMillis() > stopTime) {
+                        log.info("Stop on timeout, iteration: " + i);
+
+                        break;
+                    }
+
                     log.info("Iteration: " + i);
+                }
 
                 boolean rollback = i % 10 == 0;
 
@@ -557,4 +579,4 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
             return old;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index 9405a19..3a2bc81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -200,7 +200,9 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
     private void testRebalance(final Operation operation) throws Exception {
         interceptor = new RebalanceUpdateInterceptor();
 
-        for (int iter = 0; iter < TEST_ITERATIONS; iter++) {
+        long stopTime = System.currentTimeMillis() + 2 * 60_000;
+
+        for (int iter = 0; iter < TEST_ITERATIONS && System.currentTimeMillis() < stopTime; iter++) {
             log.info("Iteration: " + iter);
 
             failed = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
index 9458a63..6e2e91f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
@@ -115,10 +115,10 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
         if (gridCount() > 1)
             testPutTx(keyForNode(1), PESSIMISTIC);
     }
-    
+
     /**
      * TODO: IGNITE-592.
-     *  
+     *
      * @throws Exception If failed.
      */
     public void testPutTxOptimistic() throws Exception {
@@ -227,4 +227,4 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
 
         assertFalse(failed);
     }
-}
\ No newline at end of file
+}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..71772ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi();
+
+        assertTrue(commSpi.isUsePairedConnections());
+
+        commSpi.setUsePairedConnections(false);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
new file mode 100644
index 0000000..919aea6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+/**
+ *
+ */
+public class IgniteCacheConnectionRecovery10ConnectionsTest extends IgniteCacheConnectionRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(10);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
index 2f700f3..a91de67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
@@ -107,7 +107,7 @@ public class IgniteCacheCreatePutTest extends GridCommonAbstractTest {
         try {
             int iter = 0;
 
-            while (System.currentTimeMillis() < stopTime) {
+            while (System.currentTimeMillis() < stopTime && iter < 5) {
                 log.info("Iteration: " + iter++);
 
                 try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
index 0460a8f..1bfd727 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
@@ -58,6 +58,7 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
 
         commSpi.setSocketWriteTimeout(1000);
         commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(connectionsPerNode());
 
         cfg.setCommunicationSpi(commSpi);
 
@@ -76,6 +77,13 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
     }
 
     /**
+     * @return Value for {@link TcpCommunicationSpi#setConnectionsPerNode(int)}.
+     */
+    protected int connectionsPerNode() {
+        return TcpCommunicationSpi.DFLT_CONN_PER_NODE;
+    }
+
+    /**
      * @return Cache atomicity mode.
      */
     protected abstract CacheAtomicityMode atomicityMode();
@@ -174,18 +182,22 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
     static boolean closeSessions(Ignite ignite) throws Exception {
         TcpCommunicationSpi commSpi = (TcpCommunicationSpi)ignite.configuration().getCommunicationSpi();
 
-        Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
+        Map<UUID, GridCommunicationClient[]> clients = U.field(commSpi, "clients");
 
         boolean closed = false;
 
-        for (GridCommunicationClient client : clients.values()) {
-            GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null) {
+                    GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
 
-            GridNioSession ses = client0.session();
+                    GridNioSession ses = client0.session();
 
-            ses.close();
+                    ses.close();
 
-            closed = true;
+                    closed = true;
+                }
+            }
         }
 
         return closed;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
index 6256225..0dd4079 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
@@ -50,8 +50,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
         // Try provoke connection close on socket writeTimeout.
         commSpi.setSharedMemoryPort(-1);
         commSpi.setMessageQueueLimit(10);
-        commSpi.setSocketReceiveBuffer(32);
-        commSpi.setSocketSendBuffer(32);
+        commSpi.setSocketReceiveBuffer(40);
+        commSpi.setSocketSendBuffer(40);
         commSpi.setSocketWriteTimeout(100);
         commSpi.setUnacknowledgedMessagesBufferSize(1000);
         commSpi.setConnectTimeout(10_000);
@@ -66,15 +66,20 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
         super.afterTest();
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testMessageQueueLimit() throws Exception {
-        startGridsMultiThreaded(3);
-
-        for (int i = 0; i < 15; i++) {
+        for (int i = 0; i < 3; i++) {
             log.info("Iteration: " + i);
 
+            startGridsMultiThreaded(3);
+
             IgniteInternalFuture<?> fut1 = startJobThreads(50);
 
             U.sleep(100);
@@ -83,6 +88,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
 
             fut1.get();
             fut2.get();
+
+            stopAllGrids();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 3fca826..322690c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -86,7 +86,6 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         plc.setMaxSize(100000);
 
         ccfg.setEvictionPolicy(plc);
-        ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);
 
@@ -95,6 +94,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         return c;
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
index c9d18eb..e9d74ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
@@ -211,7 +211,7 @@ public class TxDeadlockDetectionNoHangsTest extends GridCommonAbstractTest {
                             tx.commit();
                         }
                         catch (Exception e) {
-                            e.printStackTrace();
+                            log.info("Ignore error: " + e);
                         }
                     }
                 }, NODES_CNT * 3, "tx-thread");

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index aa240aa..f6a06c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -111,6 +111,9 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(client);
 
+        // Test spi blocks message send, this can cause hang with striped pool.
+        cfg.setStripedPoolSize(-1);
+
         return cfg;
     }
 
@@ -274,8 +277,8 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
                     Object k;
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + transformer.apply(key) + ']');
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+                        ", tx=" + tx.xid() + ", key=" + transformer.apply(key) + ']');
 
                     cache.put(transformer.apply(key), 0);
 
@@ -309,23 +312,27 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
                         entries.put(k, 2);
                     }
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", entries=" + entries + ']');
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+                        ", tx=" + tx.xid() + ", entries=" + entries + ']');
 
                     cache.putAll(entries);
 
                     tx.commit();
                 }
                 catch (Throwable e) {
-                    U.error(log, "Expected exception: ", e);
+                    log.info("Expected exception: " + e);
+
+                    e.printStackTrace(System.out);
 
                     // At least one stack trace should contain TransactionDeadlockException.
                     if (hasCause(e, TransactionTimeoutException.class) &&
-                        hasCause(e, TransactionDeadlockException.class)
-                        ) {
-                        if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class)))
-                            U.error(log, "At least one stack trace should contain " +
-                                TransactionDeadlockException.class.getSimpleName(), e);
+                        hasCause(e, TransactionDeadlockException.class)) {
+                        if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class))) {
+                            log.info("At least one stack trace should contain " +
+                                TransactionDeadlockException.class.getSimpleName());
+
+                            e.printStackTrace(System.out);
+                        }
                     }
                 }
             }
@@ -344,7 +351,7 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         TransactionDeadlockException deadlockE = deadlockErr.get();
 
-        assertNotNull(deadlockE);
+        assertNotNull("Failed to detect deadlock", deadlockE);
 
         boolean fail = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
index 6fc7e02..7b5abf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
@@ -372,4 +372,4 @@ public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstr
             X.println("Executing cache service: " + ctx.name());
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
index adcd144..4bc9f01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.util.future;
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
@@ -227,87 +228,98 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
      *
      * @throws Exception In case of any exception.
      */
-    @SuppressWarnings("ErrorNotRethrown")
     public void testChaining() throws Exception {
+        checkChaining(null);
+
+        ExecutorService exec = Executors.newFixedThreadPool(1);
+
+        try {
+            checkChaining(exec);
+
+            GridFinishedFuture<Integer> fut = new GridFinishedFuture<>(1);
+
+            IgniteInternalFuture<Object> chain = fut.chain(new CX1<IgniteInternalFuture<Integer>, Object>() {
+                @Override public Object applyx(IgniteInternalFuture<Integer> fut) throws IgniteCheckedException {
+                    return fut.get() + 1;
+                }
+            }, exec);
+
+            assertEquals(2, chain.get());
+        }
+        finally {
+            exec.shutdown();
+        }
+    }
+
+    /**
+     * @param exec Executor for chain callback.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ErrorNotRethrown")
+    private void checkChaining(ExecutorService exec) throws Exception {
         final CX1<IgniteInternalFuture<Object>, Object> passThrough = new CX1<IgniteInternalFuture<Object>, Object>() {
             @Override public Object applyx(IgniteInternalFuture<Object> f) throws IgniteCheckedException {
                 return f.get();
             }
         };
 
-        final GridTestKernalContext ctx = new GridTestKernalContext(log);
-
-        ctx.setExecutorService(Executors.newFixedThreadPool(1));
-        ctx.setSystemExecutorService(Executors.newFixedThreadPool(1));
-
-        ctx.add(new PoolProcessor(ctx));
-        ctx.add(new GridClosureProcessor(ctx));
+        GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
+        IgniteInternalFuture<Object> chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-        ctx.start();
+        assertFalse(fut.isDone());
+        assertFalse(chain.isDone());
 
         try {
-            // Test result returned.
-
-            GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
-            IgniteInternalFuture<Object> chain = fut.chain(passThrough);
+            chain.get(20);
 
-            assertFalse(fut.isDone());
-            assertFalse(chain.isDone());
-
-            try {
-                chain.get(20);
-
-                fail("Expects timeout exception.");
-            }
-            catch (IgniteFutureTimeoutCheckedException e) {
-                info("Expected timeout exception: " + e.getMessage());
-            }
+            fail("Expects timeout exception.");
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            info("Expected timeout exception: " + e.getMessage());
+        }
 
-            fut.onDone("result");
+        fut.onDone("result");
 
-            assertEquals("result", chain.get(1));
+        assertEquals("result", chain.get(1));
 
-            // Test exception re-thrown.
+        // Test exception re-thrown.
 
-            fut = new GridFutureAdapter<>();
-            chain = fut.chain(passThrough);
+        fut = new GridFutureAdapter<>();
+        chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-            fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
+        fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
 
-            try {
-                chain.get();
+        try {
+            chain.get();
 
-                fail("Expects failed with exception.");
-            }
-            catch (ClusterGroupEmptyCheckedException e) {
-                info("Expected exception: " + e.getMessage());
-            }
+            fail("Expects failed with exception.");
+        }
+        catch (ClusterGroupEmptyCheckedException e) {
+            info("Expected exception: " + e.getMessage());
+        }
 
-            // Test error re-thrown.
+        // Test error re-thrown.
 
-            fut = new GridFutureAdapter<>();
-            chain = fut.chain(passThrough);
+        fut = new GridFutureAdapter<>();
+        chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-            try {
-                fut.onDone(new StackOverflowError("test error"));
+        try {
+            fut.onDone(new StackOverflowError("test error"));
 
+            if (exec == null)
                 fail("Expects failed with error.");
-            }
-            catch (StackOverflowError e) {
-                info("Expected error: " + e.getMessage());
-            }
+        }
+        catch (StackOverflowError e) {
+            info("Expected error: " + e.getMessage());
+        }
 
-            try {
-                chain.get();
+        try {
+            chain.get();
 
-                fail("Expects failed with error.");
-            }
-            catch (StackOverflowError e) {
-                info("Expected error: " + e.getMessage());
-            }
+            fail("Expects failed with error.");
         }
-        finally {
-            ctx.stop(false);
+        catch (StackOverflowError e) {
+            info("Expected error: " + e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
index 201fd27..d403784 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
@@ -114,7 +114,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
                 proceedExceptionCaught(ses, ex);
             }
 
-            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
                 sndEvt.compareAndSet(null, ses.<String>meta(MESSAGE_WRITE_META_NAME));
 
                 sndMsgObj.compareAndSet(null, msg);
@@ -155,7 +155,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         chain.onSessionIdleTimeout(ses);
         chain.onSessionWriteTimeout(ses);
         assertNull(chain.onSessionClose(ses));
-        assertNull(chain.onSessionWrite(ses, snd));
+        assertNull(chain.onSessionWrite(ses, snd, true));
 
         assertEquals("DCBA", connectedEvt.get());
         assertEquals("DCBA", disconnectedEvt.get());
@@ -210,10 +210,10 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             chainMeta(ses, MESSAGE_WRITE_META_NAME);
 
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -349,6 +349,11 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void sendNoFuture(Object msg) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public GridNioFuture<Object> resumeReads() {
             return null;
         }
@@ -369,13 +374,28 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+        @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public void systemMessage(Object msg) {
+            // No-op.
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
index 61a13b1..25dd780 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
@@ -83,7 +83,7 @@ public class GridNioBenchmarkClient {
      */
     public void run() throws IOException, InterruptedException {
         for (int i = 0; i < connCnt; i++)
-            exec.submit(new ClientThread());
+            exec.execute(new ClientThread());
 
         Thread.sleep(5*60*1000);
 
@@ -167,4 +167,4 @@ public class GridNioBenchmarkClient {
             return read;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
index f21f31b..a18ef32 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
@@ -196,4 +196,4 @@ public class GridP2PRecursionTaskSelfTest extends GridCommonAbstractTest {
             return ignite.compute().execute(FactorialTask.class, arg);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
index 652e47f..5ca8f26 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
@@ -69,7 +69,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
     private static final int commExtPort2 = 20100;
 
     /** */
-    private AddressResolver resolver;
+    private AddressResolver rslvr;
 
     /** */
     private boolean ipFinderUseLocPorts;
@@ -111,14 +111,15 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         cfg.setConnectorConfiguration(null);
 
         TcpCommunicationSpi commSpi = new TcpCommunicationSpi() {
-            @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+            @Override protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx)
+                throws IgniteCheckedException {
                 Map<String, Object> attrs = new HashMap<>(node.attributes());
 
                 attrs.remove(createSpiAttributeName(ATTR_PORT));
 
                 ((TcpDiscoveryNode)node).setAttributes(attrs);
 
-                return super.createTcpClient(node);
+                return super.createTcpClient(node, connIdx);
             }
         };
 
@@ -126,12 +127,13 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         commSpi.setLocalPort(commLocPort);
         commSpi.setLocalPortRange(1);
         commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(1);
 
         cfg.setCommunicationSpi(commSpi);
 
-        assert resolver != null;
+        assert rslvr != null;
 
-        cfg.setAddressResolver(resolver);
+        cfg.setAddressResolver(rslvr);
 
         return cfg;
     }
@@ -147,7 +149,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         map.put(new InetSocketAddress("127.0.0.1", locPort2), F.asList(new InetSocketAddress("127.0.0.1", extPort2)));
         map.put(new InetSocketAddress("127.0.0.1", commLocPort2), F.asList(new InetSocketAddress("127.0.0.1", commExtPort2)));
 
-        resolver = new AddressResolver() {
+        rslvr = new AddressResolver() {
             @Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr) {
                 return map.get(addr);
             }
@@ -167,7 +169,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         map.put("127.0.0.1:" + locPort2, "127.0.0.1:" + extPort2);
         map.put("127.0.0.1:" + commLocPort2, "127.0.0.1:" + commExtPort2);
 
-        resolver = new BasicAddressResolver(map);
+        rslvr = new BasicAddressResolver(map);
 
         doTestForward();
     }
@@ -180,7 +182,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
 
         map.put("127.0.0.1", "127.0.0.1");
 
-        resolver = new BasicAddressResolver(map);
+        rslvr = new BasicAddressResolver(map);
 
         ipFinderUseLocPorts = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index 076724d..3c4fea0 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -90,16 +90,36 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         super.afterTest();
 
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
+
+            for (int i = 0; i < 20; i++) {
+                GridCommunicationClient client0 = null;
+
+                for (GridCommunicationClient[] clients0 : clients.values()) {
+                    for (GridCommunicationClient client : clients0) {
+                        if (client != null) {
+                            client0 = client;
+
+                            break;
+                        }
+                    }
+
+                    if (client0 != null)
+                        break;
+                }
+
+                if (client0 == null)
+                    return;
 
-            for (int i = 0; i < 20 && !clients.isEmpty(); i++) {
                 info("Check failed for SPI [grid=" +
-                    GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") + ", spi=" + spi + ']');
+                    GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") +
+                    ", client=" + client0 +
+                    ", spi=" + spi + ']');
 
                 U.sleep(1000);
             }
 
-            assert clients.isEmpty() : "Clients: " + clients;
+            fail("Failed to wait when clients are closed.");
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index 8635d94..a649130 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -31,6 +31,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -83,6 +84,12 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
     /** Use ssl. */
     protected boolean useSsl;
 
+    /** */
+    private int connectionsPerNode = 1;
+
+    /** */
+    private boolean pairedConnections = true;
+
     /**
      *
      */
@@ -163,6 +170,34 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
     /**
      * @throws Exception If failed.
      */
+    public void testMultithreaded_10Connections() throws Exception {
+        connectionsPerNode = 10;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded_NoPairedConnections() throws Exception {
+        pairedConnections = false;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded_10ConnectionsNoPaired() throws Exception {
+        pairedConnections = false;
+        connectionsPerNode = 10;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testWithLoad() throws Exception {
         int threads = Runtime.getRuntime().availableProcessors() * 5;
 
@@ -244,7 +279,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
                 final AtomicInteger idx = new AtomicInteger();
 
                 try {
-                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                    final Callable<Void> c = new Callable<Void>() {
                         @Override public Void call() throws Exception {
                             int idx0 = idx.getAndIncrement();
 
@@ -270,7 +305,40 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
                             return null;
                         }
-                    }, threads, "test");
+                    };
+
+                    List<Thread> threadsList = new ArrayList<>();
+
+                    final AtomicBoolean fail = new AtomicBoolean();
+
+                    final AtomicLong tId = new AtomicLong();
+
+                    for (int t = 0; t < threads; t++) {
+                        Thread t0 = new Thread(new Runnable() {
+                            @Override public void run() {
+                                try {
+                                    c.call();
+                                }
+                                catch (Throwable e) {
+                                    log.error("Unexpected error: " + e, e);
+
+                                    fail.set(true);
+                                }
+                            }
+                        }) {
+                            @Override public long getId() {
+                                // Override getId to use all connections.
+                                return tId.getAndIncrement();
+                            }
+                        };
+
+                        threadsList.add(t0);
+
+                        t0.start();
+                    }
+
+                    for (Thread t0 : threadsList)
+                        t0.join();
 
                     assertTrue(latch.await(10, TimeUnit.SECONDS));
 
@@ -281,17 +349,19 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
                         final GridNioServer srv = U.field(spi, "nioSrvr");
 
+                        final int conns = pairedConnections ? 2 : 1;
+
                         GridTestUtils.waitForCondition(new GridAbsPredicate() {
                             @Override public boolean apply() {
                                 Collection sessions = U.field(srv, "sessions");
 
-                                return sessions.size() == 1;
+                                return sessions.size() == conns * connectionsPerNode;
                             }
                         }, 5000);
 
                         Collection sessions = U.field(srv, "sessions");
 
-                        assertEquals(1, sessions.size());
+                        assertEquals(conns * connectionsPerNode, sessions.size());
                     }
 
                     assertEquals(expMsgs, lsnr.cntr.get());
@@ -320,6 +390,8 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
         spi.setIdleConnectionTimeout(60_000);
         spi.setConnectTimeout(10_000);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(connectionsPerNode);
+        spi.setUsePairedConnections(pairedConnections);
 
         return spi;
     }
@@ -434,4 +506,4 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
             rsrcs.stopThreads();
     }
 
-}
\ No newline at end of file
+}


[12/50] [abbrv] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NI

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
index 1b11688..87d9225 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
@@ -611,6 +611,11 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index 1c1addd..c3e9fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -125,6 +125,11 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         );
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
     /**
      * @param key Key to add.
      * @param val Optional update value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
index fa7f367..4272a4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
@@ -244,6 +244,11 @@ public class GridNearGetRequest extends GridCacheMessage implements GridCacheDep
         return accessTtl;
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /**
      * @param ctx Cache context.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index a419887..bc16ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -1585,6 +1585,9 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
      */
     @SuppressWarnings("unchecked")
     protected IgniteInternalFuture asyncOp(final Callable<?> op) {
+        if (!asyncToggled)
+            return ctx.closures().callLocalSafe(op);
+
         IgniteInternalFuture fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index d34047e..eb5e214 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -272,7 +272,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 false,
                 null,
                 req.keyValueFilter(),
-                req.partition(),
+                req.partition() == -1 ? null : req.partition(),
                 req.className(),
                 req.clause(),
                 req.includeMetaData(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index 60c4662..9f965d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -121,7 +121,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     private int taskHash;
 
     /** Partition. */
-    private int part;
+    private int part = -1;
 
     /** */
     private AffinityTopologyVersion topVer;
@@ -478,8 +478,8 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     /**
      * @return partition.
      */
-    @Nullable public Integer partition() {
-        return part == -1 ? null : part;
+    public int partition() {
+        return part;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 6d21dcf..393fb1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -391,7 +391,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> loadMissing(
         final GridCacheContext cacheCtx,
-        AffinityTopologyVersion topVer,
+        final AffinityTopologyVersion topVer,
         final boolean readThrough,
         boolean async,
         final Collection<KeyCacheObject> keys,
@@ -472,7 +472,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                             CacheObject cacheVal = cacheCtx.toCacheObject(val);
 
                             while (true) {
-                                GridCacheEntryEx entry = cacheCtx.cache().entryEx(key);
+                                GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
 
                                 try {
                                     GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
@@ -1507,7 +1507,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
                         assert txEntry != null || readCommitted() || skipVals;
 
-                        GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey) : txEntry.cached();
+                        GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey, topVer) : txEntry.cached();
 
                         if (readCommitted() || skipVals) {
                             cacheCtx.evicts().touch(e, topologyVersion());
@@ -1658,7 +1658,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                             IgniteTxLocalAdapter.this,
                                             /*swap*/cacheCtx.isSwapOrOffheapEnabled(),
                                             /*unmarshal*/true,
-                                            /**update-metrics*/true,
+                                            /*update-metrics*/true,
                                             /*event*/!skipVals,
                                             CU.subjectId(IgniteTxLocalAdapter.this, cctx),
                                             transformClo,

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index 32fda87..fee4dd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -328,6 +328,8 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
             if (!allowOverwrite)
                 cctx.topology().readLock();
 
+            GridDhtTopologyFuture topWaitFut = null;
+
             try {
                 GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
 
@@ -352,19 +354,25 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
 
                     waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer);
                 }
-                else {
-                    fut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
-                            localUpdate(nodeId, req, updater, topic);
-                        }
-                    });
-                }
+                else
+                    topWaitFut = fut;
             }
             finally {
                 if (!allowOverwrite)
                     cctx.topology().readUnlock();
             }
 
+            if (topWaitFut != null) {
+                // Need call 'listen' after topology read lock is released.
+                topWaitFut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
+                        localUpdate(nodeId, req, updater, topic);
+                    }
+                });
+
+                return;
+            }
+
             if (job != null) {
                 try {
                     job.call();

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 3405b53..4c037b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -226,7 +226,7 @@ public class IgfsContext {
      */
     public void runInIgfsThreadPool(Runnable r) {
         try {
-            igfsSvc.submit(r);
+            igfsSvc.execute(r);
         }
         catch (RejectedExecutionException ignored) {
             // This exception will happen if network speed is too low and data comes faster
@@ -252,4 +252,4 @@ public class IgfsContext {
 
         return mgr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index e534800..4490a68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -36,6 +37,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -325,6 +327,8 @@ public class IgfsDataManager extends IgfsManager {
         IgniteInternalFuture<byte[]> fut = dataCachePrj.getAsync(key);
 
         if (secReader != null) {
+            Executor exec = igfsCtx.kernalContext().pools().poolForPolicy(GridIoPolicy.IGFS_POOL);
+
             fut = fut.chain(new CX1<IgniteInternalFuture<byte[]>, byte[]>() {
                 @Override public byte[] applyx(IgniteInternalFuture<byte[]> fut) throws IgniteCheckedException {
                     byte[] res = fut.get();
@@ -365,7 +369,7 @@ public class IgfsDataManager extends IgfsManager {
 
                     return res;
                 }
-            });
+            }, exec);
         }
         else
             igfsCtx.metrics().addReadBlocks(1, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index ab4ee85..6b23e80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -326,7 +326,7 @@ public final class IgfsImpl implements IgfsEx {
                 // Submit it to the thread pool immediately.
                 assert dualPool != null;
 
-                dualPool.submit(batch);
+                dualPool.execute(batch);
 
                 // Spin in case another batch is currently running.
                 while (true) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
index 9388a8e..7cba9bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
@@ -139,6 +139,7 @@ public class OdbcProcessor extends GridProcessorAdapter {
                             .logger(log)
                             .selectorCount(DFLT_SELECTOR_CNT)
                             .gridName(ctx.gridName())
+                            .serverName("odbc")
                             .tcpNoDelay(DFLT_TCP_NODELAY)
                             .directBuffer(DFLT_TCP_DIRECT_BUF)
                             .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 8ff15d5..5383151 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.compute;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.binary.BinaryObject;
@@ -409,6 +410,11 @@ public class PlatformCompute extends PlatformAbstractTarget {
         }
 
         /** {@inheritDoc} */
+        @Override public IgniteInternalFuture chain(IgniteClosure doneCb, Executor exec) {
+            throw new UnsupportedOperationException("Chain operation is not supported.");
+        }
+
+        /** {@inheritDoc} */
         @Override public Throwable error() {
             return fut.error();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
index b403654..71eca65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
@@ -38,8 +37,6 @@ import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT;
@@ -72,24 +69,16 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
     /** Handler. */
     private final GridRestProtocolHandler hnd;
 
-    /** JDK marshaller. */
-    private final Marshaller jdkMarshaller = new JdkMarshaller();
-
-    /** Context. */
-    private final GridKernalContext ctx;
-
     /**
      * Creates listener which will convert incoming tcp packets to rest requests and forward them to
      * a given rest handler.
      *
      * @param log Logger to use.
      * @param hnd Rest handler.
-     * @param ctx Context.
      */
-    public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd, GridKernalContext ctx) {
+    public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd) {
         this.log = log;
         this.hnd = hnd;
-        this.ctx = ctx;
     }
 
     /** {@inheritDoc} */
@@ -462,4 +451,4 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
 
         return new GridTuple3<>(cmd, quiet, retKey);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 1c1c6dc..3ba6d8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -145,7 +145,7 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
      */
     public GridTcpRestNioListener(IgniteLogger log, GridTcpRestProtocol proto, GridRestProtocolHandler hnd,
         GridKernalContext ctx) {
-        memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd, ctx);
+        memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd);
         redisLsnr = new GridRedisNioListener(log, hnd, ctx);
 
         this.log = log;

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 6338fcc..2a002a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -257,6 +257,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
                 .logger(log)
                 .selectorCount(cfg.getSelectorCount())
                 .gridName(ctx.gridName())
+                .serverName("tcp-rest")
                 .tcpNoDelay(cfg.isNoDelay())
                 .directBuffer(cfg.isDirectBuffer())
                 .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 6c26363..aeb3ef4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1131,7 +1131,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             // Start service in its own thread.
             final ExecutorService exe = svcCtx.executor();
 
-            exe.submit(new Runnable() {
+            exe.execute(new Runnable() {
                 @Override public void run() {
                     try {
                         svc.execute(svcCtx);
@@ -1394,7 +1394,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 return;
 
             try {
-                depExe.submit(new BusyRunnable() {
+                depExe.execute(new BusyRunnable() {
                     @Override public void run0() {
                         onSystemCacheUpdated(deps);
                     }
@@ -1587,7 +1587,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 else
                     topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
 
-                depExe.submit(new BusyRunnable() {
+                depExe.execute(new BusyRunnable() {
                     @Override public void run0() {
                         ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index e1937bb..3dfb3c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -153,7 +153,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cluster.ClusterGroupEmptyException;
@@ -506,10 +505,27 @@ public abstract class IgniteUtils {
         }
     };
 
-    /**
-     * Initializes enterprise check.
+    /** */
+    private static final boolean assertionsEnabled;
+
+    /*
+     *
      */
     static {
+        boolean assertionsEnabled0 = true;
+
+        try {
+            assert false;
+
+            assertionsEnabled0 = false;
+        }
+        catch (AssertionError ignored) {
+            assertionsEnabled0 = true;
+        }
+        finally {
+            assertionsEnabled = assertionsEnabled0;
+        }
+
         String osName = System.getProperty("os.name");
 
         String osLow = osName.toLowerCase();
@@ -1284,6 +1300,27 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @param threadId Thread ID.
+     * @param sb Builder.
+     */
+    public static void printStackTrace(long threadId, GridStringBuilder sb) {
+        ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+        ThreadInfo threadInfo = mxBean.getThreadInfo(threadId, Integer.MAX_VALUE);
+
+        printThreadInfo(threadInfo, sb, Collections.<Long>emptySet());
+    }
+
+    /**
+     * @return {@code true} if there is java level deadlock.
+     */
+    public static boolean deadlockPresent() {
+        ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+        return !F.isEmpty(mxBean.findDeadlockedThreads());
+    }
+
+    /**
      * Prints single thread info to a buffer.
      *
      * @param threadInfo Thread info.
@@ -6141,6 +6178,13 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @return {@code True} if assertions enabled.
+     */
+    public static boolean assertionsEnabled() {
+        return assertionsEnabled;
+    }
+
+    /**
      * Gets OS JDK string.
      *
      * @return OS JDK string.
@@ -8337,6 +8381,18 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets absolute value for long. If argument is {@link Long#MIN_VALUE}, then {@code 0} is returned.
+     *
+     * @param i Argument.
+     * @return Absolute value.
+     */
+    public static long safeAbs(long i) {
+        i = Math.abs(i);
+
+        return i < 0 ? 0 : i;
+    }
+
+    /**
      * Gets wrapper class for a primitive type.
      *
      * @param cls Class. If {@code null}, method is no-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
new file mode 100644
index 0000000..e9ec74b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
@@ -0,0 +1,667 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Striped executor.
+ */
+public class StripedExecutor implements ExecutorService {
+    /** Stripes. */
+    private final Stripe[] stripes;
+
+    /** For starvation checks. */
+    private final long[] completedCntrs;
+
+    /** */
+    private final IgniteLogger log;
+
+    /**
+     * Constructor.
+     *
+     * @param cnt Count.
+     */
+    public StripedExecutor(int cnt, String gridName, String poolName, final IgniteLogger log) {
+        A.ensure(cnt > 0, "cnt > 0");
+
+        boolean success = false;
+
+        stripes = new Stripe[cnt];
+
+        completedCntrs = new long[cnt];
+
+        Arrays.fill(completedCntrs, -1);
+
+        this.log = log;
+
+        try {
+            for (int i = 0; i < cnt; i++) {
+                stripes[i] = new StripeConcurrentQueue(
+                    gridName,
+                    poolName,
+                    i,
+                    log);
+
+                stripes[i].start();
+            }
+
+            success = true;
+        }
+        catch (Error | RuntimeException e) {
+            U.error(log, "Failed to initialize striped pool.", e);
+
+            throw e;
+        }
+        finally {
+            if (!success) {
+                for (Stripe stripe : stripes) {
+                    if (stripe != null)
+                        stripe.signalStop();
+                }
+
+                for (Stripe stripe : stripes) {
+                    if (stripe != null)
+                        stripe.awaitStop();
+                }
+            }
+        }
+    }
+
+    /**
+     * Checks starvation in striped pool. Maybe too verbose
+     * but this is needed to faster debug possible issues.
+     */
+    public void checkStarvation() {
+        for (int i = 0; i < stripes.length; i++) {
+            Stripe stripe = stripes[i];
+
+            long completedCnt = stripe.completedCnt;
+
+            boolean active = stripe.active;
+
+            if (completedCntrs[i] != -1 &&
+                completedCntrs[i] == completedCnt &&
+                active) {
+                boolean deadlockPresent = U.deadlockPresent();
+
+                GridStringBuilder sb = new GridStringBuilder();
+
+                sb.a(">>> Possible starvation in striped pool: ")
+                    .a(stripe.thread.getName()).a(U.nl())
+                    .a(stripe.queueToString()).a(U.nl())
+                    .a("deadlock: ").a(deadlockPresent).a(U.nl())
+                    .a("completed: ").a(completedCnt).a(U.nl());
+
+                U.printStackTrace(
+                    stripe.thread.getId(),
+                    sb);
+
+                String msg = sb.toString();
+
+                U.warn(log, msg);
+            }
+
+            if (active || completedCnt > 0)
+                completedCntrs[i] = completedCnt;
+        }
+    }
+
+    /**
+     * @return Stripes count.
+     */
+    public int stripes() {
+        return stripes.length;
+    }
+
+    /**
+     * Execute command.
+     *
+     * @param idx Index.
+     * @param cmd Command.
+     */
+    public void execute(int idx, Runnable cmd) {
+        if (idx == -1)
+            execute(cmd);
+        else {
+            assert idx >= 0 : idx;
+
+            stripes[idx % stripes.length].execute(cmd);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void shutdown() {
+        signalStop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(@NotNull Runnable cmd) {
+        stripes[ThreadLocalRandom.current().nextInt(stripes.length)].execute(cmd);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @return Empty list (always).
+     */
+    @NotNull @Override public List<Runnable> shutdownNow() {
+        signalStop();
+
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean awaitTermination(
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException {
+        awaitStop();
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isShutdown() {
+        for (Stripe stripe : stripes) {
+            if (stripe != null && stripe.stopping)
+                return true;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTerminated() {
+        for (Stripe stripe : stripes) {
+            if (stripe.thread.getState() != Thread.State.TERMINATED)
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Stops executor.
+     */
+    public void stop() {
+        signalStop();
+
+        awaitStop();
+    }
+
+    /**
+     * Signals all stripes.
+     */
+    private void signalStop() {
+        for (Stripe stripe : stripes)
+            stripe.signalStop();
+    }
+
+    /**
+     * @throws IgniteInterruptedException If interrupted.
+     */
+    private void awaitStop() throws IgniteInterruptedException {
+        for (Stripe stripe : stripes)
+            stripe.awaitStop();
+    }
+
+    /**
+     * @return Return total queue size of all stripes.
+     */
+    public int queueSize() {
+        int size = 0;
+
+        for (Stripe stripe : stripes)
+            size += stripe.queueSize();
+
+        return size;
+    }
+
+    /**
+     * @return Completed tasks count.
+     */
+    public long completedTasks() {
+        long cnt = 0;
+
+        for (Stripe stripe : stripes)
+            cnt += stripe.completedCnt;
+
+        return cnt;
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> Future<T> submit(
+        @NotNull Runnable task,
+        T res
+    ) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public Future<?> submit(@NotNull Runnable task) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> Future<T> submit(@NotNull Callable<T> task) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> List<Future<T>> invokeAll(@NotNull Collection<? extends Callable<T>> tasks)
+        throws InterruptedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> List<Future<T>> invokeAll(
+        @NotNull Collection<? extends Callable<T>> tasks,
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> T invokeAny(@NotNull Collection<? extends Callable<T>> tasks)
+        throws InterruptedException, ExecutionException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @Override public <T> T invokeAny(
+        @NotNull Collection<? extends Callable<T>> tasks,
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException, ExecutionException, TimeoutException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StripedExecutor.class, this);
+    }
+
+    /**
+     * Stripe.
+     */
+    private static abstract class Stripe implements Runnable {
+        /** */
+        private final String gridName;
+
+        /** */
+        private final String poolName;
+
+        /** */
+        private final int idx;
+
+        /** */
+        private final IgniteLogger log;
+
+        /** Stopping flag. */
+        private volatile boolean stopping;
+
+        /** */
+        private volatile long completedCnt;
+
+        /** */
+        private volatile boolean active;
+
+        /** Thread executing the loop. */
+        protected Thread thread;
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public Stripe(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            this.gridName = gridName;
+            this.poolName = poolName;
+            this.idx = idx;
+            this.log = log;
+        }
+
+        /**
+         * Starts the stripe.
+         */
+        void start() {
+            thread = new IgniteThread(gridName, poolName + "-stripe-" + idx, this);
+
+            thread.start();
+        }
+
+        /**
+         * Stop the stripe.
+         */
+        void signalStop() {
+            stopping = true;
+
+            U.interrupt(thread);
+        }
+
+        /**
+         * Await thread stop.
+         */
+        void awaitStop() {
+            try {
+                if (thread != null)
+                    thread.join();
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new IgniteInterruptedException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            while (!stopping) {
+                Runnable cmd;
+
+                try {
+                    cmd = take();
+
+                    if (cmd != null) {
+                        active = true;
+
+                        try {
+                            cmd.run();
+                        }
+                        finally {
+                            active = false;
+                            completedCnt++;
+                        }
+                    }
+                }
+                catch (InterruptedException e) {
+                    stopping = true;
+
+                    Thread.currentThread().interrupt();
+
+                    return;
+                }
+                catch (Throwable e) {
+                    U.error(log, "Failed to execute runnable.", e);
+                }
+            }
+        }
+
+        /**
+         * Execute the command.
+         *
+         * @param cmd Command.
+         */
+        abstract void execute(Runnable cmd);
+
+        /**
+         * @return Next runnable.
+         * @throws InterruptedException If interrupted.
+         */
+        abstract Runnable take() throws InterruptedException;
+
+        /**
+         * @return Queue size.
+         */
+        abstract int queueSize();
+
+        /**
+         * @return Stripe's queue to string presentation.
+         */
+        abstract String queueToString();
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Stripe.class, this);
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentQueue extends Stripe {
+        /** Queue. */
+        private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+        /** */
+        private volatile boolean parked;
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentQueue(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() throws InterruptedException {
+            Runnable r;
+
+            for (int i = 0; i < 2048; i++) {
+                r = queue.poll();
+
+                if (r != null)
+                    return r;
+            }
+
+            parked = true;
+
+            try {
+                for (;;) {
+                    r = queue.poll();
+
+                    if (r != null)
+                        return r;
+
+                    LockSupport.park();
+
+                    if (Thread.interrupted())
+                        throw new InterruptedException();
+                }
+            }
+            finally {
+                parked = false;
+            }
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+
+            if (parked)
+                LockSupport.unpark(thread);
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentQueue.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentQueueNoPark extends Stripe {
+        /** Queue. */
+        private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentQueueNoPark(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() {
+            for (;;) {
+                Runnable r = queue.poll();
+
+                if (r != null)
+                    return r;
+            }
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentQueueNoPark.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentBlockingQueue extends Stripe {
+        /** Queue. */
+        private final BlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentBlockingQueue(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() throws InterruptedException {
+            return queue.take();
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentBlockingQueue.class, this, super.toString());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
index 6baedbd..dc63adc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.util.future;
 
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -152,6 +153,29 @@ public class GridFinishedFuture<T> implements IgniteInternalFuture<T> {
     }
 
     /** {@inheritDoc} */
+    @Override public <T1> IgniteInternalFuture<T1> chain(final IgniteClosure<? super IgniteInternalFuture<T>, T1> doneCb, Executor exec) {
+        final GridFutureAdapter<T1> fut = new GridFutureAdapter<>();
+
+        exec.execute(new Runnable() {
+            @Override public void run() {
+                try {
+                    fut.onDone(doneCb.apply(GridFinishedFuture.this));
+                }
+                catch (GridClosureException e) {
+                    fut.onDone(e.unwrap());
+                }
+                catch (RuntimeException | Error e) {
+                    fut.onDone(e);
+
+                    throw e;
+                }
+            }
+        });
+
+        return fut;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridFinishedFuture.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index 2cd534e..c8d85cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import java.util.Arrays;
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.AbstractQueuedSynchronizer;
 import org.apache.ignite.IgniteCheckedException;
@@ -229,7 +230,13 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
 
     /** {@inheritDoc} */
     @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
-        return new ChainFuture<>(this, doneCb);
+        return new ChainFuture<>(this, doneCb, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+        Executor exec) {
+        return new ChainFuture<>(this, doneCb, exec);
     }
 
     /**
@@ -487,15 +494,17 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         /**
          * @param fut Future.
          * @param doneCb Closure.
+         * @param cbExec Optional executor to run callback.
          */
         ChainFuture(
             GridFutureAdapter<R> fut,
-            IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb
+            IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+            @Nullable Executor cbExec
         ) {
             this.fut = fut;
             this.doneCb = doneCb;
 
-            fut.listen(new GridFutureChainListener<>(this, doneCb));
+            fut.listen(new GridFutureChainListener<>(this, doneCb, cbExec));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
index 947b2ad..367f5d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
@@ -17,15 +17,17 @@
 
 package org.apache.ignite.internal.util.future;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Future listener to fill chained future with converted result of the source future.
  */
-public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
+class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -35,21 +37,43 @@ public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInte
     /** Done callback. */
     private final IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb;
 
+    /** */
+    private Executor cbExec;
+
     /**
      * Constructs chain listener.
+     *
      *  @param fut Target future.
      * @param doneCb Done callback.
+     * @param cbExec Optional executor to run callback.
      */
     public GridFutureChainListener(
         GridFutureAdapter<R> fut,
-        IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb
+        IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb,
+        @Nullable Executor cbExec
     ) {
         this.fut = fut;
         this.doneCb = doneCb;
+        this.cbExec = cbExec;
     }
 
     /** {@inheritDoc} */
-    @Override public void apply(IgniteInternalFuture<T> t) {
+    @Override public void apply(final IgniteInternalFuture<T> t) {
+        if (cbExec != null) {
+            cbExec.execute(new Runnable() {
+                @Override public void run() {
+                    applyCallback(t);
+                }
+            });
+        }
+        else
+            applyCallback(t);
+    }
+
+    /**
+     * @param t Target future.
+     */
+    private void applyCallback(IgniteInternalFuture<T> t) {
         try {
             fut.onDone(doneCb.apply(t));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
index 6820dc7..d108b56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
@@ -201,7 +201,7 @@ public class IpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
             assert ses == IpcToNioAdapter.this.ses;
 
             return send((Message)msg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
index 9b014ec..f2ab932 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
@@ -35,14 +35,24 @@ public abstract class GridAbstractCommunicationClient implements GridCommunicati
     /** Metrics listener. */
     protected final GridNioMetricsListener metricsLsnr;
 
+    /** */
+    private final int connIdx;
+
     /**
+     * @param connIdx Connection index.
      * @param metricsLsnr Metrics listener.
      */
-    protected GridAbstractCommunicationClient(@Nullable GridNioMetricsListener metricsLsnr) {
+    protected GridAbstractCommunicationClient(int connIdx, @Nullable GridNioMetricsListener metricsLsnr) {
+        this.connIdx = connIdx;
         this.metricsLsnr = metricsLsnr;
     }
 
     /** {@inheritDoc} */
+    @Override public int connectionIndex() {
+        return connIdx;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean close() {
         return reserves.compareAndSet(0, -1);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
index 0de54e9..71b2c24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
@@ -96,15 +96,20 @@ public interface GridCommunicationClient {
     /**
      * @param nodeId Remote node ID. Provided only for sync clients.
      * @param msg Message to send.
-     * @param closure Ack closure.
+     * @param c Ack closure.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if should try to resend message.
      */
-    public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> closure)
+    public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> c)
         throws IgniteCheckedException;
 
     /**
      * @return {@code True} if send is asynchronous.
      */
     public boolean async();
+
+    /**
+     * @return Connection index.
+     */
+    public int connectionIndex();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 213fd8d..7987d3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -62,13 +62,20 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+    @Override public void onExceptionCaught(
+        GridNioSession ses,
+        IgniteCheckedException ex
+    ) throws IgniteCheckedException {
         proceedExceptionCaught(ses, ex);
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return proceedSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -137,4 +144,4 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
index 9925d2e..40c87cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
@@ -107,8 +107,12 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return proceedSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -139,4 +143,4 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
                 "originalEx=" + ex + ", ex=" + e + ']');
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index 7083ccf..343e625 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -71,20 +71,27 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+    @Override public void onExceptionCaught(
+        GridNioSession ses,
+        IgniteCheckedException ex
+    ) throws IgniteCheckedException {
         proceedExceptionCaught(ses, ex);
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         // No encoding needed in direct mode.
         if (directMode)
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
 
         try {
             ByteBuffer res = parser.encode(ses, msg);
 
-            return proceedSessionWrite(ses, res);
+            return proceedSessionWrite(ses, res, fut);
         }
         catch (IOException e) {
             throw new GridNioException(e);
@@ -137,4 +144,4 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
index 5f88b1f..f7928c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
@@ -105,10 +105,15 @@ public interface GridNioFilter {
      *
      * @param ses Session instance.
      * @param msg Message to send.
-     * @return Write future.
+     * @param fut {@code True} if write future should be created.
+     * @return Write future or {@code null}.
      * @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
      */
-    public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+    public GridNioFuture<?> proceedSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException;
 
     /**
      * Forwards session close request to the next logical filter in filter chain.
@@ -149,10 +154,11 @@ public interface GridNioFilter {
      *
      * @param ses Session on which message should be written.
      * @param msg Message being written.
-     * @return Write future.
+     * @param fut {@code True} if write future should be created.
+     * @return Write future or {@code null}.
      * @throws GridNioException If GridNioException occurred while handling event.
      */
-    public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+    public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException;
 
     /**
      * Invoked when a new messages received.
@@ -241,4 +247,4 @@ public interface GridNioFilter {
      * @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
      */
     public GridNioFuture<?> onResumeReads(GridNioSession ses) throws IgniteCheckedException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
index 18ab1b2..58ddae5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
@@ -108,10 +108,14 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> proceedSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         checkNext();
 
-        return nextFilter.onSessionWrite(ses, msg);
+        return nextFilter.onSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -180,4 +184,4 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
             throw new GridNioException("Failed to proceed with filter call since previous filter is not set " +
                 "(do you use filter outside the filter chain?): " + getClass().getName());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index a3a74e3..8cc690b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -181,8 +181,12 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
      * @return Send future.
      * @throws IgniteCheckedException If IgniteCheckedException occurred while handling event.
      */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return tail.onSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return tail.onSessionWrite(ses, msg, fut);
     }
 
     /**
@@ -255,9 +259,9 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg)
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut)
             throws IgniteCheckedException {
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -290,4 +294,4 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
             return proceedResumeReads(ses);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
index b02acc8..6c0c9c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
@@ -45,9 +45,9 @@ public interface GridNioFuture<R> extends IgniteInternalFuture<R> {
     /**
      * Sets ack closure which will be applied when ack received.
      *
-     * @param closure Ack closure.
+     * @param c Ack closure.
      */
-    public void ackClosure(IgniteInClosure<IgniteException> closure);
+    public void ackClosure(IgniteInClosure<IgniteException> c);
 
     /**
      * The method will be called when ack received.

http://git-wip-us.apache.org/repos/asf/ignite/blob/05dd08b9/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 35480ac..6258c13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -35,8 +35,8 @@ public class GridNioRecoveryDescriptor {
     /** Number of acknowledged messages. */
     private long acked;
 
-    /** Unacknowledged message futures. */
-    private final ArrayDeque<GridNioFuture<?>> msgFuts;
+    /** Unacknowledged messages. */
+    private final ArrayDeque<SessionWriteRequest> msgReqs;
 
     /** Number of messages to resend. */
     private int resendCnt;
@@ -77,23 +77,40 @@ public class GridNioRecoveryDescriptor {
     /** Number of descriptor reservations (for info purposes). */
     private int reserveCnt;
 
+    /** */
+    private final boolean pairedConnections;
+
     /**
+     * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
      * @param queueLimit Maximum size of unacknowledged messages queue.
      * @param node Node.
      * @param log Logger.
      */
-    public GridNioRecoveryDescriptor(int queueLimit, ClusterNode node, IgniteLogger log) {
+    public GridNioRecoveryDescriptor(
+        boolean pairedConnections,
+        int queueLimit,
+        ClusterNode node,
+        IgniteLogger log
+    ) {
         assert !node.isLocal() : node;
         assert queueLimit > 0;
 
-        msgFuts = new ArrayDeque<>(queueLimit);
+        msgReqs = new ArrayDeque<>(queueLimit);
 
+        this.pairedConnections = pairedConnections;
         this.queueLimit = queueLimit;
         this.node = node;
         this.log = log;
     }
 
     /**
+     * @return {@code True} if in/out connections pair is used for communication with node.
+     */
+    public boolean pairedConnections() {
+        return pairedConnections;
+    }
+
+    /**
      * @return Connect count.
      */
     public long incrementConnectCount() {
@@ -154,19 +171,19 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
-     * @param fut NIO future.
+     * @param req Write request.
      * @return {@code False} if queue limit is exceeded.
      */
-    public boolean add(GridNioFuture<?> fut) {
-        assert fut != null;
+    public boolean add(SessionWriteRequest req) {
+        assert req != null;
 
-        if (!fut.skipRecovery()) {
+        if (!req.skipRecovery()) {
             if (resendCnt == 0) {
-                msgFuts.addLast(fut);
+                msgReqs.addLast(req);
 
                 sentCnt++;
 
-                return msgFuts.size() < queueLimit;
+                return msgReqs.size() < queueLimit;
             }
             else
                 resendCnt--;
@@ -181,21 +198,19 @@ public class GridNioRecoveryDescriptor {
     public void ackReceived(long rcvCnt) {
         if (log.isDebugEnabled())
             log.debug("Handle acknowledgment [acked=" + acked + ", rcvCnt=" + rcvCnt +
-                ", msgFuts=" + msgFuts.size() + ']');
+                ", msgReqs=" + msgReqs.size() + ']');
 
         while (acked < rcvCnt) {
-            GridNioFuture<?> fut = msgFuts.pollFirst();
+            SessionWriteRequest req = msgReqs.pollFirst();
 
-            assert fut != null : "Missed message future [rcvCnt=" + rcvCnt +
+            assert req != null : "Missed message [rcvCnt=" + rcvCnt +
                 ", acked=" + acked +
                 ", desc=" + this + ']';
 
-            assert fut.isDone() : fut;
-
-            if (fut.ackClosure() != null)
-                fut.ackClosure().apply(null);
+            if (req.ackClosure() != null)
+                req.ackClosure().apply(null);
 
-            fut.onAckReceived();
+            req.onAckReceived();
 
             acked++;
         }
@@ -214,7 +229,7 @@ public class GridNioRecoveryDescriptor {
      * @return {@code False} if descriptor is reserved.
      */
     public boolean onNodeLeft() {
-        GridNioFuture<?>[] futs = null;
+        SessionWriteRequest[] reqs = null;
 
         synchronized (this) {
             nodeLeft = true;
@@ -222,24 +237,24 @@ public class GridNioRecoveryDescriptor {
             if (reserved)
                 return false;
 
-            if (!msgFuts.isEmpty()) {
-                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+            if (!msgReqs.isEmpty()) {
+                reqs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
 
-                msgFuts.clear();
+                msgReqs.clear();
             }
         }
 
-        if (futs != null)
-            completeOnNodeLeft(futs);
+        if (reqs != null)
+            notifyOnNodeLeft(reqs);
 
         return true;
     }
 
     /**
-     * @return Message futures for unacknowledged messages.
+     * @return Requests for unacknowledged messages.
      */
-    public Deque<GridNioFuture<?>> messagesFutures() {
-        return msgFuts;
+    public Deque<SessionWriteRequest> messagesRequests() {
+        return msgReqs;
     }
 
     /**
@@ -277,14 +292,14 @@ public class GridNioRecoveryDescriptor {
             if (!nodeLeft)
                 ackReceived(rcvCnt);
 
-            resendCnt = msgFuts.size();
+            resendCnt = msgReqs.size();
         }
     }
 
     /**
      *
      */
-    public void connected() {
+    public void onConnected() {
         synchronized (this) {
             assert reserved : this;
             assert !connected : this;
@@ -306,10 +321,37 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
+     * @return Connected flag.
+     */
+    public boolean connected() {
+        synchronized (this) {
+            return connected;
+        }
+    }
+
+    /**
+     * @return Reserved flag.
+     */
+    public boolean reserved() {
+        synchronized (this) {
+            return reserved;
+        }
+    }
+
+    /**
+     * @return Current handshake index.
+     */
+    public Long handshakeIndex() {
+        synchronized (this) {
+            return handshakeReq != null ? handshakeReq.get1() : null;
+        }
+    }
+
+    /**
      *
      */
     public void release() {
-        GridNioFuture<?>[] futs = null;
+        SessionWriteRequest[] futs = null;
 
         synchronized (this) {
             connected = false;
@@ -329,15 +371,15 @@ public class GridNioRecoveryDescriptor {
                 notifyAll();
             }
 
-            if (nodeLeft && !msgFuts.isEmpty()) {
-                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+            if (nodeLeft && !msgReqs.isEmpty()) {
+                futs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
 
-                msgFuts.clear();
+                msgReqs.clear();
             }
         }
 
         if (futs != null)
-            completeOnNodeLeft(futs);
+            notifyOnNodeLeft(futs);
     }
 
     /**
@@ -398,16 +440,16 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
-     * @param futs Futures to complete.
+     * @param reqs Requests to notify about error.
      */
-    private void completeOnNodeLeft(GridNioFuture<?>[] futs) {
-        for (GridNioFuture<?> msg : futs) {
-            IOException e = new IOException("Failed to send message, node has left: " + node.id());
+    private void notifyOnNodeLeft(SessionWriteRequest[] reqs) {
+        IOException e = new IOException("Failed to send message, node has left: " + node.id());
 
-            ((GridNioFutureImpl)msg).onDone(e);
+        for (SessionWriteRequest req : reqs) {
+            req.onError(e);
 
-            if (msg.ackClosure() != null)
-                msg.ackClosure().apply(new IgniteException(e));
+            if (req.ackClosure() != null)
+                req.ackClosure().apply(new IgniteException(e));
         }
     }
 


[45/50] [abbrv] ignite git commit: IGNITE-4471: Fixed ODBC string-reading utility function

Posted by vo...@apache.org.
IGNITE-4471: Fixed ODBC string-reading utility function

 This closes #1377


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

Branch: refs/heads/ignite-2.0
Commit: 8cffe90010e1f0dbb0b93c33fe6a08bd27fa9477
Parents: dbc8a0f
Author: Igor Sapego <is...@gridgain.com>
Authored: Fri Dec 23 13:51:39 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 13:51:39 2016 +0300

----------------------------------------------------------------------
 .../processors/odbc/OdbcRequestHandler.java     |   2 +-
 .../internal/processors/odbc/OdbcTableMeta.java |  15 +-
 .../cpp/odbc-test/config/queries-default.xml    | 145 +++++++++++++++++++
 .../odbc-test/config/queries-test-noodbc.xml    |  84 +----------
 .../cpp/odbc-test/config/queries-test.xml       | 122 +---------------
 .../cpp/odbc-test/src/queries_test.cpp          |  76 ++++++++++
 .../cpp/odbc-test/src/utility_test.cpp          |  27 +++-
 modules/platforms/cpp/odbc/src/utility.cpp      |  10 ++
 8 files changed, 271 insertions(+), 210 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/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 2e0fd10..f922d9a 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
@@ -388,7 +388,7 @@ public class OdbcRequestHandler {
                     if (!matches("TABLE", req.tableType()))
                         continue;
 
-                    OdbcTableMeta tableMeta = new OdbcTableMeta(req.catalog(), cacheName, table.name(), "TABLE");
+                    OdbcTableMeta tableMeta = new OdbcTableMeta(null, cacheName, table.name(), "TABLE");
 
                     if (!meta.contains(tableMeta))
                         meta.add(tableMeta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
index fb7df50..ca630dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcTableMeta.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.processors.odbc;
 
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 
+import java.util.Objects;
+
 /**
  * ODBC table-related metadata.
  */
@@ -50,11 +52,11 @@ public class OdbcTableMeta {
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
-        int hash = catalog.hashCode();
+        int hash = Objects.hashCode(catalog);
 
-        hash = 31 * hash + schema.hashCode();
-        hash = 31 * hash + table.hashCode();
-        hash = 31 * hash + tableType.hashCode();
+        hash = 31 * hash + Objects.hashCode(schema);
+        hash = 31 * hash + Objects.hashCode(table);
+        hash = 31 * hash + Objects.hashCode(tableType);
 
         return hash;
     }
@@ -64,8 +66,9 @@ public class OdbcTableMeta {
         if (o instanceof OdbcTableMeta) {
             OdbcTableMeta other = (OdbcTableMeta) o;
 
-            return this == other || catalog.equals(other.catalog) && schema.equals(other.schema) &&
-                table.equals(other.table) && tableType.equals(other.tableType);
+            return this == other ||
+                    Objects.equals(catalog, other.catalog) && Objects.equals(schema, other.schema) &&
+                    Objects.equals(table, other.table) && Objects.equals(tableType, other.tableType);
         }
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-default.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-default.xml b/modules/platforms/cpp/odbc-test/config/queries-default.xml
new file mode 100644
index 0000000..6614e93
--- /dev/null
+++ b/modules/platforms/cpp/odbc-test/config/queries-default.xml
@@ -0,0 +1,145 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <bean abstract="true" id="ignite.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+        <!--<property name="odbcConfiguration"><null/></property>-->
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Long"/>
+                                <property name="valueType" value="TestType"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="i8Field" value="java.lang.Byte"/>
+                                        <entry key="i16Field" value="java.lang.Short"/>
+                                        <entry key="i32Field" value="java.lang.Integer"/>
+                                        <entry key="i64Field" value="java.lang.Long"/>
+                                        <entry key="strField" value="java.lang.String"/>
+                                        <entry key="floatField" value="java.lang.Float"/>
+                                        <entry key="doubleField" value="java.lang.Double"/>
+                                        <entry key="boolField" value="java.lang.Boolean"/>
+                                        <entry key="guidField" value="java.util.UUID"/>
+                                        <entry key="dateField" value="java.util.Date"/>
+                                        <entry key="timestampField" value="java.sql.Timestamp"/>
+                                    </map>
+                                </property>
+
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i64Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache2"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="TRANSACTIONAL"/>
+                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+
+                    <!-- Configure type metadata to enable queries. -->
+                    <property name="queryEntities">
+                        <list>
+                            <bean class="org.apache.ignite.cache.QueryEntity">
+                                <property name="keyType" value="java.lang.Long"/>
+                                <property name="valueType" value="ComplexType"/>
+
+                                <property name="fields">
+                                    <map>
+                                        <entry key="i32Field" value="java.lang.Integer"/>
+                                        <entry key="objField" value="TestObject"/>
+                                        <entry key="strField" value="java.lang.String"/>
+                                    </map>
+                                </property>
+
+                                <property name="keyFields">
+                                    <list></list>
+                                </property>
+
+                                <property name="indexes">
+                                    <list>
+                                        <bean class="org.apache.ignite.cache.QueryIndex">
+                                            <constructor-arg value="i32Field"/>
+                                        </bean>
+                                    </list>
+                                </property>
+                            </bean>
+                        </list>
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <!--
+                        Ignite provides several options for automatic discovery that can be used
+                        instead os static IP based discovery.
+                    -->
+                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+                <property name="socketTimeout" value="300" />
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
index 82173de..b21287f 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test-noodbc.xml
@@ -22,86 +22,10 @@
        xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="
         http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
-        <property name="odbcConfiguration"><null/></property>
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
 
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
+    <!-- Imports no-ODBC Ignite configuration -->
+    <import resource="queries-default.xml"/>
 
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="TestType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i8Field" value="java.lang.Byte"/>
-                                        <entry key="i16Field" value="java.lang.Short"/>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="i64Field" value="java.lang.Long"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                        <entry key="floatField" value="java.lang.Float"/>
-                                        <entry key="doubleField" value="java.lang.Double"/>
-                                        <entry key="boolField" value="java.lang.Boolean"/>
-                                        <entry key="guidField" value="java.util.UUID"/>
-                                        <entry key="dateField" value="java.util.Date"/>
-                                        <entry key="timestampField" value="java.sql.Timestamp"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i64Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery.
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-                <property name="socketTimeout" value="300" />
-            </bean>
-        </property>
-    </bean>
+    <bean parent="ignite.cfg"/>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/config/queries-test.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/config/queries-test.xml b/modules/platforms/cpp/odbc-test/config/queries-test.xml
index 906fadf..d08d4f1 100644
--- a/modules/platforms/cpp/odbc-test/config/queries-test.xml
+++ b/modules/platforms/cpp/odbc-test/config/queries-test.xml
@@ -22,129 +22,17 @@
        xmlns:util="http://www.springframework.org/schema/util"
        xsi:schemaLocation="
         http://www.springframework.org/schema/beans
-        http://www.springframework.org/schema/beans/spring-beans.xsd
-        http://www.springframework.org/schema/util
-        http://www.springframework.org/schema/util/spring-util.xsd">
-    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
-        <property name="localHost" value="127.0.0.1"/>
-        <property name="connectorConfiguration"><null/></property>
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
 
+    <!-- Imports no-ODBC Ignite configuration -->
+    <import resource="queries-default.xml"/>
+
+    <bean parent="ignite.cfg">
         <!-- Enabling ODBC. -->
         <property name="odbcConfiguration">
             <bean class="org.apache.ignite.configuration.OdbcConfiguration">
                 <property name="endpointAddress" value="127.0.0.1:11110"/>
             </bean>
         </property>
-
-        <property name="cacheConfiguration">
-            <list>
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="TestType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i8Field" value="java.lang.Byte"/>
-                                        <entry key="i16Field" value="java.lang.Short"/>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="i64Field" value="java.lang.Long"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                        <entry key="floatField" value="java.lang.Float"/>
-                                        <entry key="doubleField" value="java.lang.Double"/>
-                                        <entry key="boolField" value="java.lang.Boolean"/>
-                                        <entry key="guidField" value="java.util.UUID"/>
-                                        <entry key="dateField" value="java.util.Date"/>
-                                        <entry key="timestampField" value="java.sql.Timestamp"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i64Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-
-                <bean class="org.apache.ignite.configuration.CacheConfiguration">
-                    <property name="name" value="cache2"/>
-                    <property name="cacheMode" value="PARTITIONED"/>
-                    <property name="atomicityMode" value="TRANSACTIONAL"/>
-                    <property name="writeSynchronizationMode" value="FULL_SYNC"/>
-
-                    <!-- Configure type metadata to enable queries. -->
-                    <property name="queryEntities">
-                        <list>
-                            <bean class="org.apache.ignite.cache.QueryEntity">
-                                <property name="keyType" value="java.lang.Long"/>
-                                <property name="valueType" value="ComplexType"/>
-
-                                <property name="fields">
-                                    <map>
-                                        <entry key="i32Field" value="java.lang.Integer"/>
-                                        <entry key="objField" value="TestObject"/>
-                                        <entry key="strField" value="java.lang.String"/>
-                                    </map>
-                                </property>
-
-                                <property name="keyFields">
-                                    <list></list>
-                                </property>
-
-                                <property name="indexes">
-                                    <list>
-                                        <bean class="org.apache.ignite.cache.QueryIndex">
-                                            <constructor-arg value="i32Field"/>
-                                        </bean>
-                                    </list>
-                                </property>
-                            </bean>
-                        </list>
-                    </property>
-                </bean>
-            </list>
-        </property>
-
-        <!-- Explicitly configure TCP discovery SPI to provide list of initial nodes. -->
-        <property name="discoverySpi">
-            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
-                <property name="ipFinder">
-                    <!--
-                        Ignite provides several options for automatic discovery that can be used
-                        instead os static IP based discovery.
-                    -->
-                    <!-- Uncomment static IP finder to enable static-based discovery of initial nodes. -->
-                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
-                        <property name="addresses">
-                            <list>
-                                <!-- In distributed environment, replace with actual host IP address. -->
-                                <value>127.0.0.1:47500</value>
-                            </list>
-                        </property>
-                    </bean>
-                </property>
-                <property name="socketTimeout" value="300" />
-            </bean>
-        </property>
     </bean>
 </beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/src/queries_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/queries_test.cpp b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
index 52b885d..a304229 100644
--- a/modules/platforms/cpp/odbc-test/src/queries_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/queries_test.cpp
@@ -1342,4 +1342,80 @@ BOOST_AUTO_TEST_CASE(TestInsertMergeSelect)
     BOOST_CHECK_EQUAL(recordsNum, selectedRecordsNum);
 }
 
+template<size_t n, size_t k>
+void CheckMeta(char columns[n][k], SQLLEN columnsLen[n])
+{
+    std::string catalog(columns[0], columnsLen[0]);
+    std::string schema(columns[1], columnsLen[1]);
+    std::string table(columns[2], columnsLen[2]);
+    std::string tableType(columns[3], columnsLen[3]);
+
+    BOOST_CHECK_EQUAL(catalog, std::string(""));
+    BOOST_CHECK_EQUAL(tableType, std::string("TABLE"));
+    BOOST_CHECK_EQUAL(columnsLen[4], SQL_NULL_DATA);
+
+    if (schema == "\"cache\"")
+    {
+        BOOST_CHECK_EQUAL(table, std::string("TestType"));
+    }
+    else if (schema == "\"cache2\"")
+    {
+        BOOST_CHECK_EQUAL(table, std::string("ComplexType"));
+    }
+    else
+    {
+        BOOST_FAIL("Unknown schema: " + schema);
+    }
+}
+
+BOOST_AUTO_TEST_CASE(TestTablesMeta)
+{
+    Connect("DRIVER={Apache Ignite};ADDRESS=127.0.0.1:11110;CACHE=cache2");
+
+    SQLRETURN ret;
+
+    enum { COLUMNS_NUM = 5 };
+
+    // Five collumns: TABLE_CAT, TABLE_SCHEM, TABLE_NAME, TABLE_TYPE, REMARKS
+    char columns[COLUMNS_NUM][ODBC_BUFFER_SIZE];
+    SQLLEN columnsLen[COLUMNS_NUM];
+
+    // Binding columns.
+    for (size_t i = 0; i < COLUMNS_NUM; ++i)
+    {
+        columnsLen[i] = ODBC_BUFFER_SIZE;
+
+        ret = SQLBindCol(stmt, static_cast<SQLSMALLINT>(i + 1), SQL_C_CHAR, columns[i], columnsLen[i], &columnsLen[i]);
+
+        if (!SQL_SUCCEEDED(ret))
+            BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+    }
+
+    SQLCHAR catalogPattern[] = "";
+    SQLCHAR schemaPattern[] = "";
+    SQLCHAR tablePattern[] = "";
+    SQLCHAR tableTypePattern[] = "";
+
+    ret = SQLTables(stmt, catalogPattern, SQL_NTS, schemaPattern,
+        SQL_NTS, tablePattern, SQL_NTS, tableTypePattern, SQL_NTS);
+
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    CheckMeta<COLUMNS_NUM, ODBC_BUFFER_SIZE>(columns, columnsLen);
+
+    ret = SQLFetch(stmt);
+    if (!SQL_SUCCEEDED(ret))
+        BOOST_FAIL(GetOdbcErrorMessage(SQL_HANDLE_STMT, stmt));
+
+    CheckMeta<COLUMNS_NUM, ODBC_BUFFER_SIZE>(columns, columnsLen);
+
+    ret = SQLFetch(stmt);
+    BOOST_CHECK(ret == SQL_NO_DATA);
+}
+
 BOOST_AUTO_TEST_SUITE_END()

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc-test/src/utility_test.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc-test/src/utility_test.cpp b/modules/platforms/cpp/odbc-test/src/utility_test.cpp
index 6c4d104..a66860f 100644
--- a/modules/platforms/cpp/odbc-test/src/utility_test.cpp
+++ b/modules/platforms/cpp/odbc-test/src/utility_test.cpp
@@ -54,28 +54,43 @@ BOOST_AUTO_TEST_CASE(TestUtilityCopyStringToBuffer)
     BOOST_REQUIRE(!strcmp(buffer, str.substr(0, 10).c_str()));
 }
 
-BOOST_AUTO_TEST_CASE(TestUtilityReadString)
+BOOST_AUTO_TEST_CASE(TestUtilityWriteReadString)
 {
     using namespace ignite::impl::binary;
     using namespace ignite::impl::interop;
 
-    std::string inputStr("Hello World!");
-    std::string outputStr;
+    std::string inStr1("Hello World!");
+    std::string inStr2;
+    std::string inStr3("Lorem ipsum");
+
+    std::string outStr1;
+    std::string outStr2;
+    std::string outStr3;
+    std::string outStr4;
 
     ignite::impl::interop::InteropUnpooledMemory mem(1024);
     InteropOutputStream outStream(&mem);
     BinaryWriterImpl writer(&outStream, 0);
 
-    writer.WriteString(inputStr.data(), static_cast<int32_t>(inputStr.size()));
+    WriteString(writer, inStr1);
+    WriteString(writer, inStr2);
+    WriteString(writer, inStr3);
+    writer.WriteNull();
 
     outStream.Synchronize();
 
     InteropInputStream inStream(&mem);
     BinaryReaderImpl reader(&inStream);
 
-    ReadString(reader, outputStr);
+    ReadString(reader, outStr1);
+    ReadString(reader, outStr2);
+    ReadString(reader, outStr3);
+    ReadString(reader, outStr4);
 
-    BOOST_REQUIRE(inputStr == outputStr);
+    BOOST_REQUIRE(inStr1 == outStr1);
+    BOOST_REQUIRE(inStr2 == outStr2);
+    BOOST_REQUIRE(inStr3 == outStr3);
+    BOOST_REQUIRE(outStr4.empty());
 }
 
 BOOST_AUTO_TEST_SUITE_END()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/8cffe900/modules/platforms/cpp/odbc/src/utility.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/odbc/src/utility.cpp b/modules/platforms/cpp/odbc/src/utility.cpp
index 22191eb..756ab36 100644
--- a/modules/platforms/cpp/odbc/src/utility.cpp
+++ b/modules/platforms/cpp/odbc/src/utility.cpp
@@ -56,6 +56,7 @@ namespace ignite
         void ReadString(ignite::impl::binary::BinaryReaderImpl& reader, std::string& str)
         {
             int32_t strLen = reader.ReadString(0, 0);
+
             if (strLen > 0)
             {
                 str.resize(strLen);
@@ -63,7 +64,16 @@ namespace ignite
                 reader.ReadString(&str[0], static_cast<int32_t>(str.size()));
             }
             else
+            {
                 str.clear();
+
+                if (strLen == 0)
+                {
+                    char dummy;
+
+                    reader.ReadString(&dummy, sizeof(dummy));
+                }
+            }
         }
 
         void WriteString(ignite::impl::binary::BinaryWriterImpl& writer, const std::string & str)


[49/50] [abbrv] ignite git commit: IGNITE-4385 .NET: Allow inline AsCacheQueryable in LINQ

Posted by vo...@apache.org.
IGNITE-4385 .NET: Allow inline AsCacheQueryable in LINQ


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

Branch: refs/heads/ignite-2.0
Commit: c040c3767b3962e125aac81cb0a6c25fa565cec4
Parents: 781e5b7
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Dec 23 19:07:17 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 19:07:17 2016 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheLinqTest.cs                | 51 ++++++++++++++++++--
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |  8 +++
 2 files changed, 56 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c040c376/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
index e74f09f..798e7e8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheLinqTest.cs
@@ -425,6 +425,19 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests the cross cache join.
+        /// </summary>
+        [Test]
+        public void TestCrossCacheJoinInline()
+        {
+            var res = GetPersonCache().AsCacheQueryable().Join(GetRoleCache().AsCacheQueryable(), 
+                person => person.Key, role => role.Key.Foo, (person, role) => role).ToArray();
+
+            Assert.AreEqual(RoleCount, res.Length);
+            Assert.AreEqual(101, res[0].Key.Bar);
+        }
+
+        /// <summary>
         /// Tests the multi cache join.
         /// </summary>
         [Test]
@@ -516,7 +529,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
-        /// Tests the multiple from.
+        /// Tests query with multiple from clause.
         /// </summary>
         [Test]
         public void TestMultipleFrom()
@@ -539,6 +552,23 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests query with multiple from clause with inline query sources.
+        /// </summary>
+        [Test]
+        public void TestMultipleFromInline()
+        {
+            var filtered =
+                from person in GetPersonCache().AsCacheQueryable()
+                from role in GetRoleCache().AsCacheQueryable()
+                where person.Key == role.Key.Foo
+                select new {Person = person.Value.Name, Role = role.Value.Name};
+
+            var res = filtered.ToArray();
+
+            Assert.AreEqual(RoleCount, res.Length);
+        }
+
+        /// <summary>
         /// Tests the join of a table to itself.
         /// </summary>
         [Test]
@@ -557,6 +587,18 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
         }
 
         /// <summary>
+        /// Tests the join of a table to itself with inline queryable.
+        /// </summary>
+        [Test]
+        public void TestSelfJoinInline()
+        {
+            var qry = GetPersonCache().AsCacheQueryable().Join(GetPersonCache().AsCacheQueryable(), 
+                x => x.Value.Age, x => x.Key, (x, y) => x.Key);
+
+            Assert.AreEqual(PersonCount, qry.ToArray().Distinct().Count());
+        }
+
+        /// <summary>
         /// Tests the group by.
         /// </summary>
         [Test]
@@ -729,9 +771,11 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             var persons = GetPersonCache().AsCacheQueryable();
 
             // Invalid dateTime
-            var now = DateTime.Now;
             // ReSharper disable once ReturnValueOfPureMethodIsNotUsed
-            Assert.Throws<InvalidOperationException>(() => roles.Where(x => x.Value.Date > now).ToArray());
+            var ex = Assert.Throws<InvalidOperationException>(() =>
+                roles.Where(x => x.Value.Date > DateTime.Now).ToArray());
+            Assert.AreEqual("DateTime is not UTC. Only UTC DateTime can be used for interop with other platforms.", 
+                ex.Message);
 
             // Test retrieval
             var dates = roles.OrderBy(x => x.Value.Date).Select(x => x.Value.Date);
@@ -741,6 +785,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             // Filtering
             Assert.AreEqual(1, persons.Count(x => x.Value.Birthday == StartDateTime));
             Assert.AreEqual(PersonCount, persons.Count(x => x.Value.Birthday >= StartDateTime));
+            Assert.Greater(persons.Count(x => x.Value.Birthday > DateTime.UtcNow), 1);
 
             // Joins
             var join = 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c040c376/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
index a447bf5..4407f96 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Linq/Impl/ExpressionWalker.cs
@@ -99,6 +99,14 @@ namespace Apache.Ignite.Linq.Impl
                     return queryable;
             }
 
+            var callExpr = expression as MethodCallExpression;
+
+            if (callExpr != null)
+            {
+                // This is usually a nested query with a call to AsCacheQueryable().
+                return (ICacheQueryableInternal) Expression.Lambda(callExpr).Compile().DynamicInvoke();
+            }
+
             if (throwWhenNotFound)
                 throw new NotSupportedException("Unexpected query source: " + expression);
 


[40/50] [abbrv] ignite git commit: Web console beta-7.

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/8e7c852b/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 b123ab5..9590779 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
@@ -158,7 +158,7 @@ const PREDEFINED_QUERIES = [
 ];
 
 // Var name generator function.
-const beenNameSeed = () => {
+const beanNameSeed = () => {
     let idx = '';
     const names = [];
 
@@ -174,1551 +174,1577 @@ const beenNameSeed = () => {
     };
 };
 
-export default ['JavaTypes', 'igniteEventGroups', 'IgniteConfigurationGenerator', (JavaTypes, eventGroups, generator) => {
-    class JavaTransformer extends AbstractTransformer {
-        static generator = generator;
-
-        // Mapping for objects to method call.
-        static METHOD_MAPPING = {
-            'org.apache.ignite.configuration.CacheConfiguration': {
-                id: (ccfg) => JavaTypes.toJavaName('cache', ccfg.findProperty('name').value),
-                args: '',
-                generator: (sb, id, ccfg) => {
-                    const cacheName = ccfg.findProperty('name').value;
-                    const dataSources = JavaTransformer.collectDataSources(ccfg);
-
-                    const javadoc = [
-                        `Create configuration for cache "${cacheName}".`,
-                        '',
-                        '@return Configured cache.'
-                    ];
+export default class IgniteJavaTransformer extends AbstractTransformer {
+    // Mapping for objects to method call.
+    static METHOD_MAPPING = {
+        'org.apache.ignite.configuration.CacheConfiguration': {
+            prefix: 'cache',
+            name: 'name',
+            args: '',
+            generator: (sb, id, ccfg) => {
+                const cacheName = ccfg.findProperty('name').value;
+                const dataSources = IgniteJavaTransformer.collectDataSources(ccfg);
+
+                const javadoc = [
+                    `Create configuration for cache "${cacheName}".`,
+                    '',
+                    '@return Configured cache.'
+                ];
 
-                    if (dataSources.length)
-                        javadoc.push('@throws Exception if failed to create cache configuration.');
+                if (dataSources.length)
+                    javadoc.push('@throws Exception if failed to create cache configuration.');
 
-                    JavaTransformer.commentBlock(sb, ...javadoc);
-                    sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`);
+                IgniteJavaTransformer.commentBlock(sb, ...javadoc);
+                sb.startBlock(`public static CacheConfiguration ${id}()${dataSources.length ? ' throws Exception' : ''} {`);
 
-                    JavaTransformer.constructBean(sb, ccfg, [], true);
+                IgniteJavaTransformer.constructBean(sb, ccfg, [], true);
 
-                    sb.emptyLine();
-                    sb.append(`return ${ccfg.id};`);
+                sb.emptyLine();
+                sb.append(`return ${ccfg.id};`);
 
-                    sb.endBlock('}');
+                sb.endBlock('}');
 
-                    return sb;
-                }
-            },
-            'org.apache.ignite.cache.store.jdbc.JdbcType': {
-                id: (type) => JavaTypes.toJavaName('jdbcType', JavaTypes.shortClassName(type.findProperty('valueType').value)),
-                args: 'ccfg.getName()',
-                generator: (sb, name, jdbcType) => {
-                    const javadoc = [
-                        `Create JDBC type for "${name}".`,
-                        '',
-                        '@param cacheName Cache name.',
-                        '@return Configured JDBC type.'
-                    ];
+                return sb;
+            }
+        },
+        'org.apache.ignite.cache.store.jdbc.JdbcType': {
+            prefix: 'jdbcType',
+            name: 'valueType',
+            args: 'ccfg.getName()',
+            generator: (sb, name, jdbcType) => {
+                const javadoc = [
+                    `Create JDBC type for "${name}".`,
+                    '',
+                    '@param cacheName Cache name.',
+                    '@return Configured JDBC type.'
+                ];
 
-                    JavaTransformer.commentBlock(sb, ...javadoc);
-                    sb.startBlock(`private static JdbcType ${name}(String cacheName) {`);
+                IgniteJavaTransformer.commentBlock(sb, ...javadoc);
+                sb.startBlock(`private static JdbcType ${name}(String cacheName) {`);
 
-                    const cacheName = jdbcType.findProperty('cacheName');
+                const cacheName = jdbcType.findProperty('cacheName');
 
-                    cacheName.clsName = 'var';
-                    cacheName.value = 'cacheName';
+                cacheName.clsName = 'var';
+                cacheName.value = 'cacheName';
 
-                    JavaTransformer.constructBean(sb, jdbcType);
+                IgniteJavaTransformer.constructBean(sb, jdbcType);
 
-                    sb.emptyLine();
-                    sb.append(`return ${jdbcType.id};`);
+                sb.emptyLine();
+                sb.append(`return ${jdbcType.id};`);
 
-                    sb.endBlock('}');
+                sb.endBlock('}');
 
-                    return sb;
-                }
+                return sb;
             }
-        };
-
-        // Append comment line.
-        static comment(sb, ...lines) {
-            _.forEach(lines, (line) => sb.append(`// ${line}`));
         }
+    };
 
-        // Append comment block.
-        static commentBlock(sb, ...lines) {
-            if (lines.length === 1)
-                sb.append(`/** ${_.head(lines)} **/`);
-            else {
-                sb.append('/**');
+    // Append comment line.
+    static comment(sb, ...lines) {
+        _.forEach(lines, (line) => sb.append(`// ${line}`));
+    }
 
-                _.forEach(lines, (line) => sb.append(` * ${line}`));
+    // Append comment block.
+    static commentBlock(sb, ...lines) {
+        if (lines.length === 1)
+            sb.append(`/** ${_.head(lines)} **/`);
+        else {
+            sb.append('/**');
 
-                sb.append(' **/');
-            }
+            _.forEach(lines, (line) => sb.append(` * ${line}`));
+
+            sb.append(' **/');
         }
+    }
 
-        /**
-         * @param {Bean} bean
-         */
-        static _newBean(bean) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            if (_.isEmpty(bean.arguments))
-                return `new ${shortClsName}()`;
-
-            const args = _.map(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'MAP':
-                        return arg.id;
-                    case 'BEAN':
-                        return this._newBean(arg.value);
-                    default:
-                        return this._toObject(arg.clsName, arg.value);
-                }
-            });
+    /**
+     * @param {Bean} bean
+     */
+    static _newBean(bean) {
+        const shortClsName = this.javaTypes.shortClassName(bean.clsName);
+
+        if (_.isEmpty(bean.arguments))
+            return `new ${shortClsName}()`;
+
+        const args = _.map(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'MAP':
+                    return arg.id;
+                case 'BEAN':
+                    return this._newBean(arg.value);
+                default:
+                    return this._toObject(arg.clsName, arg.value);
+            }
+        });
 
-            if (bean.factoryMtd)
-                return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`;
+        if (bean.factoryMtd)
+            return `${shortClsName}.${bean.factoryMtd}(${args.join(', ')})`;
 
-            return `new ${shortClsName}(${args.join(', ')})`;
-        }
+        return `new ${shortClsName}(${args.join(', ')})`;
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {String} parentId
-         * @param {String} propertyName
-         * @param {String} value
-         * @private
-         */
-        static _setProperty(sb, parentId, propertyName, value) {
-            sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`);
-        }
+    /**
+     * @param {StringBuilder} sb
+     * @param {String} parentId
+     * @param {String} propertyName
+     * @param {String} value
+     * @private
+     */
+    static _setProperty(sb, parentId, propertyName, value) {
+        sb.append(`${parentId}.set${_.upperFirst(propertyName)}(${value});`);
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @param {Bean} bean
-         * @param {String} id
-
-         * @private
-         */
-        static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) {
-            _.forEach(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'MAP':
-                        this._constructMap(sb, arg, vars);
+    /**
+     * @param {StringBuilder} sb
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @param {Bean} bean
+     * @param {String} id
+
+     * @private
+     */
+    static constructBean(sb, bean, vars = [], limitLines = false, id = bean.id) {
+        _.forEach(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'MAP':
+                    this._constructMap(sb, arg, vars);
 
-                        sb.emptyLine();
+                    sb.emptyLine();
 
-                        break;
-                    default:
-                        if (this._isBean(arg.clsName) && arg.value.isComplex()) {
-                            this.constructBean(sb, arg.value, vars, limitLines);
+                    break;
+                default:
+                    if (this._isBean(arg.clsName) && arg.value.isComplex()) {
+                        this.constructBean(sb, arg.value, vars, limitLines);
 
-                            sb.emptyLine();
-                        }
-                }
-            });
+                        sb.emptyLine();
+                    }
+            }
+        });
 
-            const clsName = JavaTypes.shortClassName(bean.clsName);
+        const clsName = this.javaTypes.shortClassName(bean.clsName);
 
-            sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`);
+        sb.append(`${this.varInit(clsName, id, vars)} = ${this._newBean(bean)};`);
 
-            if (_.nonEmpty(bean.properties)) {
-                sb.emptyLine();
+        if (_.nonEmpty(bean.properties)) {
+            sb.emptyLine();
 
-                this._setProperties(sb, bean, vars, limitLines, id);
-            }
+            this._setProperties(sb, bean, vars, limitLines, id);
         }
+    }
 
-        /**
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @private
-         */
-        static constructStoreFactory(sb, bean, vars, limitLines = false) {
-            const shortClsName = JavaTypes.shortClassName(bean.clsName);
-
-            if (_.includes(vars, bean.id))
-                sb.append(`${bean.id} = ${this._newBean(bean)};`);
-            else {
-                vars.push(bean.id);
-
-                sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`);
-            }
-
-            sb.emptyLine();
+    /**
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @private
+     */
+    static constructStoreFactory(sb, bean, vars, limitLines = false) {
+        const shortClsName = this.javaTypes.shortClassName(bean.clsName);
+
+        if (_.includes(vars, bean.id))
+            sb.append(`${bean.id} = ${this._newBean(bean)};`);
+        else {
+            vars.push(bean.id);
+
+            sb.append(`${shortClsName} ${bean.id} = ${this._newBean(bean)};`);
+        }
 
-            sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory<DataSource>() {`);
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public DataSource create() {');
+        sb.emptyLine();
 
-            sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`);
+        sb.startBlock(`${bean.id}.setDataSourceFactory(new Factory<DataSource>() {`);
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public DataSource create() {');
 
-            sb.endBlock('};');
-            sb.endBlock('});');
+        sb.append(`return DataSources.INSTANCE_${bean.findProperty('dataSourceBean').id};`);
 
-            const storeFactory = _.cloneDeep(bean);
+        sb.endBlock('};');
+        sb.endBlock('});');
 
-            _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name));
+        const storeFactory = _.cloneDeep(bean);
 
-            if (storeFactory.properties.length) {
-                sb.emptyLine();
+        _.remove(storeFactory.properties, (p) => _.includes(['dataSourceBean'], p.name));
 
-                this._setProperties(sb, storeFactory, vars, limitLines);
-            }
-        }
+        if (storeFactory.properties.length) {
+            sb.emptyLine();
 
-        static _isBean(clsName) {
-            return JavaTypes.nonBuiltInClass(clsName) && JavaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+            this._setProperties(sb, storeFactory, vars, limitLines);
         }
+    }
 
-        static _toObject(clsName, val) {
-            const items = _.isArray(val) ? val : [val];
+    static _isBean(clsName) {
+        return this.javaTypes.nonBuiltInClass(clsName) && this.javaTypes.nonEnum(clsName) && _.includes(clsName, '.');
+    }
 
-            return _.map(items, (item) => {
-                if (_.isNil(item))
-                    return 'null';
+    static _toObject(clsName, val) {
+        const items = _.isArray(val) ? val : [val];
+
+        return _.map(items, (item) => {
+            if (_.isNil(item))
+                return 'null';
+
+            switch (clsName) {
+                case 'var':
+                    return item;
+                case 'byte':
+                    return `(byte) ${item}`;
+                case 'float':
+                    return `${item}f`;
+                case 'long':
+                    return `${item}L`;
+                case 'java.io.Serializable':
+                case 'java.lang.String':
+                    return `"${item.replace(/\\/g, '\\\\').replace(/"/g, '\\"')}"`;
+                case 'PATH':
+                    return `"${item.replace(/\\/g, '\\\\')}"`;
+                case 'java.lang.Class':
+                    return `${this.javaTypes.shortClassName(item)}.class`;
+                case 'java.util.UUID':
+                    return `UUID.fromString("${item}")`;
+                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())
+                            return item.id;
+
+                        return this._newBean(item);
+                    }
 
-                switch (clsName) {
-                    case 'var':
+                    if (this.javaTypes.nonEnum(clsName))
                         return item;
-                    case 'byte':
-                        return `(byte) ${item}`;
-                    case 'float':
-                        return `${item}f`;
-                    case 'long':
-                        return `${item}L`;
-                    case 'java.io.Serializable':
-                    case 'java.lang.String':
-                        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':
-                        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())
-                                return item.id;
-
-                            return this._newBean(item);
-                        }
 
-                        if (JavaTypes.nonEnum(clsName))
-                            return item;
-
-                        return `${JavaTypes.shortClassName(clsName)}.${item}`;
-                }
-            });
-        }
+                    return `${this.javaTypes.shortClassName(clsName)}.${item}`;
+            }
+        });
+    }
 
-        static _constructBeans(sb, type, items, vars, limitLines) {
-            if (this._isBean(type)) {
-                // Construct objects inline for preview or simple objects.
-                const mapper = this.METHOD_MAPPING[type];
+    static _mapperId(mapper) {
+        return (item) => this.javaTypes.toJavaName(mapper.prefix, item.findProperty(mapper.name).value);
+    }
 
-                const nextId = mapper ? mapper.id : beenNameSeed();
+    static _constructBeans(sb, type, items, vars, limitLines) {
+        if (this._isBean(type)) {
+            // Construct objects inline for preview or simple objects.
+            const mapper = this.METHOD_MAPPING[type];
 
-                // Prepare objects refs.
-                return _.map(items, (item) => {
-                    if (limitLines && mapper)
-                        return mapper.id(item) + (limitLines ? `(${mapper.args})` : '');
+            const nextId = mapper ? this._mapperId(mapper) : beanNameSeed();
 
-                    if (item.isComplex()) {
-                        const id = nextId(item);
+            // Prepare objects refs.
+            return _.map(items, (item) => {
+                if (limitLines && mapper)
+                    return nextId(item) + (limitLines ? `(${mapper.args})` : '');
 
-                        this.constructBean(sb, item, vars, limitLines, id);
+                if (item.isComplex()) {
+                    const id = nextId(item);
 
-                        sb.emptyLine();
+                    this.constructBean(sb, item, vars, limitLines, id);
 
-                        return id;
-                    }
+                    sb.emptyLine();
 
-                    return this._newBean(item);
-                });
-            }
+                    return id;
+                }
 
-            return this._toObject(type, items);
+                return this._newBean(item);
+            });
         }
 
-        /**
-         *
-         * @param sb
-         * @param parentId
-         * @param arrProp
-         * @param vars
-         * @param limitLines
-         * @private
-         */
-        static _setVarArg(sb, parentId, arrProp, vars, limitLines) {
-            const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
-
-            // Set refs to property.
-            if (refs.length === 1)
-                this._setProperty(sb, parentId, arrProp.name, _.head(refs));
-            else {
-                sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`);
-
-                const lastIdx = refs.length - 1;
-
-                _.forEach(refs, (ref, idx) => {
-                    sb.append(ref + (lastIdx !== idx ? ',' : ''));
-                });
+        return this._toObject(type, items);
+    }
 
-                sb.endBlock(');');
-            }
+    /**
+     *
+     * @param sb
+     * @param parentId
+     * @param arrProp
+     * @param vars
+     * @param limitLines
+     * @private
+     */
+    static _setVarArg(sb, parentId, arrProp, vars, limitLines) {
+        const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
+
+        // Set refs to property.
+        if (refs.length === 1)
+            this._setProperty(sb, parentId, arrProp.name, _.head(refs));
+        else {
+            sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(`);
+
+            const lastIdx = refs.length - 1;
+
+            _.forEach(refs, (ref, idx) => {
+                sb.append(ref + (lastIdx !== idx ? ',' : ''));
+            });
+
+            sb.endBlock(');');
         }
+    }
 
-        /**
-         *
-         * @param sb
-         * @param parentId
-         * @param arrProp
-         * @param vars
-         * @param limitLines
-         * @private
-         */
-        static _setArray(sb, parentId, arrProp, vars, limitLines) {
-            const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
+    /**
+     *
+     * @param sb
+     * @param parentId
+     * @param arrProp
+     * @param vars
+     * @param limitLines
+     * @private
+     */
+    static _setArray(sb, parentId, arrProp, vars, limitLines) {
+        const refs = this._constructBeans(sb, arrProp.typeClsName, arrProp.items, vars, limitLines);
 
-            const arrType = JavaTypes.shortClassName(arrProp.typeClsName);
+        const arrType = this.javaTypes.shortClassName(arrProp.typeClsName);
 
-            // Set refs to property.
-            sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`);
+        // Set refs to property.
+        sb.startBlock(`${parentId}.set${_.upperFirst(arrProp.name)}(new ${arrType}[] {`);
 
-            const lastIdx = refs.length - 1;
+        const lastIdx = refs.length - 1;
 
-            _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : '')));
+        _.forEach(refs, (ref, idx) => sb.append(ref + (lastIdx !== idx ? ',' : '')));
 
-            sb.endBlock('});');
-        }
+        sb.endBlock('});');
+    }
 
-        static _constructMap(sb, map, vars = []) {
-            const keyClsName = JavaTypes.shortClassName(map.keyClsName);
-            const valClsName = JavaTypes.shortClassName(map.valClsName);
+    static _constructMap(sb, map, vars = []) {
+        const keyClsName = this.javaTypes.shortClassName(map.keyClsName);
+        const valClsName = this.javaTypes.shortClassName(map.valClsName);
 
-            const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap';
+        const mapClsName = map.ordered ? 'LinkedHashMap' : 'HashMap';
 
-            const type = `${mapClsName}<${keyClsName}, ${valClsName}>`;
+        const type = `${mapClsName}<${keyClsName}, ${valClsName}>`;
 
-            sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`);
+        sb.append(`${this.varInit(type, map.id, vars)} = new ${mapClsName}<>();`);
 
-            sb.emptyLine();
+        sb.emptyLine();
 
-            _.forEach(map.entries, (entry) => {
-                const key = this._toObject(map.keyClsName, entry[map.keyField]);
-                const val = entry[map.valField];
+        _.forEach(map.entries, (entry) => {
+            const key = this._toObject(map.keyClsName, entry[map.keyField]);
+            const val = entry[map.valField];
 
-                if (_.isArray(val) && map.valClsName === 'java.lang.String') {
-                    if (val.length > 1) {
-                        sb.startBlock(`${map.id}.put(${key},`);
+            if (_.isArray(val) && map.valClsName === 'java.lang.String') {
+                if (val.length > 1) {
+                    sb.startBlock(`${map.id}.put(${key},`);
 
-                        _.forEach(val, (line, idx) => {
-                            sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`);
-                        });
+                    _.forEach(val, (line, idx) => {
+                        sb.append(`"${line}"${idx !== val.length - 1 ? ' +' : ''}`);
+                    });
 
-                        sb.endBlock(');');
-                    }
-                    else
-                        sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`);
+                    sb.endBlock(');');
                 }
                 else
-                    sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`);
-            });
-        }
+                    sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, _.head(val))});`);
+            }
+            else
+                sb.append(`${map.id}.put(${key}, ${this._toObject(map.valClsName, val)});`);
+        });
+    }
 
-        static varInit(type, id, vars) {
-            if (_.includes(vars, id))
-                return id;
+    static varInit(type, id, vars) {
+        if (_.includes(vars, id))
+            return id;
 
-            vars.push(id);
+        vars.push(id);
 
-            return `${type} ${id}`;
-        }
+        return `${type} ${id}`;
+    }
 
-        /**
-         *
-         * @param {StringBuilder} sb
-         * @param {Bean} bean
-         * @param {String} id
-         * @param {Array.<String>} vars
-         * @param {Boolean} limitLines
-         * @returns {StringBuilder}
-         */
-        static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) {
-            _.forEach(bean.properties, (prop, idx) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`);
-
-                        break;
-                    case 'EVENT_TYPES':
-                        if (prop.eventTypes.length === 1)
-                            this._setProperty(sb, id, prop.name, _.head(prop.eventTypes));
-                        else {
-                            sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`);
+    /**
+     *
+     * @param {StringBuilder} sb
+     * @param {Bean} bean
+     * @param {String} id
+     * @param {Array.<String>} vars
+     * @param {Boolean} limitLines
+     * @returns {StringBuilder}
+     */
+    static _setProperties(sb = new StringBuilder(), bean, vars = [], limitLines = false, id = bean.id) {
+        _.forEach(bean.properties, (prop, idx) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    this._setProperty(sb, id, 'dataSource', `DataSources.INSTANCE_${prop.id}`);
+
+                    break;
+                case 'EVENT_TYPES':
+                    if (prop.eventTypes.length === 1)
+                        this._setProperty(sb, id, prop.name, _.head(prop.eventTypes));
+                    else {
+                        sb.append(`int[] ${prop.id} = new int[${_.head(prop.eventTypes)}.length`);
+
+                        _.forEach(_.tail(prop.eventTypes), (evtGrp) => {
+                            sb.append(`    + ${evtGrp}.length`);
+                        });
 
-                            _.forEach(_.tail(prop.eventTypes), (evtGrp) => {
-                                sb.append(`    + ${evtGrp}.length`);
-                            });
+                        sb.append('];');
 
-                            sb.append('];');
+                        sb.emptyLine();
 
+                        sb.append('int k = 0;');
+
+                        _.forEach(prop.eventTypes, (evtGrp, evtIdx) => {
                             sb.emptyLine();
 
-                            sb.append('int k = 0;');
+                            sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`);
 
-                            _.forEach(prop.eventTypes, (evtGrp, evtIdx) => {
-                                sb.emptyLine();
+                            if (evtIdx < prop.eventTypes.length - 1)
+                                sb.append(`k += ${evtGrp}.length;`);
+                        });
 
-                                sb.append(`System.arraycopy(${evtGrp}, 0, ${prop.id}, k, ${evtGrp}.length);`);
+                        sb.emptyLine();
 
-                                if (evtIdx < prop.eventTypes.length - 1)
-                                    sb.append(`k += ${evtGrp}.length;`);
-                            });
+                        sb.append(`cfg.setIncludeEventTypes(${prop.id});`);
+                    }
 
-                            sb.emptyLine();
+                    break;
+                case 'ARRAY':
+                    if (prop.varArg)
+                        this._setVarArg(sb, id, prop, vars, limitLines);
+                    else
+                        this._setArray(sb, id, prop, vars, limitLines);
 
-                            sb.append(`cfg.setIncludeEventTypes(${prop.id});`);
-                        }
+                    break;
+                case 'COLLECTION':
+                    const nonBean = !this._isBean(prop.typeClsName);
 
-                        break;
-                    case 'ARRAY':
-                        if (prop.varArg)
-                            this._setVarArg(sb, id, prop, vars, limitLines);
-                        else
-                            this._setArray(sb, id, prop, vars, limitLines);
+                    if (nonBean && prop.implClsName === 'java.util.ArrayList') {
+                        const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item));
 
-                        break;
-                    case 'COLLECTION':
-                        const nonBean = !this._isBean(prop.typeClsName);
+                        if (items.length > 1) {
+                            sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`);
 
-                        if (nonBean && prop.implClsName === 'java.util.ArrayList') {
-                            const items = _.map(prop.items, (item) => this._toObject(prop.typeClsName, item));
+                            _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : '')));
 
-                            if (items.length > 1) {
-                                sb.startBlock(`${id}.set${_.upperFirst(prop.name)}(Arrays.asList(`);
+                            sb.endBlock('));');
+                        }
+                        else
+                            this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`);
+                    }
+                    else {
+                        const colTypeClsName = this.javaTypes.shortClassName(prop.typeClsName);
+                        const implClsName = this.javaTypes.shortClassName(prop.implClsName);
 
-                                _.forEach(items, (item, i) => sb.append(item + (i !== items.length - 1 ? ',' : '')));
+                        sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`);
 
-                                sb.endBlock('));');
-                            }
-                            else
-                                this._setProperty(sb, id, prop.name, `Arrays.asList(${items})`);
+                        sb.emptyLine();
+
+                        if (nonBean) {
+                            _.forEach(this._toObject(colTypeClsName, prop.items), (item) => {
+                                sb.append(`${prop.id}.add("${item}");`);
+
+                                sb.emptyLine();
+                            });
                         }
                         else {
-                            const colTypeClsName = JavaTypes.shortClassName(prop.typeClsName);
-                            const implClsName = JavaTypes.shortClassName(prop.implClsName);
+                            _.forEach(prop.items, (item) => {
+                                this.constructBean(sb, item, vars, limitLines);
 
-                            sb.append(`${this.varInit(`${implClsName}<${colTypeClsName}>`, prop.id, vars)} = new ${implClsName}<>();`);
+                                sb.append(`${prop.id}.add(${item.id});`);
 
-                            sb.emptyLine();
+                                sb.emptyLine();
+                            });
+                        }
 
-                            if (nonBean) {
-                                _.forEach(this._toObject(colTypeClsName, prop.items), (item) => {
-                                    sb.append(`${prop.id}.add("${item}");`);
+                        this._setProperty(sb, id, prop.name, prop.id);
+                    }
 
-                                    sb.emptyLine();
-                                });
-                            }
-                            else {
-                                _.forEach(prop.items, (item) => {
-                                    this.constructBean(sb, item, vars, limitLines);
+                    break;
+                case 'MAP':
+                    this._constructMap(sb, prop, vars);
 
-                                    sb.append(`${prop.id}.add(${item.id});`);
+                    if (_.nonEmpty(prop.entries))
+                        sb.emptyLine();
 
-                                    sb.emptyLine();
-                                });
-                            }
+                    this._setProperty(sb, id, prop.name, prop.id);
 
-                            this._setProperty(sb, id, prop.name, prop.id);
-                        }
+                    break;
+                case 'java.util.Properties':
+                    sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`);
 
-                        break;
-                    case 'MAP':
-                        this._constructMap(sb, prop, vars);
+                    if (_.nonEmpty(prop.entries))
+                        sb.emptyLine();
 
-                        if (_.nonEmpty(prop.entries))
-                            sb.emptyLine();
+                    _.forEach(prop.entries, (entry) => {
+                        const key = this._toObject('java.lang.String', entry.name);
+                        const val = this._toObject('java.lang.String', entry.value);
 
-                        this._setProperty(sb, id, prop.name, prop.id);
+                        sb.append(`${prop.id}.setProperty(${key}, ${val});`);
+                    });
 
-                        break;
-                    case 'java.util.Properties':
-                        sb.append(`${this.varInit('Properties', prop.id, vars)} = new Properties();`);
+                    sb.emptyLine();
 
-                        if (_.nonEmpty(prop.entries))
-                            sb.emptyLine();
+                    this._setProperty(sb, id, prop.name, prop.id);
 
-                        _.forEach(prop.entries, (entry) => {
-                            const key = this._toObject('java.lang.String', entry.name);
-                            const val = this._toObject('java.lang.String', entry.value);
+                    break;
+                case 'BEAN':
+                    const embedded = prop.value;
 
-                            sb.append(`${prop.id}.setProperty(${key}, ${val});`);
-                        });
+                    if (_.includes(STORE_FACTORY, embedded.clsName)) {
+                        this.constructStoreFactory(sb, embedded, vars, limitLines);
 
                         sb.emptyLine();
 
-                        this._setProperty(sb, id, prop.name, prop.id);
+                        this._setProperty(sb, id, prop.name, embedded.id);
+                    }
+                    else if (embedded.isComplex()) {
+                        this.constructBean(sb, embedded, vars, limitLines);
 
-                        break;
-                    case 'BEAN':
-                        const embedded = prop.value;
+                        sb.emptyLine();
 
-                        if (_.includes(STORE_FACTORY, embedded.clsName)) {
-                            this.constructStoreFactory(sb, embedded, vars, limitLines);
+                        this._setProperty(sb, id, prop.name, embedded.id);
+                    }
+                    else
+                        this._setProperty(sb, id, prop.name, this._newBean(embedded));
 
-                            sb.emptyLine();
+                    break;
+                default:
+                    this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value));
+            }
 
-                            this._setProperty(sb, id, prop.name, embedded.id);
-                        }
-                        else if (embedded.isComplex()) {
-                            this.constructBean(sb, embedded, vars, limitLines);
+            this._emptyLineIfNeeded(sb, bean.properties, idx);
+        });
 
-                            sb.emptyLine();
+        return sb;
+    }
 
-                            this._setProperty(sb, id, prop.name, embedded.id);
-                        }
-                        else
-                            this._setProperty(sb, id, prop.name, this._newBean(embedded));
+    static _collectMapImports(prop) {
+        const imports = [];
 
-                        break;
-                    default:
-                        this._setProperty(sb, id, prop.name, this._toObject(prop.clsName, prop.value));
-                }
+        imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap');
+        imports.push(prop.keyClsName);
+        imports.push(prop.valClsName);
 
-                this._emptyLineIfNeeded(sb, bean.properties, idx);
-            });
+        return imports;
+    }
 
-            return sb;
-        }
+    static collectBeanImports(bean) {
+        const imports = [bean.clsName];
 
-        static collectBeanImports(bean) {
-            const imports = [bean.clsName];
+        _.forEach(bean.arguments, (arg) => {
+            switch (arg.clsName) {
+                case 'BEAN':
+                    imports.push(...this.collectPropertiesImports(arg.value.properties));
 
-            _.forEach(bean.arguments, (arg) => {
-                switch (arg.clsName) {
-                    case 'BEAN':
-                        imports.push(...this.collectPropertiesImports(arg.value.properties));
+                    break;
+                case 'java.lang.Class':
+                    imports.push(this.javaTypes.fullClassName(arg.value));
 
-                        break;
-                    case 'java.lang.Class':
-                        imports.push(JavaTypes.fullClassName(arg.value));
+                    break;
 
-                        break;
-                    default:
-                        imports.push(arg.clsName);
-                }
-            });
+                case 'MAP':
+                    imports.push(...this._collectMapImports(arg));
 
-            imports.push(...this.collectPropertiesImports(bean.properties));
+                    break;
+                default:
+                    imports.push(arg.clsName);
+            }
+        });
 
-            if (_.includes(STORE_FACTORY, bean.clsName))
-                imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory');
+        imports.push(...this.collectPropertiesImports(bean.properties));
 
-            return imports;
-        }
+        if (_.includes(STORE_FACTORY, bean.clsName))
+            imports.push('javax.sql.DataSource', 'javax.cache.configuration.Factory');
 
-        /**
-         * @param {Array.<Object>} props
-         * @returns {Array.<String>}
-         */
-        static collectPropertiesImports(props) {
-            const imports = [];
+        return imports;
+    }
 
-            _.forEach(props, (prop) => {
-                switch (prop.clsName) {
-                    case 'DATA_SOURCE':
-                        imports.push(prop.value.clsName);
+    /**
+     * @param {Array.<Object>} props
+     * @returns {Array.<String>}
+     */
+    static collectPropertiesImports(props) {
+        const imports = [];
 
-                        break;
-                    case 'PROPERTY':
-                    case 'PROPERTY_CHAR':
-                    case 'PROPERTY_INT':
-                        imports.push('java.io.InputStream', 'java.util.Properties');
+        _.forEach(props, (prop) => {
+            switch (prop.clsName) {
+                case 'DATA_SOURCE':
+                    imports.push(prop.value.clsName);
 
-                        break;
-                    case 'BEAN':
-                        imports.push(...this.collectBeanImports(prop.value));
+                    break;
+                case 'PROPERTY':
+                case 'PROPERTY_CHAR':
+                case 'PROPERTY_INT':
+                    imports.push('java.io.InputStream', 'java.util.Properties');
 
-                        break;
-                    case 'ARRAY':
-                        imports.push(prop.typeClsName);
+                    break;
+                case 'BEAN':
+                    imports.push(...this.collectBeanImports(prop.value));
 
-                        if (this._isBean(prop.typeClsName))
-                            _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
+                    break;
+                case 'ARRAY':
+                    imports.push(prop.typeClsName);
 
-                        break;
-                    case 'COLLECTION':
-                        imports.push(prop.typeClsName);
+                    if (this._isBean(prop.typeClsName))
+                        _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
 
-                        if (this._isBean(prop.typeClsName)) {
-                            _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
+                    break;
+                case 'COLLECTION':
+                    imports.push(prop.typeClsName);
 
-                            imports.push(prop.implClsName);
-                        }
-                        else if (prop.implClsName === 'java.util.ArrayList')
-                            imports.push('java.util.Arrays');
-                        else
-                            imports.push(prop.implClsName);
-
-                        break;
-                    case 'MAP':
-                        imports.push(prop.ordered ? 'java.util.LinkedHashMap' : 'java.util.HashMap');
-                        imports.push(prop.keyClsName);
-                        imports.push(prop.valClsName);
-
-                        break;
-                    default:
-                        if (!JavaTypes.nonEnum(prop.clsName))
-                            imports.push(prop.clsName);
-                }
-            });
+                    if (this._isBean(prop.typeClsName)) {
+                        _.forEach(prop.items, (item) => imports.push(...this.collectBeanImports(item)));
 
-            return imports;
-        }
+                        imports.push(prop.implClsName);
+                    }
+                    else if (prop.implClsName === 'java.util.ArrayList')
+                        imports.push('java.util.Arrays');
+                    else
+                        imports.push(prop.implClsName);
 
-        static _prepareImports(imports) {
-            return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.'))));
-        }
+                    break;
+                case 'MAP':
+                    imports.push(...this._collectMapImports(prop));
 
-        /**
-         * @param {Bean} bean
-         * @returns {Array.<String>}
-         */
-        static collectStaticImports(bean) {
-            const imports = [];
+                    break;
+                default:
+                    if (!this.javaTypes.nonEnum(prop.clsName))
+                        imports.push(prop.clsName);
+            }
+        });
 
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'EVENT_TYPES':
-                        _.forEach(prop.eventTypes, (value) => {
-                            const evtGrp = _.find(eventGroups, {value});
+        return imports;
+    }
 
-                            imports.push(`${evtGrp.class}.${evtGrp.value}`);
-                        });
+    static _prepareImports(imports) {
+        return _.sortedUniq(_.sortBy(_.filter(imports, (cls) => !cls.startsWith('java.lang.') && _.includes(cls, '.'))));
+    }
 
-                        break;
-                    default:
-                        // No-op.
-                }
-            });
+    /**
+     * @param {Bean} bean
+     * @returns {Array.<String>}
+     */
+    static collectStaticImports(bean) {
+        const imports = [];
 
-            return imports;
-        }
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'EVENT_TYPES':
+                    _.forEach(prop.eventTypes, (value) => {
+                        const evtGrp = _.find(this.eventGroups, {value});
 
-        /**
-         * @param {Bean} bean
-         * @returns {Object}
-         */
-        static collectBeansWithMapping(bean) {
-            const beans = {};
+                        imports.push(`${evtGrp.class}.${evtGrp.value}`);
+                    });
 
-            _.forEach(bean.properties, (prop) => {
-                switch (prop.clsName) {
-                    case 'BEAN':
-                        _.merge(beans, this.collectBeansWithMapping(prop.value));
+                    break;
+                default:
+                    // No-op.
+            }
+        });
 
-                        break;
-                    case 'ARRAY':
-                        if (this._isBean(prop.typeClsName)) {
-                            const mapping = this.METHOD_MAPPING[prop.typeClsName];
+        return imports;
+    }
 
-                            _.reduce(prop.items, (acc, item) => {
-                                if (mapping) {
-                                    acc[mapping.id(item)] = item;
+    /**
+     * @param {Bean} bean
+     * @returns {Object}
+     */
+    static collectBeansWithMapping(bean) {
+        const beans = {};
 
-                                    _.merge(acc, this.collectBeansWithMapping(item));
-                                }
-                                return acc;
-                            }, beans);
-                        }
+        _.forEach(bean.properties, (prop) => {
+            switch (prop.clsName) {
+                case 'BEAN':
+                    _.merge(beans, this.collectBeansWithMapping(prop.value));
 
-                        break;
-                    default:
-                        // No-op.
-                }
-            });
+                    break;
+                case 'ARRAY':
+                    if (this._isBean(prop.typeClsName)) {
+                        const mapper = this.METHOD_MAPPING[prop.typeClsName];
 
-            return beans;
-        }
+                        const mapperId = mapper ? this._mapperId(mapper) : null;
 
-        /**
-         * Build Java startup class with configuration.
-         *
-         * @param {Bean} cfg
-         * @param pkg Package name.
-         * @param {String} clsName Class name for generate factory class otherwise generate code snippet.
-         * @param {Array.<Object>} clientNearCaches Is client node.
-         * @returns {StringBuilder}
-         */
-        static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) {
-            const sb = new StringBuilder();
-
-            sb.append(`package ${pkg};`);
-            sb.emptyLine();
+                        _.reduce(prop.items, (acc, item) => {
+                            if (mapperId)
+                                acc[mapperId(item)] = item;
 
-            const imports = this.collectBeanImports(cfg);
+                            _.merge(acc, this.collectBeansWithMapping(item));
 
-            if (_.nonEmpty(clientNearCaches))
-                imports.push('org.apache.ignite.configuration.NearCacheConfiguration');
+                            return acc;
+                        }, beans);
+                    }
 
-            if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource'))
-                imports.push('java.sql.SQLException');
+                    break;
+                default:
+                    // No-op.
+            }
+        });
 
-            const hasProps = this.hasProperties(cfg);
+        return beans;
+    }
 
-            if (hasProps)
-                imports.push('java.util.Properties', 'java.io.InputStream');
+    /**
+     * Build Java startup class with configuration.
+     *
+     * @param {Bean} cfg
+     * @param pkg Package name.
+     * @param {String} clsName Class name for generate factory class otherwise generate code snippet.
+     * @param {Array.<Object>} clientNearCaches Is client node.
+     * @returns {StringBuilder}
+     */
+    static igniteConfiguration(cfg, pkg, clsName, clientNearCaches) {
+        const sb = new StringBuilder();
 
-            _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
+        sb.append(`package ${pkg};`);
+        sb.emptyLine();
 
-            sb.emptyLine();
+        const imports = this.collectBeanImports(cfg);
 
-            const staticImports = this._prepareImports(this.collectStaticImports(cfg));
+        const nearCacheBeans = [];
 
-            if (staticImports.length) {
-                _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`));
+        if (_.nonEmpty(clientNearCaches)) {
+            imports.push('org.apache.ignite.configuration.NearCacheConfiguration');
 
-                sb.emptyLine();
-            }
+            _.forEach(clientNearCaches, (cache) => {
+                const nearCacheBean = this.generator.cacheNearClient(cache);
 
-            this.mainComment(sb);
-            sb.startBlock(`public class ${clsName} {`);
+                nearCacheBean.cacheName = cache.name;
 
-            // 2. Add external property file
-            if (hasProps) {
-                this.commentBlock(sb, 'Secret properties loading.');
-                sb.append('private static final Properties props = new Properties();');
-                sb.emptyLine();
-                sb.startBlock('static {');
-                sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {');
-                sb.append('props.load(in);');
-                sb.endBlock('}');
-                sb.startBlock('catch (Exception ignored) {');
-                sb.append('// No-op.');
-                sb.endBlock('}');
-                sb.endBlock('}');
-                sb.emptyLine();
-            }
+                imports.push(...this.collectBeanImports(nearCacheBean));
 
-            // 3. Add data sources.
-            const dataSources = this.collectDataSources(cfg);
+                nearCacheBeans.push(nearCacheBean);
+            });
+        }
 
-            if (dataSources.length) {
-                this.commentBlock(sb, 'Helper class for datasource creation.');
-                sb.startBlock('public static class DataSources {');
+        if (_.includes(imports, 'oracle.jdbc.pool.OracleDataSource'))
+            imports.push('java.sql.SQLException');
 
-                _.forEach(dataSources, (ds, idx) => {
-                    const dsClsName = JavaTypes.shortClassName(ds.clsName);
+        const hasProps = this.hasProperties(cfg);
 
-                    if (idx !== 0)
-                        sb.emptyLine();
+        if (hasProps)
+            imports.push('java.util.Properties', 'java.io.InputStream');
 
-                    sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`);
-                    sb.emptyLine();
+        _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
 
-                    sb.startBlock(`private static ${dsClsName} create${ds.id}() {`);
+        sb.emptyLine();
 
-                    if (dsClsName === 'OracleDataSource')
-                        sb.startBlock('try {');
+        const staticImports = this._prepareImports(this.collectStaticImports(cfg));
 
-                    this.constructBean(sb, ds);
+        if (staticImports.length) {
+            _.forEach(this._prepareImports(staticImports), (cls) => sb.append(`import static ${cls};`));
 
-                    sb.emptyLine();
-                    sb.append(`return ${ds.id};`);
+            sb.emptyLine();
+        }
 
-                    if (dsClsName === 'OracleDataSource') {
-                        sb.endBlock('}');
-                        sb.startBlock('catch (SQLException ex) {');
-                        sb.append('throw new Error(ex);');
-                        sb.endBlock('}');
-                    }
+        this.mainComment(sb);
+        sb.startBlock(`public class ${clsName} {`);
 
-                    sb.endBlock('}');
-                });
+        // 2. Add external property file
+        if (hasProps) {
+            this.commentBlock(sb, 'Secret properties loading.');
+            sb.append('private static final Properties props = new Properties();');
+            sb.emptyLine();
+            sb.startBlock('static {');
+            sb.startBlock('try (InputStream in = IgniteConfiguration.class.getClassLoader().getResourceAsStream("secret.properties")) {');
+            sb.append('props.load(in);');
+            sb.endBlock('}');
+            sb.startBlock('catch (Exception ignored) {');
+            sb.append('// No-op.');
+            sb.endBlock('}');
+            sb.endBlock('}');
+            sb.emptyLine();
+        }
 
-                sb.endBlock('}');
+        // 3. Add data sources.
+        const dataSources = this.collectDataSources(cfg);
+
+        if (dataSources.length) {
+            this.commentBlock(sb, 'Helper class for datasource creation.');
+            sb.startBlock('public static class DataSources {');
+
+            _.forEach(dataSources, (ds, idx) => {
+                const dsClsName = this.javaTypes.shortClassName(ds.clsName);
 
+                if (idx !== 0)
+                    sb.emptyLine();
+
+                sb.append(`public static final ${dsClsName} INSTANCE_${ds.id} = create${ds.id}();`);
                 sb.emptyLine();
-            }
 
-            _.forEach(clientNearCaches, (cache) => {
-                this.commentBlock(sb, `Configuration of near cache for cache: ${cache.name}.`,
-                    '',
-                    '@return Near cache configuration.',
-                    '@throws Exception If failed to construct near cache configuration instance.'
-                );
+                sb.startBlock(`private static ${dsClsName} create${ds.id}() {`);
 
-                const nearCacheBean = generator.cacheNearClient(cache);
+                if (dsClsName === 'OracleDataSource')
+                    sb.startBlock('try {');
 
-                sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`);
+                this.constructBean(sb, ds);
 
-                this.constructBean(sb, nearCacheBean);
                 sb.emptyLine();
+                sb.append(`return ${ds.id};`);
 
-                sb.append(`return ${nearCacheBean.id};`);
-                sb.endBlock('}');
+                if (dsClsName === 'OracleDataSource') {
+                    sb.endBlock('}');
+                    sb.startBlock('catch (SQLException ex) {');
+                    sb.append('throw new Error(ex);');
+                    sb.endBlock('}');
+                }
 
-                sb.emptyLine();
+                sb.endBlock('}');
             });
 
-            this.commentBlock(sb, 'Configure grid.',
+            sb.endBlock('}');
+
+            sb.emptyLine();
+        }
+
+        _.forEach(nearCacheBeans, (nearCacheBean) => {
+            this.commentBlock(sb, `Configuration of near cache for cache: ${nearCacheBean.cacheName}.`,
                 '',
-                '@return Ignite configuration.',
-                '@throws Exception If failed to construct Ignite configuration instance.'
+                '@return Near cache configuration.',
+                '@throws Exception If failed to construct near cache configuration instance.'
             );
-            sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {');
 
-            this.constructBean(sb, cfg, [], true);
+            sb.startBlock(`public static NearCacheConfiguration ${nearCacheBean.id}() throws Exception {`);
 
+            this.constructBean(sb, nearCacheBean);
             sb.emptyLine();
 
-            sb.append(`return ${cfg.id};`);
-
+            sb.append(`return ${nearCacheBean.id};`);
             sb.endBlock('}');
 
-            const beans = this.collectBeansWithMapping(cfg);
+            sb.emptyLine();
+        });
 
-            _.forEach(beans, (bean, id) => {
-                sb.emptyLine();
+        this.commentBlock(sb, 'Configure grid.',
+            '',
+            '@return Ignite configuration.',
+            '@throws Exception If failed to construct Ignite configuration instance.'
+        );
+        sb.startBlock('public static IgniteConfiguration createConfiguration() throws Exception {');
 
-                this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean);
-            });
+        this.constructBean(sb, cfg, [], true);
 
-            sb.endBlock('}');
+        sb.emptyLine();
 
-            return sb;
-        }
+        sb.append(`return ${cfg.id};`);
 
-        static cluster(cluster, pkg, clsName, client) {
-            const cfg = this.generator.igniteConfiguration(cluster, client);
+        sb.endBlock('}');
 
-            const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
+        const beans = this.collectBeansWithMapping(cfg);
 
-            return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches);
-        }
-
-        /**
-         * Generate source code for type by its domain model.
-         *
-         * @param fullClsName Full class name.
-         * @param fields Fields.
-         * @param addConstructor If 'true' then empty and full constructors should be generated.
-         * @returns {StringBuilder}
-         */
-        static pojo(fullClsName, fields, addConstructor) {
-            const dotIdx = fullClsName.lastIndexOf('.');
+        _.forEach(beans, (bean, id) => {
+            sb.emptyLine();
 
-            const pkg = fullClsName.substring(0, dotIdx);
-            const clsName = fullClsName.substring(dotIdx + 1);
+            this.METHOD_MAPPING[bean.clsName].generator(sb, id, bean);
+        });
 
-            const sb = new StringBuilder();
+        sb.endBlock('}');
 
-            sb.append(`package ${pkg};`);
-            sb.emptyLine();
-
-            const imports = ['java.io.Serializable'];
+        return sb;
+    }
 
-            _.forEach(fields, (field) => imports.push(JavaTypes.fullClassName(field.javaFieldType)));
+    static cluster(cluster, pkg, clsName, client) {
+        const cfg = this.generator.igniteConfiguration(cluster, client);
 
-            _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
+        const clientNearCaches = client ? _.filter(cluster.caches, (cache) => _.get(cache, 'clientNearConfiguration.enabled')) : [];
 
-            sb.emptyLine();
+        return this.igniteConfiguration(cfg, pkg, clsName, clientNearCaches);
+    }
 
-            this.mainComment(sb,
-                `${clsName} definition.`,
-                ''
-            );
-            sb.startBlock(`public class ${clsName} implements Serializable {`);
-            sb.append('/** */');
-            sb.append('private static final long serialVersionUID = 0L;');
-            sb.emptyLine();
+    /**
+     * Generate source code for type by its domain model.
+     *
+     * @param fullClsName Full class name.
+     * @param fields Fields.
+     * @param addConstructor If 'true' then empty and full constructors should be generated.
+     * @returns {StringBuilder}
+     */
+    static pojo(fullClsName, fields, addConstructor) {
+        const dotIdx = fullClsName.lastIndexOf('.');
 
-            // Generate fields declaration.
-            _.forEach(fields, (field) => {
-                const fldName = field.javaFieldName;
-                const fldType = JavaTypes.shortClassName(field.javaFieldType);
+        const pkg = fullClsName.substring(0, dotIdx);
+        const clsName = fullClsName.substring(dotIdx + 1);
 
-                sb.append(`/** Value for ${fldName}. */`);
-                sb.append(`private ${fldType} ${fldName};`);
+        const sb = new StringBuilder();
 
-                sb.emptyLine();
-            });
+        sb.append(`package ${pkg};`);
+        sb.emptyLine();
 
-            // Generate constructors.
-            if (addConstructor) {
-                this.commentBlock(sb, 'Empty constructor.');
-                sb.startBlock(`public ${clsName}() {`);
-                this.comment(sb, 'No-op.');
-                sb.endBlock('}');
+        const imports = ['java.io.Serializable'];
 
-                sb.emptyLine();
+        _.forEach(fields, (field) => imports.push(this.javaTypes.fullClassName(field.javaFieldType)));
 
-                this.commentBlock(sb, 'Full constructor.');
+        _.forEach(this._prepareImports(imports), (cls) => sb.append(`import ${cls};`));
 
-                const arg = (field) => {
-                    const fldType = JavaTypes.shortClassName(field.javaFieldType);
+        sb.emptyLine();
 
-                    return `${fldType} ${field.javaFieldName}`;
-                };
+        this.mainComment(sb,
+            `${clsName} definition.`,
+            ''
+        );
+        sb.startBlock(`public class ${clsName} implements Serializable {`);
+        sb.append('/** */');
+        sb.append('private static final long serialVersionUID = 0L;');
+        sb.emptyLine();
 
-                sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`);
+        // Generate fields declaration.
+        _.forEach(fields, (field) => {
+            const fldName = field.javaFieldName;
+            const fldType = this.javaTypes.shortClassName(field.javaFieldType);
 
-                _.forEach(_.tail(fields), (field, idx) => {
-                    sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`);
-                });
+            sb.append(`/** Value for ${fldName}. */`);
+            sb.append(`private ${fldType} ${fldName};`);
 
-                _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`));
+            sb.emptyLine();
+        });
 
-                sb.endBlock('}');
+        // Generate constructors.
+        if (addConstructor) {
+            this.commentBlock(sb, 'Empty constructor.');
+            sb.startBlock(`public ${clsName}() {`);
+            this.comment(sb, 'No-op.');
+            sb.endBlock('}');
 
-                sb.emptyLine();
-            }
+            sb.emptyLine();
 
-            // Generate getters and setters methods.
-            _.forEach(fields, (field) => {
-                const fldType = JavaTypes.shortClassName(field.javaFieldType);
-                const fldName = field.javaFieldName;
+            this.commentBlock(sb, 'Full constructor.');
 
-                this.commentBlock(sb,
-                    `Gets ${fldName}`,
-                    '',
-                    `@return Value for ${fldName}.`
-                );
-                sb.startBlock(`public ${fldType} ${JavaTypes.toJavaName('get', fldName)}() {`);
-                sb.append('return ' + fldName + ';');
-                sb.endBlock('}');
+            const arg = (field) => {
+                const fldType = this.javaTypes.shortClassName(field.javaFieldType);
 
-                sb.emptyLine();
+                return `${fldType} ${field.javaFieldName}`;
+            };
 
-                this.commentBlock(sb,
-                    `Sets ${fldName}`,
-                    '',
-                    `@param ${fldName} New value for ${fldName}.`
-                );
-                sb.startBlock(`public void ${JavaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`);
-                sb.append(`this.${fldName} = ${fldName};`);
-                sb.endBlock('}');
+            sb.startBlock(`public ${clsName}(${arg(_.head(fields))}${fields.length === 1 ? ') {' : ','}`);
 
-                sb.emptyLine();
+            _.forEach(_.tail(fields), (field, idx) => {
+                sb.append(`${arg(field)}${idx !== fields.length - 2 ? ',' : ') {'}`);
             });
 
-            // Generate equals() method.
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public boolean equals(Object o) {');
-            sb.startBlock('if (this == o)');
-            sb.append('return true;');
+            _.forEach(fields, (field) => sb.append(`this.${field.javaFieldName} = ${field.javaFieldName};`));
 
-            sb.endBlock('');
+            sb.endBlock('}');
 
-            sb.startBlock(`if (!(o instanceof ${clsName}))`);
-            sb.append('return false;');
+            sb.emptyLine();
+        }
 
-            sb.endBlock('');
+        // Generate getters and setters methods.
+        _.forEach(fields, (field) => {
+            const fldType = this.javaTypes.shortClassName(field.javaFieldType);
+            const fldName = field.javaFieldName;
 
-            sb.append(`${clsName} that = (${clsName})o;`);
+            this.commentBlock(sb,
+                `Gets ${fldName}`,
+                '',
+                `@return Value for ${fldName}.`
+            );
+            sb.startBlock(`public ${fldType} ${this.javaTypes.toJavaName('get', fldName)}() {`);
+            sb.append('return ' + fldName + ';');
+            sb.endBlock('}');
 
-            _.forEach(fields, (field) => {
-                sb.emptyLine();
+            sb.emptyLine();
 
-                const javaName = field.javaFieldName;
-                const javaType = field.javaFieldType;
+            this.commentBlock(sb,
+                `Sets ${fldName}`,
+                '',
+                `@param ${fldName} New value for ${fldName}.`
+            );
+            sb.startBlock(`public void ${this.javaTypes.toJavaName('set', fldName)}(${fldType} ${fldName}) {`);
+            sb.append(`this.${fldName} = ${fldName};`);
+            sb.endBlock('}');
 
-                switch (javaType) {
-                    case 'float':
-                        sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`);
+            sb.emptyLine();
+        });
 
-                        break;
-                    case 'double':
-                        sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`);
+        // Generate equals() method.
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public boolean equals(Object o) {');
+        sb.startBlock('if (this == o)');
+        sb.append('return true;');
 
-                        break;
-                    default:
-                        if (JavaTypes.isJavaPrimitive(javaType))
-                            sb.startBlock('if (' + javaName + ' != that.' + javaName + ')');
-                        else
-                            sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)');
-                }
+        sb.endBlock('');
 
-                sb.append('return false;');
+        sb.startBlock(`if (!(o instanceof ${clsName}))`);
+        sb.append('return false;');
 
-                sb.endBlock('');
-            });
+        sb.endBlock('');
 
-            sb.append('return true;');
-            sb.endBlock('}');
+        sb.append(`${clsName} that = (${clsName})o;`);
 
+        _.forEach(fields, (field) => {
             sb.emptyLine();
 
-            // Generate hashCode() method.
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public int hashCode() {');
+            const javaName = field.javaFieldName;
+            const javaType = field.javaFieldType;
 
-            let first = true;
-            let tempVar = false;
+            switch (javaType) {
+                case 'float':
+                    sb.startBlock(`if (Float.compare(${javaName}, that.${javaName}) != 0)`);
 
-            _.forEach(fields, (field) => {
-                const javaName = field.javaFieldName;
-                const javaType = field.javaFieldType;
+                    break;
+                case 'double':
+                    sb.startBlock(`if (Double.compare(${javaName}, that.${javaName}) != 0)`);
 
-                let fldHashCode;
+                    break;
+                default:
+                    if (this.javaTypes.isPrimitive(javaType))
+                        sb.startBlock('if (' + javaName + ' != that.' + javaName + ')');
+                    else
+                        sb.startBlock('if (' + javaName + ' != null ? !' + javaName + '.equals(that.' + javaName + ') : that.' + javaName + ' != null)');
+            }
 
-                switch (javaType) {
-                    case 'boolean':
-                        fldHashCode = `${javaName} ? 1 : 0`;
+            sb.append('return false;');
 
-                        break;
-                    case 'byte':
-                    case 'short':
-                        fldHashCode = `(int)${javaName}`;
+            sb.endBlock('');
+        });
 
-                        break;
-                    case 'int':
-                        fldHashCode = `${javaName}`;
+        sb.append('return true;');
+        sb.endBlock('}');
 
-                        break;
-                    case 'long':
-                        fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`;
+        sb.emptyLine();
 
-                        break;
-                    case 'float':
-                        fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
+        // Generate hashCode() method.
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public int hashCode() {');
 
-                        break;
-                    case 'double':
-                        sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`);
+        let first = true;
+        let tempVar = false;
 
-                        tempVar = true;
+        _.forEach(fields, (field) => {
+            const javaName = field.javaFieldName;
+            const javaType = field.javaFieldType;
 
-                        fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
+            let fldHashCode;
 
-                        break;
-                    default:
-                        fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`;
-                }
+            switch (javaType) {
+                case 'boolean':
+                    fldHashCode = `${javaName} ? 1 : 0`;
 
-                sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`);
+                    break;
+                case 'byte':
+                case 'short':
+                    fldHashCode = `(int)${javaName}`;
 
-                first = false;
+                    break;
+                case 'int':
+                    fldHashCode = `${javaName}`;
 
-                sb.emptyLine();
-            });
+                    break;
+                case 'long':
+                    fldHashCode = `(int)(${javaName} ^ (${javaName} >>> 32))`;
 
-            sb.append('return res;');
-            sb.endBlock('}');
+                    break;
+                case 'float':
+                    fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
 
-            sb.emptyLine();
+                    break;
+                case 'double':
+                    sb.append(`${tempVar ? 'ig_hash_temp' : 'long ig_hash_temp'} = Double.doubleToLongBits(${javaName});`);
 
-            this.commentBlock(sb, '{@inheritDoc}');
-            sb.startBlock('@Override public String toString() {');
-            sb.startBlock(`return "${clsName} [" + `);
+                    tempVar = true;
 
-            _.forEach(fields, (field, idx) => {
-                sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`);
-            });
+                    fldHashCode = `${javaName} != +0.0f ? Float.floatToIntBits(${javaName}) : 0`;
 
-            sb.endBlock('"]";');
-            sb.endBlock('}');
+                    break;
+                default:
+                    fldHashCode = `${javaName} != null ? ${javaName}.hashCode() : 0`;
+            }
 
-            sb.endBlock('}');
+            sb.append(first ? `int res = ${fldHashCode};` : `res = 31 * res + ${fldHashCode.startsWith('(') ? fldHashCode : `(${fldHashCode})`};`);
 
-            return sb.asString();
-        }
+            first = false;
 
-        /**
-         * Generate source code for type by its domain models.
-         *
-         * @param caches List of caches to generate POJOs for.
-         * @param addConstructor If 'true' then generate constructors.
-         * @param includeKeyFields If 'true' then include key fields into value POJO.
-         */
-        static pojos(caches, addConstructor, includeKeyFields) {
-            const pojos = [];
-
-            _.forEach(caches, (cache) => {
-                _.forEach(cache.domains, (domain) => {
-                    // Process only  domains with 'generatePojo' flag and skip already generated classes.
-                    if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) &&
-                        // Skip domain models without value fields.
-                        _.nonEmpty(domain.valueFields)) {
-                        const pojo = {};
-
-                        // Key class generation only if key is not build in java class.
-                        if (_.nonNil(domain.keyFields) && domain.keyFields.length > 0) {
-                            pojo.keyType = domain.keyType;
-                            pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor);
-                        }
+            sb.emptyLine();
+        });
 
-                        const valueFields = _.clone(domain.valueFields);
+        sb.append('return res;');
+        sb.endBlock('}');
 
-                        if (includeKeyFields) {
-                            _.forEach(domain.keyFields, (fld) => {
-                                if (!_.find(valueFields, {javaFieldName: fld.javaFieldName}))
-                                    valueFields.push(fld);
-                            });
-                        }
+        sb.emptyLine();
 
-                        pojo.valueType = domain.valueType;
-                        pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor);
+        this.commentBlock(sb, '{@inheritDoc}');
+        sb.startBlock('@Override public String toString() {');
+        sb.startBlock(`return "${clsName} [" + `);
 
-                        pojos.push(pojo);
-                    }
-                });
-            });
+        _.forEach(fields, (field, idx) => {
+            sb.append(`"${field.javaFieldName}=" + ${field.javaFieldName}${idx < fields.length - 1 ? ' + ", " + ' : ' +'}`);
+        });
 
-            return pojos;
-        }
+        sb.endBlock('"]";');
+        sb.endBlock('}');
 
-        // Generate creation and execution of cache query.
-        static _multilineQuery(sb, query, prefix, postfix) {
-            if (_.isEmpty(query))
-                return;
+        sb.endBlock('}');
 
-            _.forEach(query, (line, ix) => {
-                if (ix === 0) {
-                    if (query.length === 1)
-                        sb.append(`${prefix}"${line}"${postfix}`);
-                    else
-                        sb.startBlock(`${prefix}"${line}" +`);
+        return sb.asString();
+    }
+
+    /**
+     * Generate source code for type by its domain models.
+     *
+     * @param caches List of caches to generate POJOs for.
+     * @param addConstructor If 'true' then generate constructors.
+     * @param includeKeyFields If 'true' then include key fields into value POJO.
+     */
+    static pojos(caches, addConstructor, includeKeyFields) {
+        const pojos = [];
+
+        _.forEach(caches, (cache) => {
+            _.forEach(cache.domains, (domain) => {
+                // Process only  domains with 'generatePojo' flag and skip already generated classes.
+                if (domain.generatePojo && !_.find(pojos, {valueType: domain.valueType}) &&
+                    // Skip domain models without value fields.
+                    _.nonEmpty(domain.valueFields)) {
+                    const pojo = {
+                        keyType: domain.keyType,
+                        valueType: domain.valueType
+                    };
+
+                    // Key class generation only if key is not build in java class.
+                    if (this.javaTypes.nonBuiltInClass(domain.keyType) && _.nonEmpty(domain.keyFields))
+                        pojo.keyClass = this.pojo(domain.keyType, domain.keyFields, addConstructor);
+
+                    const valueFields = _.clone(domain.valueFields);
+
+                    if (includeKeyFields) {
+                        _.forEach(domain.keyFields, (fld) => {
+                            if (!_.find(valueFields, {javaFieldName: fld.javaFieldName}))
+                                valueFields.push(fld);
+                        });
+                    }
+
+                    pojo.valueClass = this.pojo(domain.valueType, valueFields, addConstructor);
+
+                    pojos.push(pojo);
                 }
-                else
-                    sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`);
             });
+        });
 
-            if (query.length > 1)
-                sb.endBlock('');
-            else
-                sb.emptyLine();
-        }
+        return pojos;
+    }
 
-        // Generate creation and execution of prepared statement.
-        static _prepareStatement(sb, conVar, query) {
-            this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();');
-        }
+    // Generate creation and execution of cache query.
+    static _multilineQuery(sb, query, prefix, postfix) {
+        if (_.isEmpty(query))
+            return;
 
-        static demoStartup(sb, cluster, shortFactoryCls) {
-            const cachesWithDataSource = _.filter(cluster.caches, (cache) => {
-                const kind = _.get(cache, 'cacheStoreFactory.kind');
+        _.forEach(query, (line, ix) => {
+            if (ix === 0) {
+                if (query.length === 1)
+                    sb.append(`${prefix}"${line}"${postfix}`);
+                else
+                    sb.startBlock(`${prefix}"${line}" +`);
+            }
+            else
+                sb.append(`"${line}"${ix === query.length - 1 ? postfix : ' +'}`);
+        });
 
-                if (kind) {
-                    const store = cache.cacheStoreFactory[kind];
+        if (query.length > 1)
+            sb.endBlock('');
+        else
+            sb.emptyLine();
+    }
 
-                    return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect;
-                }
+    // Generate creation and execution of prepared statement.
+    static _prepareStatement(sb, conVar, query) {
+        this._multilineQuery(sb, query, `${conVar}.prepareStatement(`, ').executeUpdate();');
+    }
 
-                return false;
-            });
+    static demoStartup(sb, cluster, shortFactoryCls) {
+        const cachesWithDataSource = _.filter(cluster.caches, (cache) => {
+            const kind = _.get(cache, 'cacheStoreFactory.kind');
 
-            const uniqDomains = [];
+            if (kind) {
+                const store = cache.cacheStoreFactory[kind];
 
-            // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache.
-            const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => {
-                const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) &&
-                    !_.includes(uniqDomains, domain));
+                return (store.connectVia === 'DataSource' || _.isNil(store.connectVia)) && store.dialect;
+            }
 
-                if (_.nonEmpty(domains)) {
-                    uniqDomains.push(...domains);
+            return false;
+        });
 
-                    acc.push({
-                        cache,
-                        domains
-                    });
-                }
+        const uniqDomains = [];
 
-                return acc;
-            }, []);
+        // Prepare array of cache and his demo domain model list. Every domain is contained only in first cache.
+        const demoTypes = _.reduce(cachesWithDataSource, (acc, cache) => {
+            const domains = _.filter(cache.domains, (domain) => _.nonEmpty(domain.valueFields) &&
+                !_.includes(uniqDomains, domain));
 
-            if (_.nonEmpty(demoTypes)) {
-                // Group domain modes by data source
-                const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean);
+            if (_.nonEmpty(domains)) {
+                uniqDomains.push(...domains);
 
-                let rndNonDefined = true;
+                acc.push({
+                    cache,
+                    domains
+                });
+            }
 
-                const generatedConsts = [];
+            return acc;
+        }, []);
 
-                _.forEach(typeByDs, (types) => {
-                    _.forEach(types, (type) => {
-                        _.forEach(type.domains, (domain) => {
-                            const valType = domain.valueType.toUpperCase();
+        if (_.nonEmpty(demoTypes)) {
+            // Group domain modes by data source
+            const typeByDs = _.groupBy(demoTypes, ({cache}) => cache.cacheStoreFactory[cache.cacheStoreFactory.kind].dataSourceBean);
 
-                            const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
+            let rndNonDefined = true;
 
-                            if (desc) {
-                                if (rndNonDefined && desc.rndRequired) {
-                                    this.commentBlock(sb, 'Random generator for demo data.');
-                                    sb.append('private static final Random rnd = new Random();');
+            const generatedConsts = [];
 
-                                    sb.emptyLine();
+            _.forEach(typeByDs, (types) => {
+                _.forEach(types, (type) => {
+                    _.forEach(type.domains, (domain) => {
+                        const valType = domain.valueType.toUpperCase();
 
-                                    rndNonDefined = false;
-                                }
+                        const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
 
-                                _.forEach(desc.insertCntConsts, (cnt) => {
-                                    if (!_.includes(generatedConsts, cnt.name)) {
-                                        this.commentBlock(sb, cnt.comment);
-                                        sb.append(`private static final int ${cnt.name} = ${cnt.val};`);
+                        if (desc) {
+                            if (rndNonDefined && desc.rndRequired) {
+                                this.commentBlock(sb, 'Random generator for demo data.');
+                                sb.append('private static final Random rnd = new Random();');
 
-                                        sb.emptyLine();
+                                sb.emptyLine();
 
-                                        generatedConsts.push(cnt.name);
-                                    }
-                                });
+                                rndNonDefined = false;
                             }
-                        });
-                    });
-                });
 
-                // Generation of fill database method
-                this.commentBlock(sb, 'Fill data for Demo.');
-                sb.startBlock('private static void prepareDemoData() throws SQLException {');
+                            _.forEach(desc.insertCntConsts, (cnt) => {
+                                if (!_.includes(generatedConsts, cnt.name)) {
+                                    this.commentBlock(sb, cnt.comment);
+                                    sb.append(`private static final int ${cnt.name} = ${cnt.val};`);
 
-                let firstDs = true;
+                                    sb.emptyLine();
 
-                _.forEach(typeByDs, (types, ds) => {
-                    const conVar = ds + 'Con';
+                                    generatedConsts.push(cnt.name);
+                                }
+                            });
+                        }
+                    });
+                });
+            });
 
-                    if (firstDs)
-                        firstDs = false;
-                    else
-                        sb.emptyLine();
+            // Generation of fill database method
+            this.commentBlock(sb, 'Fill data for Demo.');
+            sb.startBlock('private static void prepareDemoData() throws SQLException {');
 
-                    sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`);
+            let firstDs = true;
 
-                    let first = true;
-                    let stmtFirst = true;
+            _.forEach(typeByDs, (types, ds) => {
+                const conVar = ds + 'Con';
 
-                    _.forEach(types, (type) => {
-                        _.forEach(type.domains, (domain) => {
-                            const valType = domain.valueType.toUpperCase();
+                if (firstDs)
+                    firstDs = false;
+                else
+                    sb.emptyLine();
 
-                            const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
+                sb.startBlock(`try (Connection ${conVar} = ${shortFactoryCls}.DataSources.INSTANCE_${ds}.getConnection()) {`);
 
-                            if (desc) {
-                                if (first)
-                                    first = false;
-                                else
-                                    sb.emptyLine();
+                let first = true;
+                let stmtFirst = true;
 
-                                this.comment(sb, `Generate ${desc.type}.`);
+                _.forEach(types, (type) => {
+                    _.forEach(type.domains, (domain) => {
+                        const valType = domain.valueType.toUpperCase();
 
-                                if (desc.schema)
-                                    this._prepareStatement(sb, conVar, [`CREATE SCHEMA IF NOT EXISTS ${desc.schema}`]);
+                        const desc = _.find(PREDEFINED_QUERIES, (qry) => valType.endsWith(qry.type));
 
-                                this._prepareStatement(sb, conVar, desc.create);
+                        if (desc) {
+                            if (first)
+                                first = false;
+                            else
+                                sb.emptyLi

<TRUNCATED>

[48/50] [abbrv] ignite git commit: IGNITE-4415 .NET: Fix duplicate entity set handling in EntityFramework caching

Posted by vo...@apache.org.
IGNITE-4415 .NET: Fix duplicate entity set handling in EntityFramework caching


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

Branch: refs/heads/ignite-2.0
Commit: 781e5b7c93a1ee47299fc94519efbd0bea020940
Parents: 6ca8670
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Dec 23 16:53:59 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Dec 23 16:54:35 2016 +0300

----------------------------------------------------------------------
 .../EntityFrameworkCacheTest.cs                 | 54 +++++++++++++++++++-
 .../Impl/DbCommandInfo.cs                       | 21 +++++++-
 2 files changed, 72 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/781e5b7c/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
index 0e095f4..158df12 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework.Tests/EntityFrameworkCacheTest.cs
@@ -321,6 +321,55 @@ namespace Apache.Ignite.EntityFramework.Tests
         }
 
         /// <summary>
+        /// Queries with entity sets used multiple times are handled correctly.
+        /// </summary>
+        [Test]
+        public void TestDuplicateEntitySets()
+        {
+            using (var ctx = GetDbContext())
+            {
+                var blog = new Blog
+                {
+                    Name = "Foo",
+                    Posts = new List<Post>
+                    {
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Foo"},
+                        new Post {Title = "Bar"}
+                    }
+                };
+                ctx.Blogs.Add(blog);
+
+                Assert.AreEqual(5, ctx.SaveChanges());
+
+                var res = ctx.Blogs.Select(b => new
+                {
+                    X = b.Posts.FirstOrDefault(p => p.Title == b.Name),
+                    Y = b.Posts.Count(p => p.Title == b.Name)
+                }).ToArray();
+
+                Assert.AreEqual(1, res.Length);
+                Assert.AreEqual("Foo", res[0].X.Title);
+                Assert.AreEqual(3, res[0].Y);
+
+                // Modify and check updated result.
+                ctx.Posts.Remove(ctx.Posts.First(x => x.Title == "Foo"));
+                Assert.AreEqual(1, ctx.SaveChanges());
+
+                res = ctx.Blogs.Select(b => new
+                {
+                    X = b.Posts.FirstOrDefault(p => p.Title == b.Name),
+                    Y = b.Posts.Count(p => p.Title == b.Name)
+                }).ToArray();
+
+                Assert.AreEqual(1, res.Length);
+                Assert.AreEqual("Foo", res[0].X.Title);
+                Assert.AreEqual(2, res[0].Y);
+            }
+        }
+
+        /// <summary>
         /// Tests transactions created with BeginTransaction.
         /// </summary>
         [Test]
@@ -664,8 +713,11 @@ namespace Apache.Ignite.EntityFramework.Tests
         {
             TestUtils.RunMultiThreaded(CreateRemoveBlog, 4, 5);
 
+            // Run once again to force cleanup.
+            CreateRemoveBlog();
+
             // Wait for the cleanup to complete.
-            Thread.Sleep(2000);
+            Thread.Sleep(1000);
 
             // Only one version of data is in the cache.
             Assert.AreEqual(1, _cache.GetSize());

http://git-wip-us.apache.org/repos/asf/ignite/blob/781e5b7c/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
index 7f18170..6b5db05 100644
--- a/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.EntityFramework/Impl/DbCommandInfo.cs
@@ -127,6 +127,7 @@ namespace Apache.Ignite.EntityFramework.Impl
 
             expression.Accept(visitor);
 
+            // Should be sorted and unique.
             return visitor.EntitySets.ToArray();
         }
 
@@ -135,8 +136,9 @@ namespace Apache.Ignite.EntityFramework.Impl
         /// </summary>
         private class ScanExpressionVisitor : BasicCommandTreeVisitor
         {
-            /** */
-            private readonly List<EntitySetBase> _entitySets = new List<EntitySetBase>();
+            /** Unique and sorted entity sets. */
+            private readonly SortedSet<EntitySetBase> _entitySets = 
+                new SortedSet<EntitySetBase>(EntitySetComparer.Instance);
 
             /// <summary>
             /// Gets the entity sets.
@@ -154,5 +156,20 @@ namespace Apache.Ignite.EntityFramework.Impl
                 base.Visit(expression);
             }
         }
+
+        /// <summary>
+        /// Compares entity sets by name.
+        /// </summary>
+        private class EntitySetComparer : IComparer<EntitySetBase>
+        {
+            /** Default instance. */
+            public static readonly EntitySetComparer Instance = new EntitySetComparer();
+
+            /** <inheritdoc /> */
+            public int Compare(EntitySetBase x, EntitySetBase y)
+            {
+                return string.CompareOrdinal(x.Name, y.Name);
+            }
+        }
     }
 }


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

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

# Conflicts:
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
#	modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
#	modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateInvokeRequest.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
#	modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
#	modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
#	modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
#	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/ignite-2.0
Commit: 64247b9228451e46abb8029e09c7fc6ed4e16d2d
Parents: 147277d 8dd4ada
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 19 15:54:39 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 19 15:54:39 2016 +0300

----------------------------------------------------------------------
 .../store/cassandra/CassandraCacheStore.java    |   9 +-
 .../store/cassandra/datasource/DataSource.java  |   9 +
 .../rest/RestProcessorMultiStartSelfTest.java   |  48 +-
 .../java/org/apache/ignite/IgniteServices.java  |  16 +
 .../apache/ignite/IgniteSystemProperties.java   |   6 +
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++-
 .../ignite/cache/store/CacheStoreAdapter.java   |   6 +
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |  19 +-
 .../store/jdbc/JdbcTypesDefaultTransformer.java | 112 ++--
 .../apache/ignite/internal/IgniteKernal.java    |  28 +-
 .../ignite/internal/IgniteServicesImpl.java     |   9 +-
 .../internal/binary/BinaryClassDescriptor.java  |  12 +-
 .../ignite/internal/binary/BinaryUtils.java     |  10 +-
 .../binary/builder/BinaryObjectBuilderImpl.java |  11 +-
 .../discovery/GridDiscoveryManager.java         | 118 +---
 .../affinity/GridAffinityProcessor.java         |   2 +-
 .../processors/cache/CacheLockCandidates.java   |  42 ++
 .../cache/CacheLockCandidatesList.java          |  71 +++
 .../cache/CacheStoreBalancingWrapper.java       |   6 +
 .../processors/cache/GridCacheAdapter.java      |   8 +-
 .../processors/cache/GridCacheEntryEx.java      |   3 +-
 .../cache/GridCacheLoaderWriterStore.java       |   6 +
 .../processors/cache/GridCacheMapEntry.java     | 117 +++-
 .../processors/cache/GridCacheMvcc.java         | 376 +++++++----
 .../processors/cache/GridCacheMvccCallback.java |   4 +-
 .../cache/GridCacheMvccCandidate.java           |  80 +--
 .../processors/cache/GridCacheMvccManager.java  |  19 +-
 .../GridCachePartitionExchangeManager.java      | 157 ++---
 .../processors/cache/GridCachePreloader.java    |  11 +-
 .../cache/GridCachePreloaderAdapter.java        |   5 +-
 .../processors/cache/GridCacheProcessor.java    |  10 +-
 .../processors/cache/GridCacheUtils.java        |  17 -
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../CacheDataStructuresManager.java             |   6 +-
 .../distributed/GridDistributedCacheEntry.java  | 303 +++------
 .../dht/GridClientPartitionTopology.java        | 120 ++--
 .../distributed/dht/GridDhtCacheEntry.java      |  32 +-
 .../distributed/dht/GridDhtLockFuture.java      |  34 +-
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 +++++----
 .../dht/GridDhtTransactionalCacheAdapter.java   |   1 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |   5 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   8 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 230 ++++---
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  56 +-
 .../dht/preloader/GridDhtPreloader.java         |   9 +-
 .../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      |  22 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../cache/transactions/IgniteTxManager.java     |   5 +-
 .../closure/GridClosureProcessor.java           |  31 +-
 .../internal/processors/job/GridJobWorker.java  |  76 ++-
 .../processors/odbc/OdbcRequestHandler.java     |  14 +-
 .../platform/PlatformContextImpl.java           |   2 +-
 .../dotnet/PlatformDotNetCacheStore.java        |  11 +
 .../platform/services/PlatformServices.java     |   2 +-
 .../platform/utils/PlatformUtils.java           |  28 +
 .../processors/rest/GridRestProcessor.java      |  15 +
 .../service/GridServiceProcessor.java           |  15 +-
 .../processors/service/GridServiceProxy.java    |  18 +-
 .../processors/task/GridTaskWorker.java         |   7 +
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/marshaller/jdk/JdkMarshaller.java    |   4 +-
 .../optimized/OptimizedMarshaller.java          |   8 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  41 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   4 +
 .../resources/META-INF/classnames.properties    |  86 ++-
 .../AbstractAffinityFunctionSelfTest.java       |   2 +-
 .../jdbc/JdbcTypesDefaultTransformerTest.java   | 283 +++++++++
 .../IgniteComputeTopologyExceptionTest.java     |   5 +-
 .../binary/BinaryMarshallerSelfTest.java        |  66 ++
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +-
 .../CacheSerializableTransactionsTest.java      | 604 +++++++++++++++++-
 .../cache/GridCacheMvccFlagsTest.java           |   8 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java | 334 ++++++++--
 .../processors/cache/GridCacheMvccSelfTest.java | 212 +++----
 .../GridCachePartitionedAffinitySpreadTest.java |   7 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  77 +--
 ...heapCacheMetricsForClusterGroupSelfTest.java | 141 +++++
 .../cache/OffheapCacheOnClientsTest.java        | 143 +++++
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 -------
 .../GridCacheRebalancingSyncSelfTest.java       |   2 +
 .../CacheOffHeapAndSwapMetricsSelfTest.java     | 621 -------------------
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java | 621 +++++++++++++++++++
 .../closure/GridClosureSerializationTest.java   | 177 ++++++
 ...gniteServiceProxyTimeoutInitializedTest.java | 284 +++++++++
 .../loadtests/hashmap/GridHashMapLoadTest.java  |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../IgniteCacheMetricsSelfTestSuite.java        |   6 +-
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +
 .../resources/META-INF/classnames.properties    | 114 ++++
 .../processors/query/h2/IgniteH2Indexing.java   |  77 ++-
 .../h2/twostep/GridReduceQueryExecutor.java     |  14 +-
 ...niteCachePartitionedFieldsQuerySelfTest.java |  25 +
 101 files changed, 4783 insertions(+), 2473 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
----------------------------------------------------------------------
diff --cc modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
index 9058837,0000000..b4bed0d
mode 100644,000000..100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/CassandraCacheStore.java
@@@ -1,519 -1,0 +1,522 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *      http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.ignite.cache.store.cassandra;
 +
 +import com.datastax.driver.core.BoundStatement;
 +import com.datastax.driver.core.PreparedStatement;
 +import com.datastax.driver.core.Row;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.Map;
 +import java.util.HashMap;
 +import java.util.concurrent.ExecutorService;
 +import java.util.concurrent.Executors;
 +import java.util.concurrent.Future;
 +import javax.cache.Cache;
 +import javax.cache.integration.CacheLoaderException;
 +import javax.cache.integration.CacheWriterException;
 +import org.apache.ignite.IgniteCheckedException;
 +import org.apache.ignite.IgniteLogger;
 +import org.apache.ignite.cache.store.CacheStore;
 +import org.apache.ignite.cache.store.CacheStoreSession;
 +import org.apache.ignite.cache.store.cassandra.datasource.DataSource;
 +import org.apache.ignite.cache.store.cassandra.persistence.KeyValuePersistenceSettings;
 +import org.apache.ignite.cache.store.cassandra.persistence.PersistenceController;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 +import org.apache.ignite.cache.store.cassandra.session.ExecutionAssistant;
 +import org.apache.ignite.cache.store.cassandra.session.GenericBatchExecutionAssistant;
 +import org.apache.ignite.cache.store.cassandra.session.LoadCacheCustomQueryWorker;
- import org.apache.ignite.cache.store.cassandra.session.transaction.DeleteMutation;
- import org.apache.ignite.cache.store.cassandra.session.transaction.Mutation;
- import org.apache.ignite.cache.store.cassandra.session.transaction.WriteMutation;
++import org.apache.ignite.internal.util.typedef.internal.S;
 +import org.apache.ignite.internal.util.typedef.internal.U;
 +import org.apache.ignite.lang.IgniteBiInClosure;
 +import org.apache.ignite.logger.NullLogger;
 +import org.apache.ignite.resources.CacheStoreSessionResource;
 +import org.apache.ignite.resources.LoggerResource;
 +
 +/**
 + * Implementation of {@link CacheStore} backed by Cassandra database.
 + *
 + * @param <K> Ignite cache key type.
 + * @param <V> Ignite cache value type.
 + */
 +public class CassandraCacheStore<K, V> implements CacheStore<K, V> {
 +    /** Buffer to store mutations performed withing transaction. */
 +    private static final String TRANSACTION_BUFFER = "CASSANDRA_TRANSACTION_BUFFER";
 +
 +    /** Auto-injected store session. */
 +    @SuppressWarnings("unused")
 +    @CacheStoreSessionResource
 +    private CacheStoreSession storeSes;
 +
 +    /** Auto-injected logger instance. */
 +    @SuppressWarnings("unused")
 +    @LoggerResource
 +    private IgniteLogger log;
 +
 +    /** Cassandra data source. */
 +    private DataSource dataSrc;
 +
 +    /** Max workers thread count. These threads are responsible for load cache. */
 +    private int maxPoolSize = Runtime.getRuntime().availableProcessors();
 +
 +    /** Controller component responsible for serialization logic. */
 +    private final PersistenceController controller;
 +
 +    /**
 +     * Store constructor.
 +     *
 +     * @param dataSrc Data source.
 +     * @param settings Persistence settings for Ignite key and value objects.
 +     * @param maxPoolSize Max workers thread count.
 +     */
 +    public CassandraCacheStore(DataSource dataSrc, KeyValuePersistenceSettings settings, int maxPoolSize) {
 +        this.dataSrc = dataSrc;
 +        this.controller = new PersistenceController(settings);
 +        this.maxPoolSize = maxPoolSize;
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void loadCache(IgniteBiInClosure<K, V> clo, Object... args) throws CacheLoaderException {
 +        if (clo == null)
 +            return;
 +
 +        if (args == null || args.length == 0)
 +            args = new String[] {"select * from " + controller.getPersistenceSettings().getKeyspace() + "." + cassandraTable() + ";"};
 +
 +        ExecutorService pool = null;
 +
 +        Collection<Future<?>> futs = new ArrayList<>(args.length);
 +
 +        try {
 +            pool = Executors.newFixedThreadPool(maxPoolSize);
 +
 +            CassandraSession ses = getCassandraSession();
 +
 +            for (Object obj : args) {
 +                if (obj == null || !(obj instanceof String) || !((String)obj).trim().toLowerCase().startsWith("select"))
 +                    continue;
 +
 +                futs.add(pool.submit(new LoadCacheCustomQueryWorker<>(ses, (String) obj, controller, log, clo)));
 +            }
 +
 +            for (Future<?> fut : futs)
 +                U.get(fut);
 +
 +            if (log != null && log.isDebugEnabled() && storeSes != null)
 +                log.debug("Cache loaded from db: " + storeSes.cacheName());
 +        }
 +        catch (IgniteCheckedException e) {
 +            if (storeSes != null)
 +                throw new CacheLoaderException("Failed to load Ignite cache: " + storeSes.cacheName(), e.getCause());
 +            else
 +                throw new CacheLoaderException("Failed to load cache", e.getCause());
 +        }
 +        finally {
 +            U.shutdownNow(getClass(), pool, log);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void sessionEnd(boolean commit) throws CacheWriterException {
 +        if (!storeSes.isWithinTransaction())
 +            return;
 +
 +        List<Mutation> mutations = mutations();
 +        if (mutations == null || mutations.isEmpty())
 +            return;
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(mutations);
 +        }
 +        finally {
 +            mutations.clear();
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings({"unchecked"})
 +    @Override public V load(final K key) throws CacheLoaderException {
 +        if (key == null)
 +            return null;
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            return ses.execute(new ExecutionAssistant<V>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return false;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getLoadStatement(cassandraTable(), false);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "READ";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public V process(Row row) {
 +                    return row == null ? null : (V)controller.buildValueObject(row);
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings("unchecked")
 +    @Override public Map<K, V> loadAll(Iterable<? extends K> keys) throws CacheLoaderException {
 +        if (keys == null || !keys.iterator().hasNext())
 +            return new HashMap<>();
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            return ses.execute(new GenericBatchExecutionAssistant<Map<K, V>, K>() {
 +                private Map<K, V> data = new HashMap<>();
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getLoadStatement(cassandraTable(), true);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override  public BoundStatement bindStatement(PreparedStatement statement, K key) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_READ";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Map<K, V> processedData() {
 +                    return data;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override protected void process(Row row) {
 +                    data.put((K)controller.buildKeyObject(row), (V)controller.buildValueObject(row));
 +                }
 +            }, keys);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void write(final Cache.Entry<? extends K, ? extends V> entry) throws CacheWriterException {
 +        if (entry == null || entry.getKey() == null)
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            accumulate(new WriteMutation(entry, cassandraTable(), controller));
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new ExecutionAssistant<Void>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return true;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getWriteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "WRITE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Void process(Row row) {
 +                    return null;
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void writeAll(Collection<Cache.Entry<? extends K, ? extends V>> entries) throws CacheWriterException {
 +        if (entries == null || entries.isEmpty())
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            for (Cache.Entry<?, ?> entry : entries)
 +                accumulate(new WriteMutation(entry, cassandraTable(), controller));
 +
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new GenericBatchExecutionAssistant<Void, Cache.Entry<? extends K, ? extends V>>() {
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getWriteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement,
 +                    Cache.Entry<? extends K, ? extends V> entry) {
 +                    return controller.bindKeyValue(statement, entry.getKey(), entry.getValue());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_WRITE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return true;
 +                }
 +            }, entries);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void delete(final Object key) throws CacheWriterException {
 +        if (key == null)
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            accumulate(new DeleteMutation(key, cassandraTable(), controller));
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new ExecutionAssistant<Void>() {
 +                /** {@inheritDoc} */
 +                @Override public boolean tableExistenceRequired() {
 +                    return false;
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getDeleteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "DELETE";
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public Void process(Row row) {
 +                    return null;
 +                }
 +            });
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void deleteAll(Collection<?> keys) throws CacheWriterException {
 +        if (keys == null || keys.isEmpty())
 +            return;
 +
 +        if (storeSes.isWithinTransaction()) {
 +            for (Object key : keys)
 +                accumulate(new DeleteMutation(key, cassandraTable(), controller));
 +
 +            return;
 +        }
 +
 +        CassandraSession ses = getCassandraSession();
 +
 +        try {
 +            ses.execute(new GenericBatchExecutionAssistant<Void, Object>() {
 +                /** {@inheritDoc} */
 +                @Override public String getTable() {
 +                    return cassandraTable();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String getStatement() {
 +                    return controller.getDeleteStatement(cassandraTable());
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public BoundStatement bindStatement(PreparedStatement statement, Object key) {
 +                    return controller.bindKey(statement, key);
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public KeyValuePersistenceSettings getPersistenceSettings() {
 +                    return controller.getPersistenceSettings();
 +                }
 +
 +                /** {@inheritDoc} */
 +                @Override public String operationName() {
 +                    return "BULK_DELETE";
 +                }
 +            }, keys);
 +        }
 +        finally {
 +            U.closeQuiet(ses);
 +        }
 +    }
 +
 +    /**
 +     * Gets Cassandra session wrapper or creates new if it doesn't exist.
 +     * This wrapper hides all the low-level Cassandra interaction details by providing only high-level methods.
 +     *
 +     * @return Cassandra session wrapper.
 +     */
 +    private CassandraSession getCassandraSession() {
 +        return dataSrc.session(log != null ? log : new NullLogger());
 +    }
 +
 +    /**
 +     * Returns table name to use for all Cassandra based operations (READ/WRITE/DELETE).
 +     *
 +     * @return Table name.
 +     */
 +    private String cassandraTable() {
 +        return controller.getPersistenceSettings().getTable() != null ?
 +            controller.getPersistenceSettings().getTable() : storeSes.cacheName().trim().toLowerCase();
 +    }
 +
 +    /**
 +     * Accumulates mutation in the transaction buffer.
 +     *
 +     * @param mutation Mutation operation.
 +     */
 +    private void accumulate(Mutation mutation) {
 +        //noinspection unchecked
 +        List<Mutation> mutations = (List<Mutation>)storeSes.properties().get(TRANSACTION_BUFFER);
 +
 +        if (mutations == null) {
 +            mutations = new LinkedList<>();
 +            storeSes.properties().put(TRANSACTION_BUFFER, mutations);
 +        }
 +
 +        mutations.add(mutation);
 +    }
 +
 +    /**
 +     * Returns all the mutations performed withing transaction.
 +     *
 +     * @return Mutations
 +     */
 +    private List<Mutation> mutations() {
 +        //noinspection unchecked
 +        return (List<Mutation>)storeSes.properties().get(TRANSACTION_BUFFER);
 +    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(CassandraCacheStore.class, this);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
----------------------------------------------------------------------
diff --cc modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
index f582aac,0000000..1ba3c7d
mode 100644,000000..100644
--- a/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
+++ b/modules/cassandra/store/src/main/java/org/apache/ignite/cache/store/cassandra/datasource/DataSource.java
@@@ -1,647 -1,0 +1,656 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one or more
 + * contributor license agreements.  See the NOTICE file distributed with
 + * this work for additional information regarding copyright ownership.
 + * The ASF licenses this file to You under the Apache License, Version 2.0
 + * (the "License"); you may not use this file except in compliance with
 + * the License.  You may obtain a copy of the License at
 + *
 + *      http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +
 +package org.apache.ignite.cache.store.cassandra.datasource;
 +
 +import com.datastax.driver.core.AuthProvider;
 +import com.datastax.driver.core.Cluster;
 +import com.datastax.driver.core.ConsistencyLevel;
 +import com.datastax.driver.core.NettyOptions;
 +import com.datastax.driver.core.PoolingOptions;
 +import com.datastax.driver.core.ProtocolOptions;
 +import com.datastax.driver.core.ProtocolVersion;
 +import com.datastax.driver.core.SSLOptions;
 +import com.datastax.driver.core.SocketOptions;
 +import com.datastax.driver.core.policies.AddressTranslator;
 +import com.datastax.driver.core.policies.LoadBalancingPolicy;
 +import com.datastax.driver.core.policies.ReconnectionPolicy;
 +import com.datastax.driver.core.policies.RetryPolicy;
 +import com.datastax.driver.core.policies.SpeculativeExecutionPolicy;
 +
 +import java.io.Externalizable;
 +import java.io.IOException;
 +import java.io.ObjectInput;
 +import java.io.ObjectOutput;
 +import java.io.Serializable;
 +import java.net.InetAddress;
 +import java.net.InetSocketAddress;
 +import java.util.LinkedList;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.ignite.IgniteException;
 +import org.apache.ignite.IgniteLogger;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSession;
 +import org.apache.ignite.cache.store.cassandra.session.CassandraSessionImpl;
 +import org.apache.ignite.internal.util.typedef.internal.U;
++import org.apache.ignite.internal.util.tostring.GridToStringExclude;
++import org.apache.ignite.internal.util.typedef.internal.S;
 +
 +/**
 + * Data source abstraction to specify configuration of the Cassandra session to be used.
 + */
 +public class DataSource implements Externalizable {
 +    /** */
 +    private static final long serialVersionUID = 0L;
 +
 +    /**
 +     * Null object, used as a replacement for those Cassandra connection options which
 +     * don't support serialization (RetryPolicy, LoadBalancingPolicy and etc).
 +     */
 +    private static final UUID NULL_OBJECT = UUID.fromString("45ffae47-3193-5910-84a2-048fe65735d9");
 +
 +    /** Number of rows to immediately fetch in CQL statement execution. */
 +    private Integer fetchSize;
 +
 +    /** Consistency level for READ operations. */
 +    private ConsistencyLevel readConsistency;
 +
 +    /** Consistency level for WRITE operations. */
 +    private ConsistencyLevel writeConsistency;
 +
 +    /** Username to use for authentication. */
++    @GridToStringExclude
 +    private String user;
 +
 +    /** Password to use for authentication. */
++    @GridToStringExclude
 +    private String pwd;
 +
 +    /** Port to use for Cassandra connection. */
 +    private Integer port;
 +
 +    /** List of contact points to connect to Cassandra cluster. */
 +    private List<InetAddress> contactPoints;
 +
 +    /** List of contact points with ports to connect to Cassandra cluster. */
 +    private List<InetSocketAddress> contactPointsWithPorts;
 +
 +    /** Maximum time to wait for schema agreement before returning from a DDL query. */
 +    private Integer maxSchemaAgreementWaitSeconds;
 +
 +    /** The native protocol version to use. */
 +    private Integer protoVer;
 +
 +    /** Compression to use for the transport. */
 +    private String compression;
 +
 +    /** Use SSL for communications with Cassandra. */
 +    private Boolean useSSL;
 +
 +    /** Enables metrics collection. */
 +    private Boolean collectMetrix;
 +
 +    /** Enables JMX reporting of the metrics. */
 +    private Boolean jmxReporting;
 +
 +    /** Credentials to use for authentication. */
 +    private Credentials creds;
 +
 +    /** Load balancing policy to use. */
 +    private LoadBalancingPolicy loadBalancingPlc;
 +
 +    /** Reconnection policy to use. */
 +    private ReconnectionPolicy reconnectionPlc;
 +
 +    /** Retry policy to use. */
 +    private RetryPolicy retryPlc;
 +
 +    /** Address translator to use. */
 +    private AddressTranslator addrTranslator;
 +
 +    /** Speculative execution policy to use. */
 +    private SpeculativeExecutionPolicy speculativeExecutionPlc;
 +
 +    /** Authentication provider to use. */
 +    private AuthProvider authProvider;
 +
 +    /** SSL options to use. */
 +    private SSLOptions sslOptions;
 +
 +    /** Connection pooling options to use. */
 +    private PoolingOptions poolingOptions;
 +
 +    /** Socket options to use. */
 +    private SocketOptions sockOptions;
 +
 +    /** Netty options to use for connection. */
 +    private NettyOptions nettyOptions;
 +
 +    /** Cassandra session wrapper instance. */
 +    private volatile CassandraSession ses;
 +
 +    /**
 +     * Sets user name to use for authentication.
 +     *
 +     * @param user user name
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setUser(String user) {
 +        this.user = user;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets password to use for authentication.
 +     *
 +     * @param pwd password
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPassword(String pwd) {
 +        this.pwd = pwd;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets port to use for Cassandra connection.
 +     *
 +     * @param port port
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPort(int port) {
 +        this.port = port;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets list of contact points to connect to Cassandra cluster.
 +     *
 +     * @param points contact points
 +     */
 +    public void setContactPoints(String... points) {
 +        if (points == null || points.length == 0)
 +            return;
 +
 +        for (String point : points) {
 +            if (point.contains(":")) {
 +                if (contactPointsWithPorts == null)
 +                    contactPointsWithPorts = new LinkedList<>();
 +
 +                String[] chunks = point.split(":");
 +
 +                try {
 +                    contactPointsWithPorts.add(InetSocketAddress.createUnresolved(chunks[0].trim(), Integer.parseInt(chunks[1].trim())));
 +                }
 +                catch (Throwable e) {
 +                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
 +                }
 +            }
 +            else {
 +                if (contactPoints == null)
 +                    contactPoints = new LinkedList<>();
 +
 +                try {
 +                    contactPoints.add(InetAddress.getByName(point));
 +                }
 +                catch (Throwable e) {
 +                    throw new IllegalArgumentException("Incorrect contact point '" + point + "' specified for Cassandra cache storage", e);
 +                }
 +            }
 +        }
 +
 +        invalidate();
 +    }
 +
 +    /** Sets maximum time to wait for schema agreement before returning from a DDL query. */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setMaxSchemaAgreementWaitSeconds(int seconds) {
 +        maxSchemaAgreementWaitSeconds = seconds;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets the native protocol version to use.
 +     *
 +     * @param ver version number
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setProtocolVersion(int ver) {
 +        protoVer = ver;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets compression algorithm to use for the transport.
 +     *
 +     * @param compression Compression algorithm.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setCompression(String compression) {
 +        this.compression = compression == null || compression.trim().isEmpty() ? null : compression.trim();
 +
 +        try {
 +            if (this.compression != null)
 +                ProtocolOptions.Compression.valueOf(this.compression);
 +        }
 +        catch (Throwable e) {
 +            throw new IgniteException("Incorrect compression '" + compression + "' specified for Cassandra connection", e);
 +        }
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables SSL for communications with Cassandra.
 +     *
 +     * @param use Flag to enable/disable SSL.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setUseSSL(boolean use) {
 +        useSSL = use;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables metrics collection.
 +     *
 +     * @param collect Flag to enable/disable metrics collection.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setCollectMetrix(boolean collect) {
 +        collectMetrix = collect;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Enables JMX reporting of the metrics.
 +     *
 +     * @param enableReporting Flag to enable/disable JMX reporting.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setJmxReporting(boolean enableReporting) {
 +        jmxReporting = enableReporting;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets number of rows to immediately fetch in CQL statement execution.
 +     *
 +     * @param size Number of rows to fetch.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setFetchSize(int size) {
 +        fetchSize = size;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Set consistency level for READ operations.
 +     *
 +     * @param level Consistency level.
 +     */
 +    public void setReadConsistency(String level) {
 +        readConsistency = parseConsistencyLevel(level);
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Set consistency level for WRITE operations.
 +     *
 +     * @param level Consistency level.
 +     */
 +    public void setWriteConsistency(String level) {
 +        writeConsistency = parseConsistencyLevel(level);
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets credentials to use for authentication.
 +     *
 +     * @param creds Credentials.
 +     */
 +    public void setCredentials(Credentials creds) {
 +        this.creds = creds;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets load balancing policy.
 +     *
 +     * @param plc Load balancing policy.
 +     */
 +    public void setLoadBalancingPolicy(LoadBalancingPolicy plc) {
 +        loadBalancingPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets reconnection policy.
 +     *
 +     * @param plc Reconnection policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setReconnectionPolicy(ReconnectionPolicy plc) {
 +        reconnectionPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets retry policy.
 +     *
 +     * @param plc Retry policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setRetryPolicy(RetryPolicy plc) {
 +        retryPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets address translator.
 +     *
 +     * @param translator Address translator.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setAddressTranslator(AddressTranslator translator) {
 +        addrTranslator = translator;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets speculative execution policy.
 +     *
 +     * @param plc Speculative execution policy.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSpeculativeExecutionPolicy(SpeculativeExecutionPolicy plc) {
 +        speculativeExecutionPlc = plc;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets authentication provider.
 +     *
 +     * @param provider Authentication provider.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setAuthProvider(AuthProvider provider) {
 +        authProvider = provider;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets SSL options.
 +     *
 +     * @param options SSL options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSslOptions(SSLOptions options) {
 +        sslOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets pooling options.
 +     *
 +     * @param options pooling options to use.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setPoolingOptions(PoolingOptions options) {
 +        poolingOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets socket options to use.
 +     *
 +     * @param options Socket options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setSocketOptions(SocketOptions options) {
 +        sockOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Sets netty options to use.
 +     *
 +     * @param options netty options.
 +     */
 +    @SuppressWarnings("UnusedDeclaration")
 +    public void setNettyOptions(NettyOptions options) {
 +        nettyOptions = options;
 +
 +        invalidate();
 +    }
 +
 +    /**
 +     * Creates Cassandra session wrapper if it wasn't created yet and returns it
 +     *
 +     * @param log logger
 +     * @return Cassandra session wrapper
 +     */
 +    @SuppressWarnings("deprecation")
 +    public synchronized CassandraSession session(IgniteLogger log) {
 +        if (ses != null)
 +            return ses;
 +
 +        Cluster.Builder builder = Cluster.builder();
 +
 +        if (user != null)
 +            builder = builder.withCredentials(user, pwd);
 +
 +        if (port != null)
 +            builder = builder.withPort(port);
 +
 +        if (contactPoints != null)
 +            builder = builder.addContactPoints(contactPoints);
 +
 +        if (contactPointsWithPorts != null)
 +            builder = builder.addContactPointsWithPorts(contactPointsWithPorts);
 +
 +        if (maxSchemaAgreementWaitSeconds != null)
 +            builder = builder.withMaxSchemaAgreementWaitSeconds(maxSchemaAgreementWaitSeconds);
 +
 +        if (protoVer != null)
 +            builder = builder.withProtocolVersion(ProtocolVersion.fromInt(protoVer));
 +
 +        if (compression != null) {
 +            try {
 +                builder = builder.withCompression(ProtocolOptions.Compression.valueOf(compression.trim().toLowerCase()));
 +            }
 +            catch (IllegalArgumentException e) {
 +                throw new IgniteException("Incorrect compression option '" + compression + "' specified for Cassandra connection", e);
 +            }
 +        }
 +
 +        if (useSSL != null && useSSL)
 +            builder = builder.withSSL();
 +
 +        if (sslOptions != null)
 +            builder = builder.withSSL(sslOptions);
 +
 +        if (collectMetrix != null && !collectMetrix)
 +            builder = builder.withoutMetrics();
 +
 +        if (jmxReporting != null && !jmxReporting)
 +            builder = builder.withoutJMXReporting();
 +
 +        if (creds != null)
 +            builder = builder.withCredentials(creds.getUser(), creds.getPassword());
 +
 +        if (loadBalancingPlc != null)
 +            builder = builder.withLoadBalancingPolicy(loadBalancingPlc);
 +
 +        if (reconnectionPlc != null)
 +            builder = builder.withReconnectionPolicy(reconnectionPlc);
 +
 +        if (retryPlc != null)
 +            builder = builder.withRetryPolicy(retryPlc);
 +
 +        if (addrTranslator != null)
 +            builder = builder.withAddressTranslator(addrTranslator);
 +
 +        if (speculativeExecutionPlc != null)
 +            builder = builder.withSpeculativeExecutionPolicy(speculativeExecutionPlc);
 +
 +        if (authProvider != null)
 +            builder = builder.withAuthProvider(authProvider);
 +
 +        if (poolingOptions != null)
 +            builder = builder.withPoolingOptions(poolingOptions);
 +
 +        if (sockOptions != null)
 +            builder = builder.withSocketOptions(sockOptions);
 +
 +        if (nettyOptions != null)
 +            builder = builder.withNettyOptions(nettyOptions);
 +
 +        return ses = new CassandraSessionImpl(builder, fetchSize, readConsistency, writeConsistency, log);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @Override public void writeExternal(ObjectOutput out) throws IOException {
 +        out.writeObject(fetchSize);
 +        out.writeObject(readConsistency);
 +        out.writeObject(writeConsistency);
 +        U.writeString(out, user);
 +        U.writeString(out, pwd);
 +        out.writeObject(port);
 +        out.writeObject(contactPoints);
 +        out.writeObject(contactPointsWithPorts);
 +        out.writeObject(maxSchemaAgreementWaitSeconds);
 +        out.writeObject(protoVer);
 +        U.writeString(out, compression);
 +        out.writeObject(useSSL);
 +        out.writeObject(collectMetrix);
 +        out.writeObject(jmxReporting);
 +        out.writeObject(creds);
 +        writeObject(out, loadBalancingPlc);
 +        writeObject(out, reconnectionPlc);
 +        writeObject(out, addrTranslator);
 +        writeObject(out, speculativeExecutionPlc);
 +        writeObject(out, authProvider);
 +        writeObject(out, sslOptions);
 +        writeObject(out, poolingOptions);
 +        writeObject(out, sockOptions);
 +        writeObject(out, nettyOptions);
 +    }
 +
 +    /** {@inheritDoc} */
 +    @SuppressWarnings("unchecked")
 +    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
 +        fetchSize = (Integer)in.readObject();
 +        readConsistency = (ConsistencyLevel)in.readObject();
 +        writeConsistency = (ConsistencyLevel)in.readObject();
 +        user = U.readString(in);
 +        pwd = U.readString(in);
 +        port = (Integer)in.readObject();
 +        contactPoints = (List<InetAddress>)in.readObject();
 +        contactPointsWithPorts = (List<InetSocketAddress>)in.readObject();
 +        maxSchemaAgreementWaitSeconds = (Integer)in.readObject();
 +        protoVer = (Integer)in.readObject();
 +        compression = U.readString(in);
 +        useSSL = (Boolean)in.readObject();
 +        collectMetrix = (Boolean)in.readObject();
 +        jmxReporting = (Boolean)in.readObject();
 +        creds = (Credentials)in.readObject();
 +        loadBalancingPlc = (LoadBalancingPolicy)readObject(in);
 +        reconnectionPlc = (ReconnectionPolicy)readObject(in);
 +        addrTranslator = (AddressTranslator)readObject(in);
 +        speculativeExecutionPlc = (SpeculativeExecutionPolicy)readObject(in);
 +        authProvider = (AuthProvider)readObject(in);
 +        sslOptions = (SSLOptions)readObject(in);
 +        poolingOptions = (PoolingOptions)readObject(in);
 +        sockOptions = (SocketOptions)readObject(in);
 +        nettyOptions = (NettyOptions)readObject(in);
 +    }
 +
 +    /**
 +     * Helper method used to serialize class members
 +     * @param out the stream to write the object to
 +     * @param obj the object to be written
 +     * @throws IOException Includes any I/O exceptions that may occur
 +     */
 +    private void writeObject(ObjectOutput out, Object obj) throws IOException {
 +        out.writeObject(obj == null || !(obj instanceof Serializable) ? NULL_OBJECT : obj);
 +    }
 +
 +    /**
 +     * Helper method used to deserialize class members
 +     * @param in the stream to read data from in order to restore the object
 +     * @throws IOException Includes any I/O exceptions that may occur
 +     * @throws ClassNotFoundException If the class for an object being restored cannot be found
 +     * @return deserialized object
 +     */
 +    private Object readObject(ObjectInput in) throws IOException, ClassNotFoundException {
 +        Object obj = in.readObject();
 +        return NULL_OBJECT.equals(obj) ? null : obj;
 +    }
 +
 +    /**
 +     * Parses consistency level provided as string.
 +     *
 +     * @param level consistency level string.
 +     *
 +     * @return consistency level.
 +     */
 +    private ConsistencyLevel parseConsistencyLevel(String level) {
 +        if (level == null)
 +            return null;
 +
 +        try {
 +            return ConsistencyLevel.valueOf(level.trim().toUpperCase());
 +        }
 +        catch (Throwable e) {
 +            throw new IgniteException("Incorrect consistency level '" + level + "' specified for Cassandra connection", e);
 +        }
 +    }
 +
 +    /**
 +     * Invalidates session.
 +     */
 +    private synchronized void invalidate() {
 +        ses = null;
 +    }
++
++    /** {@inheritDoc} */
++    @Override public String toString() {
++        return S.toString(DataSource.class, this);
++    }
 +}

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/IgniteServices.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/IgniteServicesImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryClassDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/binary/builder/BinaryObjectBuilderImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 5f0b8a0,4d59d50..8187e8f
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@@ -3076,8 -3233,8 +3076,8 @@@ public abstract class GridCacheAdapter<
      }
  
      /** {@inheritDoc} */
 -    @Override public CacheMetrics clusterMetrics() {
 +    @Override public final CacheMetrics clusterMetrics() {
-         return clusterMetrics(ctx.grid().cluster().forCacheNodes(ctx.name()));
+         return clusterMetrics(ctx.grid().cluster().forDataNodes(ctx.name()));
      }
  
      /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/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/64247b92/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/64247b92/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/64247b92/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/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcRequestHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index aeb3ef4,3690f35..d26242d
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@@ -1587,9 -1590,9 +1590,9 @@@ public class GridServiceProcessor exten
                  else
                      topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
  
 -                depExe.submit(new BusyRunnable() {
 +                depExe.execute(new BusyRunnable() {
                      @Override public void run0() {
-                         ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
+                         ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
  
                          if (oldest != null && oldest.isLocal()) {
                              final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index ee5b65c,43017db..8f8d78a
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@@ -104,24 -104,9 +104,25 @@@ public abstract class AbstractAffinityF
      }
  
      /**
+      * @param backups Number of backups.
       * @throws Exception If failed.
       */
 +    public void testNullKeyForPartitionCalculation() throws Exception {
 +        AffinityFunction aff = affinityFunction();
 +
 +        try {
 +            aff.partition(null);
 +
 +            fail("Should throw IllegalArgumentException due to NULL affinity key.");
 +        } catch (IllegalArgumentException e) {
 +            e.getMessage().contains("Null key is passed for a partition calculation. " +
 +                "Make sure that an affinity key that is used is initialized properly.");
 +        }
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
      protected void checkNodeRemoved(int backups) throws Exception {
          checkNodeRemoved(backups, 1, 1);
      }

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/internal/binary/BinaryMarshallerSelfTest.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 554bb3d,deec72a..1e73e79
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@@ -39,10 -39,7 +39,11 @@@ import org.apache.ignite.cache.store.jd
  import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
  import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
  import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
 +import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
 +import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
 +import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
 +import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
+ import org.apache.ignite.cache.store.jdbc.JdbcTypesDefaultTransformerTest;
  import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
  import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
  import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
index 2d06f3a,350b715..b28619c
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteKernalSelfTestSuite.java
@@@ -66,7 -66,7 +66,8 @@@ import org.apache.ignite.internal.proce
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingDefaultMarshallerTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingJdkMarshallerTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest;
 +import org.apache.ignite.internal.processors.service.IgniteServiceDynamicCachesSelfTest;
+ import org.apache.ignite.internal.processors.service.IgniteServiceProxyTimeoutInitializedTest;
  import org.apache.ignite.internal.processors.service.IgniteServiceReassignmentTest;
  import org.apache.ignite.internal.processors.service.ServicePredicateAccessCacheTest;
  import org.apache.ignite.internal.util.GridStartupWithUndefinedIgniteHomeSelfTest;
@@@ -142,7 -142,7 +143,8 @@@ public class IgniteKernalSelfTestSuite 
          suite.addTestSuite(GridServiceProxyNodeStopSelfTest.class);
          suite.addTestSuite(GridServiceProxyClientReconnectSelfTest.class);
          suite.addTestSuite(IgniteServiceReassignmentTest.class);
+         suite.addTestSuite(IgniteServiceProxyTimeoutInitializedTest.class);
 +        suite.addTestSuite(IgniteServiceDynamicCachesSelfTest.class);
  
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingDefaultMarshallerTest.class);
          suite.addTestSuite(IgniteServiceDeploymentClassLoadingOptimizedMarshallerTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 5df44db,362ddd8..c541185
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@@ -794,40 -774,29 +794,45 @@@ public class IgniteH2Indexing implement
          throws IgniteCheckedException {
          final Connection conn = connectionForSpace(spaceName);
  
-         initLocalQueryContext(conn, enforceJoinOrder, filters);
+         setupConnection(conn, false, enforceJoinOrder);
  
-         Prepared p = null;
+         final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
  
-         try {
-             final PreparedStatement stmt = preparedStatementWithParams(conn, qry, params, true);
++        Prepared p = GridSqlQueryParser.prepared((JdbcPreparedStatement)stmt);
 +
-             p = GridSqlQueryParser.prepared((JdbcPreparedStatement) stmt);
++        if (!p.isQuery()) {
++            GridH2QueryContext.clearThreadLocal();
 +
-             if (!p.isQuery()) {
-                 GridH2QueryContext.clearThreadLocal();
++            SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
 +
-                 SqlFieldsQuery fldsQry = new SqlFieldsQuery(qry);
++            if (params != null)
++                fldsQry.setArgs(params.toArray());
 +
-                 if (params != null)
-                     fldsQry.setArgs(params.toArray());
++            fldsQry.setEnforceJoinOrder(enforceJoinOrder);
++            fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
 +
-                 fldsQry.setEnforceJoinOrder(enforceJoinOrder);
-                 fldsQry.setTimeout(timeout, TimeUnit.MILLISECONDS);
++            return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
++        }
 +
-                 return dmlProc.updateLocalSqlFields(spaceName, stmt, fldsQry, filters, cancel);
-             }
+         List<GridQueryFieldMetadata> meta;
  
-             List<GridQueryFieldMetadata> meta;
+         try {
+             meta = meta(stmt.getMetaData());
+         }
+         catch (SQLException e) {
+             throw new IgniteCheckedException("Cannot prepare query metadata", e);
+         }
  
-             try {
-                 meta = meta(stmt.getMetaData());
-             }
-             catch (SQLException e) {
-                 throw new IgniteCheckedException("Cannot prepare query metadata", e);
-             }
+         final GridH2QueryContext ctx = new GridH2QueryContext(nodeId, nodeId, 0, LOCAL)
+             .filter(filters).distributedJoins(false);
  
-             return new GridQueryFieldsResultAdapter(meta, null) {
-                 @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException{
+         return new GridQueryFieldsResultAdapter(meta, null) {
+             @Override public GridCloseableIterator<List<?>> iterator() throws IgniteCheckedException {
+                 assert GridH2QueryContext.get() == null;
+ 
+                 GridH2QueryContext.set(ctx);
+ 
+                 try {
                      ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
  
                      return new FieldsIterator(rs);

http://git-wip-us.apache.org/repos/asf/ignite/blob/64247b92/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------