You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/04/16 13:57:11 UTC

[01/50] [abbrv] ignite git commit: IGNITE-8101 Ability to terminate system workers by JMX for test purposes.

Repository: ignite
Updated Branches:
  refs/heads/ignite-7708 83be65ea7 -> eced6a11a


IGNITE-8101 Ability to terminate system workers by JMX for test purposes.

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: a96ac04755d2c7448508990fe9e1dad62bcccc90
Parents: 3a71765
Author: Dmitriy Sorokin <d....@gmail.com>
Authored: Tue Apr 10 22:20:41 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Tue Apr 10 22:20:41 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  7 ++
 .../failure/StopNodeOrHaltFailureHandler.java   |  2 +-
 .../ignite/internal/GridKernalContext.java      |  8 ++
 .../ignite/internal/GridKernalContextImpl.java  | 10 +++
 .../apache/ignite/internal/IgniteKernal.java    | 16 +++-
 .../discovery/GridDiscoveryManager.java         |  2 +-
 .../GridCachePartitionExchangeManager.java      |  3 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |  3 +-
 .../wal/reader/StandaloneGridKernalContext.java |  6 ++
 .../timeout/GridTimeoutProcessor.java           |  3 +-
 .../ignite/internal/util/IgniteUtils.java       |  7 +-
 .../worker/WorkersControlMXBeanImpl.java        | 62 +++++++++++++++
 .../ignite/internal/worker/WorkersRegistry.java | 80 ++++++++++++++++++++
 .../ignite/internal/worker/package-info.java    | 22 ++++++
 .../ignite/mxbean/WorkersControlMXBean.java     | 49 ++++++++++++
 15 files changed, 271 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 152d845..9da123e 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -423,6 +423,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_MBEANS_DISABLED = "IGNITE_MBEANS_DISABLED";
 
     /**
+     * If property is set to {@code true}, then test features will be enabled.
+     *
+     * Default is {@code false}.
+     */
+    public static final String IGNITE_TEST_FEATURES_ENABLED = "IGNITE_TEST_FEATURES_ENABLED";
+
+    /**
      * Property controlling size of buffer holding last exception. Default value of {@code 1000}.
      */
     public static final String IGNITE_EXCEPTION_REGISTRY_MAX_SIZE = "IGNITE_EXCEPTION_REGISTRY_MAX_SIZE";

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java b/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java
index 4f74406..3ce4ff6 100644
--- a/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/failure/StopNodeOrHaltFailureHandler.java
@@ -92,7 +92,7 @@ public class StopNodeOrHaltFailureHandler implements FailureHandler {
             ).start();
         }
         else {
-            U.error(log, "JVM will be halted immediately on ignite failure: [failureCtx=" + failureCtx + ']');
+            U.error(log, "JVM will be halted immediately due to the failure: [failureCtx=" + failureCtx + ']');
 
             Runtime.getRuntime().halt(Ignition.KILL_EXIT_CODE);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 0b40054..505c3d6 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
@@ -32,6 +32,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
+import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
@@ -423,6 +424,13 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridIndexingManager indexing();
 
     /**
+     * Gets workers registry.
+     *
+     * @return Workers registry.
+     */
+    public WorkersRegistry workersRegistry();
+
+    /**
      * Gets data structures processor.
      *
      * @return Data structures processor.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 3408334..ac49708 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
@@ -46,6 +46,7 @@ import org.apache.ignite.internal.managers.eventstorage.GridEventStorageManager;
 import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
+import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.authentication.IgniteAuthenticationProcessor;
 import org.apache.ignite.internal.processors.cache.CacheConflictResolutionManager;
@@ -361,6 +362,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     private Map<String, Object> attrs = new HashMap<>();
 
     /** */
+    @GridToStringExclude
+    private final WorkersRegistry workersRegistry = new WorkersRegistry();
+
+    /** */
     private IgniteEx grid;
 
     /** */
@@ -780,6 +785,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public WorkersRegistry workersRegistry() {
+        return workersRegistry;
+    }
+
+    /** {@inheritDoc} */
     @Override public GridAffinityProcessor affinity() {
         return affProc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 0b102e5..1cb07b9 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
@@ -125,7 +125,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheUtilityKey;
-import org.apache.ignite.internal.processors.cache.GridCacheUtils;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.binary.CacheObjectBinaryProcessorImpl;
@@ -185,6 +184,8 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.worker.WorkersControlMXBeanImpl;
+import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
@@ -197,6 +198,7 @@ import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.mxbean.ClusterMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.StripedExecutorMXBean;
+import org.apache.ignite.mxbean.WorkersControlMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
 import org.apache.ignite.plugin.IgnitePlugin;
 import org.apache.ignite.plugin.PluginNotFoundException;
@@ -1085,7 +1087,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             // Register MBeans.
             mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc, p2pExecSvc,
                 mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
-                qryExecSvc, schemaExecSvc, customExecSvcs);
+                qryExecSvc, schemaExecSvc, customExecSvcs, ctx.workersRegistry());
 
             // Lifecycle bean notifications.
             notifyLifecycleBeans(AFTER_NODE_START);
@@ -4175,7 +4177,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             IgniteStripedThreadPoolExecutor callbackExecSvc,
             ExecutorService qryExecSvc,
             ExecutorService schemaExecSvc,
-            @Nullable final Map<String, ? extends ExecutorService> customExecSvcs
+            @Nullable final Map<String, ? extends ExecutorService> customExecSvcs,
+            WorkersRegistry workersRegistry
         ) throws IgniteCheckedException {
             if (U.IGNITE_MBEANS_DISABLED)
                 return;
@@ -4221,6 +4224,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet())
                     registerExecutorMBean(entry.getKey(), entry.getValue());
             }
+
+            if (U.IGNITE_TEST_FEATURES_ENABLED) {
+                WorkersControlMXBean workerCtrlMXBean = new WorkersControlMXBeanImpl(workersRegistry);
+
+                registerMBean("Kernal", workerCtrlMXBean.getClass().getSimpleName(),
+                    workerCtrlMXBean, WorkersControlMXBean.class);
+            }
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index b0d3256..a1d84e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -2591,7 +2591,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          *
          */
         private DiscoveryWorker() {
-            super(ctx.igniteInstanceName(), "disco-event-worker", GridDiscoveryManager.this.log);
+            super(ctx.igniteInstanceName(), "disco-event-worker", GridDiscoveryManager.this.log, ctx.workersRegistry());
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 e40493f..1a0e65f 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
@@ -2077,7 +2077,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
          * Constructor.
          */
         private ExchangeWorker() {
-            super(cctx.igniteInstanceName(), "partition-exchanger", GridCachePartitionExchangeManager.this.log);
+            super(cctx.igniteInstanceName(), "partition-exchanger", GridCachePartitionExchangeManager.this.log,
+                cctx.kernalContext().workersRegistry());
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java
index 613e93b..7adabc3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedTtlCleanupManager.java
@@ -121,7 +121,8 @@ public class GridCacheSharedTtlCleanupManager extends GridCacheSharedManagerAdap
          * Creates cleanup worker.
          */
         CleanupWorker() {
-            super(cctx.igniteInstanceName(), "ttl-cleanup-worker", cctx.logger(GridCacheSharedTtlCleanupManager.class));
+            super(cctx.igniteInstanceName(), "ttl-cleanup-worker", cctx.logger(GridCacheSharedTtlCleanupManager.class),
+                cctx.kernalContext().workersRegistry());
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
index 429a5ce..cb04575 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/reader/StandaloneGridKernalContext.java
@@ -86,6 +86,7 @@ import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
 import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.worker.WorkersRegistry;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.plugin.PluginNotFoundException;
 import org.apache.ignite.plugin.PluginProvider;
@@ -454,6 +455,11 @@ public class StandaloneGridKernalContext implements GridKernalContext {
     }
 
     /** {@inheritDoc} */
+    @Override public WorkersRegistry workersRegistry() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public DataStructuresProcessor dataStructures() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
index a09d6fa..25151cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessor.java
@@ -145,7 +145,8 @@ public class GridTimeoutProcessor extends GridProcessorAdapter {
          *
          */
         TimeoutWorker() {
-            super(ctx.config().getIgniteInstanceName(), "grid-timeout-worker", GridTimeoutProcessor.this.log);
+            super(ctx.config().getIgniteInstanceName(), "grid-timeout-worker",
+                GridTimeoutProcessor.this.log, ctx.workersRegistry());
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/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 93f4fb4..42e96fb 100755
--- 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
@@ -522,7 +522,12 @@ public abstract class IgniteUtils {
     };
 
     /** Ignite MBeans disabled flag. */
-    public static boolean IGNITE_MBEANS_DISABLED = IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED);
+    public static boolean IGNITE_MBEANS_DISABLED =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_MBEANS_DISABLED);
+
+    /** Ignite test features enabled flag. */
+    public static boolean IGNITE_TEST_FEATURES_ENABLED =
+        IgniteSystemProperties.getBoolean(IgniteSystemProperties.IGNITE_TEST_FEATURES_ENABLED);
 
     /** */
     private static final boolean assertionsEnabled;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
new file mode 100644
index 0000000..9e427e8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersControlMXBeanImpl.java
@@ -0,0 +1,62 @@
+/*
+ * 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.worker;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.mxbean.WorkersControlMXBean;
+
+/**
+ * MBean that provides control of system workersRegistry.
+ */
+public class WorkersControlMXBeanImpl implements WorkersControlMXBean {
+    /** System worker registry. */
+    private final WorkersRegistry workerRegistry;
+
+    /**
+     * Constructor.
+     *
+     * @param registry System worker registry.
+     */
+    public WorkersControlMXBeanImpl(WorkersRegistry registry) {
+        workerRegistry = registry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public List<String> getWorkerNames() {
+        return new ArrayList<>(workerRegistry.names());
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean terminateWorker(String name) {
+        GridWorker w = workerRegistry.worker(name);
+
+        if (w == null || w.isCancelled())
+            return false;
+
+        Thread t = w.runner();
+
+        if (t == null)
+            return false;
+
+        t.interrupt();
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
new file mode 100644
index 0000000..e8d46fb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/WorkersRegistry.java
@@ -0,0 +1,80 @@
+/*
+ * 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.worker;
+
+import java.util.Collection;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.ignite.internal.util.worker.GridWorker;
+import org.apache.ignite.internal.util.worker.GridWorkerListener;
+
+/**
+ * Workers registry.
+ */
+public class WorkersRegistry implements GridWorkerListener {
+    /** Registered workers. */
+    private final ConcurrentMap<String, GridWorker> registeredWorkers = new ConcurrentHashMap<>();
+
+    /**
+     * Adds worker to the registry.
+     *
+     * @param w Worker.
+     */
+    public void register(GridWorker w) {
+        if (registeredWorkers.putIfAbsent(w.name(), w) != null)
+            throw new IllegalStateException("Worker is already registered [worker=" + w + ']');
+    }
+
+    /**
+     * Removes worker from the registry.
+     *
+     * @param name Worker name.
+     */
+    public void unregister(String name) {
+        registeredWorkers.remove(name);
+    }
+
+    /**
+     * Returns names of all registered workers.
+     *
+     * @return Registered worker names.
+     */
+    public Collection<String> names() {
+        return registeredWorkers.keySet();
+    }
+
+    /**
+     * Returns worker with given name.
+     *
+     * @param name Name.
+     * @return Registered {@link GridWorker} with name {@code name} or {@code null} if not found.
+     */
+    public GridWorker worker(String name) {
+        return registeredWorkers.get(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStarted(GridWorker w) {
+        register(w);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onStopped(GridWorker w) {
+        unregister(w.name());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java b/modules/core/src/main/java/org/apache/ignite/internal/worker/package-info.java
new file mode 100644
index 0000000..03ca621
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/worker/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. -->
+ * System worker registry and control MBean implementation.
+ */
+package org.apache.ignite.internal.worker;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a96ac047/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
new file mode 100644
index 0000000..0f5419b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/mxbean/WorkersControlMXBean.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.mxbean;
+
+import java.util.List;
+
+/**
+ * MBean that provides ability to terminate worker that registered in the workers registry.
+ */
+@MXBeanDescription("MBean that provides ability to terminate worker that registered in the workers registry.")
+public interface WorkersControlMXBean {
+    /**
+     * Returns names of all registered workers.
+     *
+     * @return Worker names.
+     */
+    @MXBeanDescription("Names of registered workers.")
+    public List<String> getWorkerNames();
+
+    /**
+     * Terminates worker.
+     *
+     * @param name Worker name.
+     * @return {@code True} if worker has been terminated successfully, {@code false} otherwise.
+     */
+    @MXBeanDescription("Terminates worker.")
+    @MXBeanParametersNames(
+        "name"
+    )
+    @MXBeanParametersDescriptions(
+        "Name of worker to terminate."
+    )
+    public boolean terminateWorker(String name);
+}


[27/50] [abbrv] ignite git commit: IGNITE-8110 GridCacheWriteBehindStore.Flusher thread uses the wrong transformation from milliseconds to nanoseconds. - Fixes #3742.

Posted by ag...@apache.org.
IGNITE-8110 GridCacheWriteBehindStore.Flusher thread uses the wrong transformation from milliseconds to nanoseconds. - Fixes #3742.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: adaedb44b767da49c4ec6c57c655b13caa3edd35
Parents: 47ca3e0
Author: Anton Kurbanov <an...@gmail.com>
Authored: Thu Apr 12 20:31:50 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Thu Apr 12 20:31:50 2018 +0300

----------------------------------------------------------------------
 .../cache/store/GridCacheWriteBehindStore.java  |  2 +-
 .../GridCacheWriteBehindStoreSelfTest.java      | 41 +++++++++++++++++++-
 2 files changed, 40 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/adaedb44/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 44cadd6..82ff3aa 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
@@ -895,7 +895,7 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
         protected Thread thread;
 
         /** Cache flushing frequence in nanos. */
-        protected long cacheFlushFreqNanos = cacheFlushFreq * 1000;
+        protected long cacheFlushFreqNanos = cacheFlushFreq * 1000 * 1000;
 
         /** Writer lock. */
         private final Lock flusherWriterLock = new ReentrantLock();

http://git-wip-us.apache.org/repos/asf/ignite/blob/adaedb44/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java
index 9a487a4..af21fc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStoreSelfTest.java
@@ -107,6 +107,43 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore
     }
 
     /**
+     * Checks that write behind cache flush frequency was correctly adjusted to nanos expecting putAllCnt to be
+     * less or equal than elapsed time divided by flush frequency.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSimpleStoreFlushFrequencyWithoutCoalescing() throws Exception {
+        initStore(1, false);
+
+        long writeBehindFlushFreqNanos = FLUSH_FREQUENCY * 1000 * 1000;
+
+        int threshold = store.getWriteBehindStoreBatchSize() / 10;
+
+        try {
+            long start = System.nanoTime();
+
+            for (int i = 0; i < threshold / 2; i++)
+                store.write(new CacheEntryImpl<>(i, "v" + i));
+
+            U.sleep(FLUSH_FREQUENCY + 300);
+
+            for (int i = threshold / 2; i < threshold; i++)
+                store.write(new CacheEntryImpl<>(i, "v" + i));
+
+            long elapsed = System.nanoTime() - start;
+
+            U.sleep(FLUSH_FREQUENCY + 300);
+
+            int expFlushOps = (int)(1 + elapsed / writeBehindFlushFreqNanos);
+
+            assertTrue(delegate.getPutAllCount() <= expFlushOps);
+        }
+        finally {
+            shutdownStore();
+        }
+    }
+
+    /**
      * Simple store test.
      *
      * @param writeCoalescing Write coalescing flag.
@@ -254,7 +291,6 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore
 
             int delegatePutCnt = delegate.getPutAllCount();
 
-
             fut.get();
 
             log().info(">>> [putCnt = " + actualPutCnt.get() + ", delegatePutCnt=" + delegatePutCnt + "]");
@@ -262,7 +298,8 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore
             assertTrue("No puts were made to the underlying store", delegatePutCnt > 0);
             if (store.getWriteCoalescing()) {
                 assertTrue("Too many puts were made to the underlying store", delegatePutCnt < actualPutCnt.get() / 10);
-            } else {
+            }
+            else {
                 assertTrue("Too few puts cnt=" + actualPutCnt.get() + " << storePutCnt=" + delegatePutCnt, delegatePutCnt > actualPutCnt.get() / 2);
             }
         }


[41/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.java
new file mode 100644
index 0000000..e48d946
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/UpdatesStrategy.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.ml.nn;
+
+import java.io.Serializable;
+import java.util.List;
+import org.apache.ignite.ml.math.functions.IgniteFunction;
+import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator;
+
+/**
+ * Class encapsulating update strategies for group trainers based on updates.
+ *
+ * @param <M> Type of model to be optimized.
+ * @param <U> Type of update.
+ */
+public class UpdatesStrategy<M, U extends Serializable> {
+    /**
+     * {@link ParameterUpdateCalculator}.
+     */
+    private ParameterUpdateCalculator<M, U> updatesCalculator;
+
+    /**
+     * Function used to reduce updates in one training (for example, sum all sequential gradient updates to get one
+     * gradient update).
+     */
+    private IgniteFunction<List<U>, U> locStepUpdatesReducer;
+
+    /**
+     * Function used to reduce updates from different trainings (for example, averaging of gradients of all parallel trainings).
+     */
+    private IgniteFunction<List<U>, U> allUpdatesReducer;
+
+    /**
+     * Construct instance of this class with given parameters.
+     *
+     * @param updatesCalculator Parameter update calculator.
+     * @param locStepUpdatesReducer Function used to reduce updates in one training
+     * (for example, sum all sequential gradient updates to get one gradient update).
+     * @param allUpdatesReducer Function used to reduce updates from different trainings
+     * (for example, averaging of gradients of all parallel trainings).
+     */
+    public UpdatesStrategy(
+        ParameterUpdateCalculator<M, U> updatesCalculator,
+        IgniteFunction<List<U>, U> locStepUpdatesReducer,
+        IgniteFunction<List<U>, U> allUpdatesReducer) {
+        this.updatesCalculator = updatesCalculator;
+        this.locStepUpdatesReducer = locStepUpdatesReducer;
+        this.allUpdatesReducer = allUpdatesReducer;
+    }
+
+    /**
+     * Get parameter update calculator (see {@link ParameterUpdateCalculator}).
+     *
+     * @return Parameter update calculator.
+     */
+    public ParameterUpdateCalculator<M, U> getUpdatesCalculator() {
+        return updatesCalculator;
+    }
+
+    /**
+     * Get function used to reduce updates in one training
+     * (for example, sum all sequential gradient updates to get one gradient update).
+     *
+     * @return Function used to reduce updates in one training
+     * (for example, sum all sequential gradient updates to get on gradient update).
+     */
+    public IgniteFunction<List<U>, U> locStepUpdatesReducer() {
+        return locStepUpdatesReducer;
+    }
+
+    /**
+     * Get function used to reduce updates from different trainings
+     * (for example, averaging of gradients of all parallel trainings).
+     *
+     * @return Function used to reduce updates from different trainings.
+     */
+    public IgniteFunction<List<U>, U> allUpdatesReducer() {
+        return allUpdatesReducer;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java
deleted file mode 100644
index 15ed914..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java
+++ /dev/null
@@ -1,202 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization;
-
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.StorageConstants;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.math.impls.vector.FunctionVector;
-import org.apache.ignite.ml.optimization.util.SparseDistributedMatrixMapReducer;
-
-/**
- * Gradient descent optimizer.
- */
-public class GradientDescent {
-    /**
-     * Function which computes gradient of the loss function at any given point.
-     */
-    private final GradientFunction lossGradient;
-
-    /**
-     * Weights updater applied on every gradient descent step to decide how weights should be changed.
-     */
-    private final Updater updater;
-
-    /**
-     * Max number of gradient descent iterations.
-     */
-    private int maxIterations = 1000;
-
-    /**
-     * Convergence tolerance is condition which decides iteration termination.
-     */
-    private double convergenceTol = 1e-8;
-
-    /**
-     * New gradient descent instance based of loss function and updater.
-     *
-     * @param lossGradient Function which computes gradient of the loss function at any given point
-     * @param updater Weights updater applied on every gradient descent step to decide how weights should be changed
-     */
-    public GradientDescent(GradientFunction lossGradient, Updater updater) {
-        this.lossGradient = lossGradient;
-        this.updater = updater;
-    }
-
-    /**
-     * Sets max number of gradient descent iterations.
-     *
-     * @param maxIterations Max number of gradient descent iterations
-     * @return This gradient descent instance
-     */
-    public GradientDescent withMaxIterations(int maxIterations) {
-        assert maxIterations >= 0;
-
-        this.maxIterations = maxIterations;
-
-        return this;
-    }
-
-    /**
-     * Sets convergence tolerance.
-     *
-     * @param convergenceTol Condition which decides iteration termination
-     * @return This gradient descent instance
-     */
-    public GradientDescent withConvergenceTol(double convergenceTol) {
-        assert convergenceTol >= 0;
-
-        this.convergenceTol = convergenceTol;
-
-        return this;
-    }
-
-    /**
-     * Computes point where loss function takes minimal value.
-     *
-     * @param data Inputs parameters of loss function
-     * @param initWeights Initial weights
-     * @return Point where loss function takes minimal value
-     */
-    public Vector optimize(Matrix data, Vector initWeights) {
-        Vector weights = initWeights, oldWeights = null, oldGradient = null;
-        IgniteFunction<Vector, Vector> gradientFunction = getLossGradientFunction(data);
-
-        for (int iteration = 0; iteration < maxIterations; iteration++) {
-            Vector gradient = gradientFunction.apply(weights);
-            Vector newWeights = updater.compute(oldWeights, oldGradient, weights, gradient, iteration);
-
-            if (isConverged(weights, newWeights))
-                return newWeights;
-            else {
-                oldGradient = gradient;
-                oldWeights = weights;
-                weights = newWeights;
-            }
-        }
-        return weights;
-    }
-
-    /**
-     * Calculates gradient based in distributed matrix using {@link SparseDistributedMatrixMapReducer}.
-     *
-     * @param data Distributed matrix
-     * @param weights Point to calculate gradient
-     * @return Gradient
-     */
-    private Vector calculateDistributedGradient(SparseDistributedMatrix data, Vector weights) {
-        SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(data);
-        return mapReducer.mapReduce(
-            (matrix, args) -> {
-                Matrix inputs = extractInputs(matrix);
-                Vector groundTruth = extractGroundTruth(matrix);
-
-                return lossGradient.compute(inputs, groundTruth, args);
-            },
-            gradients -> {
-                int cnt = 0;
-                Vector resGradient = new DenseLocalOnHeapVector(data.columnSize());
-
-                for (Vector gradient : gradients) {
-                    if (gradient != null) {
-                        resGradient = resGradient.plus(gradient);
-                        cnt++;
-                    }
-                }
-
-                return resGradient.divide(cnt);
-            },
-            weights);
-    }
-
-    /**
-     * Tests if gradient descent process converged.
-     *
-     * @param weights Weights
-     * @param newWeights New weights
-     * @return {@code true} if process has converged, otherwise {@code false}
-     */
-    private boolean isConverged(Vector weights, Vector newWeights) {
-        if (convergenceTol == 0)
-            return false;
-        else {
-            double solutionVectorDiff = weights.minus(newWeights).kNorm(2.0);
-            return solutionVectorDiff < convergenceTol * Math.max(newWeights.kNorm(2.0), 1.0);
-        }
-    }
-
-    /**
-     * Extracts first column with ground truth from the data set matrix.
-     *
-     * @param data data to build model
-     * @return Ground truth vector
-     */
-    private Vector extractGroundTruth(Matrix data) {
-        return data.getCol(0);
-    }
-
-    /**
-     * Extracts all inputs from data set matrix and updates matrix so that first column contains value 1.0.
-     *
-     * @param data data to build model
-     * @return Inputs matrix
-     */
-    private Matrix extractInputs(Matrix data) {
-        data = data.copy();
-        data.assignColumn(0, new FunctionVector(data.rowSize(), row -> 1.0));
-        return data;
-    }
-
-    /** Makes carrying of the gradient function and fixes data matrix. */
-    private IgniteFunction<Vector, Vector> getLossGradientFunction(Matrix data) {
-        if (data instanceof SparseDistributedMatrix) {
-            SparseDistributedMatrix distributedMatrix = (SparseDistributedMatrix)data;
-
-            if (distributedMatrix.getStorage().storageMode() == StorageConstants.ROW_STORAGE_MODE)
-                return weights -> calculateDistributedGradient(distributedMatrix, weights);
-        }
-
-        Matrix inputs = extractInputs(data);
-        Vector groundTruth = extractGroundTruth(data);
-
-        return weights -> lossGradient.compute(inputs, groundTruth, weights);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java
deleted file mode 100644
index a6a1e71..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization;
-
-import java.io.Serializable;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-
-/**
- * Function which computes gradient of the loss function at any given point.
- */
-@FunctionalInterface
-public interface GradientFunction extends Serializable {
-    /** */
-    Vector compute(Matrix inputs, Vector groundTruth, Vector pnt);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java
deleted file mode 100644
index 4d90e3b..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization;
-
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-
-/**
- * Function which computes gradient of least square loss function.
- */
-public class LeastSquaresGradientFunction implements GradientFunction {
-    /**
-     * {@inheritDoc}
-     */
-    @Override public Vector compute(Matrix inputs, Vector groundTruth, Vector pnt) {
-        return inputs.transpose().times(inputs.times(pnt).minus(groundTruth));
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java
deleted file mode 100644
index 20f861e..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization.util;
-
-import java.util.Collection;
-import java.util.Map;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.distributed.keys.RowColMatrixKey;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.storage.matrix.SparseDistributedMatrixStorage;
-
-/**
- * Wrapper of {@link SparseDistributedMatrix} which allow to perform computation on every node containing a part of the
- * distributed matrix, get results and then reduce them.
- */
-public class SparseDistributedMatrixMapReducer {
-    /** */
-    private final SparseDistributedMatrix distributedMatrix;
-
-    /** */
-    public SparseDistributedMatrixMapReducer(
-        SparseDistributedMatrix distributedMatrix) {
-        this.distributedMatrix = distributedMatrix;
-    }
-
-    /** */
-    public <R, T> R mapReduce(IgniteBiFunction<Matrix, T, R> mapper, IgniteFunction<Collection<R>, R> reducer, T args) {
-        Ignite ignite = Ignition.localIgnite();
-        SparseDistributedMatrixStorage storage = (SparseDistributedMatrixStorage)distributedMatrix.getStorage();
-
-        int colSize = distributedMatrix.columnSize();
-
-        Collection<R> results = ignite
-            .compute(ignite.cluster().forDataNodes(storage.cacheName()))
-            .broadcast(arguments -> {
-                Ignite locIgnite = Ignition.localIgnite();
-
-                Affinity<RowColMatrixKey> affinity = locIgnite.affinity(storage.cacheName());
-                ClusterNode locNode = locIgnite.cluster().localNode();
-
-                Map<ClusterNode, Collection<RowColMatrixKey>> keys = affinity.mapKeysToNodes(storage.getAllKeys());
-                Collection<RowColMatrixKey> locKeys = keys.get(locNode);
-
-                if (locKeys != null) {
-                    int idx = 0;
-                    Matrix locMatrix = new DenseLocalOnHeapMatrix(locKeys.size(), colSize);
-
-                    for (RowColMatrixKey key : locKeys) {
-                        Map<Integer, Double> row = storage.cache().get(key);
-
-                        for (Map.Entry<Integer, Double> cell : row.entrySet())
-                            locMatrix.set(idx, cell.getKey(), cell.getValue());
-
-                        idx++;
-                    }
-                    return mapper.apply(locMatrix, arguments);
-                }
-                return null;
-            }, args);
-        return reducer.apply(results);
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
index 9526db1..095aa31 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
@@ -17,18 +17,17 @@
 
 package org.apache.ignite.ml.regressions.linear;
 
+import java.util.Arrays;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleLabeledDatasetDataBuilder;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap;
 import org.apache.ignite.ml.math.isolve.lsqr.AbstractLSQR;
 import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeap;
 import org.apache.ignite.ml.math.isolve.lsqr.LSQRResult;
 import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
 
-import java.util.Arrays;
-
 /**
  * Trainer of the linear regression model based on LSQR algorithm.
  *
@@ -43,7 +42,10 @@ public class LinearRegressionLSQRTrainer implements SingleLabelDatasetTrainer<Li
 
         try (LSQROnHeap<K, V> lsqr = new LSQROnHeap<>(
             datasetBuilder,
-            new LinSysPartitionDataBuilderOnHeap<>(new FeatureExtractorWrapper<>(featureExtractor), lbExtractor)
+            new SimpleLabeledDatasetDataBuilder<>(
+                new FeatureExtractorWrapper<>(featureExtractor),
+                lbExtractor.andThen(e -> new double[]{e})
+            )
         )) {
             res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java
deleted file mode 100644
index 5de3cda..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.decompositions.QRDSolver;
-import org.apache.ignite.ml.math.decompositions.QRDecomposition;
-import org.apache.ignite.ml.math.impls.vector.FunctionVector;
-
-/**
- * Linear regression trainer based on least squares loss function and QR decomposition.
- */
-public class LinearRegressionQRTrainer implements Trainer<LinearRegressionModel, Matrix> {
-    /**
-     * {@inheritDoc}
-     */
-    @Override public LinearRegressionModel train(Matrix data) {
-        Vector groundTruth = extractGroundTruth(data);
-        Matrix inputs = extractInputs(data);
-
-        QRDecomposition decomposition = new QRDecomposition(inputs);
-        QRDSolver solver = new QRDSolver(decomposition.getQ(), decomposition.getR());
-
-        Vector variables = solver.solve(groundTruth);
-        Vector weights = variables.viewPart(1, variables.size() - 1);
-
-        double intercept = variables.get(0);
-
-        return new LinearRegressionModel(weights, intercept);
-    }
-
-    /**
-     * Extracts first column with ground truth from the data set matrix.
-     *
-     * @param data data to build model
-     * @return Ground truth vector
-     */
-    private Vector extractGroundTruth(Matrix data) {
-        return data.getCol(0);
-    }
-
-    /**
-     * Extracts all inputs from data set matrix and updates matrix so that first column contains value 1.0.
-     *
-     * @param data data to build model
-     * @return Inputs matrix
-     */
-    private Matrix extractInputs(Matrix data) {
-        data = data.copy();
-
-        data.assignColumn(0, new FunctionVector(data.rowSize(), row -> 1.0));
-
-        return data;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
index 9be3fdd..98b8885 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
@@ -30,7 +30,7 @@ import org.apache.ignite.ml.nn.MultilayerPerceptron;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
 
 import java.io.Serializable;
 import java.util.Arrays;
@@ -110,6 +110,9 @@ public class LinearRegressionSGDTrainer<P extends Serializable> implements Singl
 
         double[] p = mlp.parameters().getStorage().data();
 
-        return new LinearRegressionModel(new DenseLocalOnHeapVector(Arrays.copyOf(p, p.length - 1)), p[p.length - 1]);
+        return new LinearRegressionModel(new DenseLocalOnHeapVector(
+            Arrays.copyOf(p, p.length - 1)),
+            p[p.length - 1]
+        );
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java
deleted file mode 100644
index b4f83d9..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/Trainer.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers;
-
-import org.apache.ignite.ml.Model;
-
-/** Trainer interface. */
-@Deprecated
-// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one
-public interface Trainer<M extends Model, T> {
-    /**
-     * Train the model based on provided data.
-     *
-     * @param data Data for training.
-     * @return Trained model.
-     */
-    public M train(T data);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java
deleted file mode 100644
index e20a55a..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/BaseLocalProcessorJob.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.compute.ComputeJob;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-
-/**
- * Base job for group training.
- * It's purpose is to apply worker to each element (cache key or cache entry) of given cache specified
- * by keySupplier. Worker produces {@link ResultAndUpdates} object which contains 'side effects' which are updates
- * needed to apply to caches and computation result.
- * After we get all {@link ResultAndUpdates} we merge all 'update' parts of them for each node
- * and apply them on corresponding node, also we reduce all 'result' by some given reducer.
- *
- * @param <K> Type of keys of cache used for group trainer.
- * @param <V> Type of values of cache used for group trainer.
- * @param <T> Type of elements to which workers are applier.
- * @param <R> Type of result of worker.
- */
-public abstract class BaseLocalProcessorJob<K, V, T, R extends Serializable> implements ComputeJob {
-    /**
-     * UUID of group training.
-     */
-    protected UUID trainingUUID;
-
-    /**
-     * Worker.
-     */
-    protected IgniteFunction<T, ResultAndUpdates<R>> worker;
-
-    /**
-     * Supplier of keys determining elements to which worker should be applied.
-     */
-    protected IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keySupplier;
-
-    /**
-     * Operator used to reduce results from worker.
-     */
-    protected IgniteFunction<List<R>, R> reducer;
-
-    /**
-     * Name of cache used for training.
-     */
-    protected String cacheName;
-
-    /**
-     * Construct instance of this class with given arguments.
-     *
-     * @param worker Worker.
-     * @param keySupplier Supplier of keys.
-     * @param reducer Reducer.
-     * @param trainingUUID UUID of training.
-     * @param cacheName Name of cache used for training.
-     */
-    public BaseLocalProcessorJob(
-        IgniteFunction<T, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keySupplier,
-        IgniteFunction<List<R>, R> reducer,
-        UUID trainingUUID, String cacheName) {
-        this.worker = worker;
-        this.keySupplier = keySupplier;
-        this.reducer = reducer;
-        this.trainingUUID = trainingUUID;
-        this.cacheName = cacheName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void cancel() {
-        // NO-OP.
-    }
-
-    /** {@inheritDoc} */
-    @Override public R execute() throws IgniteException {
-        List<ResultAndUpdates<R>> resultsAndUpdates = toProcess().
-            map(worker).
-            collect(Collectors.toList());
-
-        ResultAndUpdates<R> totalRes = ResultAndUpdates.sum(reducer, resultsAndUpdates.stream().filter(Objects::nonNull).collect(Collectors.toList()));
-
-        totalRes.applyUpdates(ignite());
-
-        return totalRes.result();
-    }
-
-    /**
-     * Get stream of elements to process.
-     *
-     * @return Stream of elements to process.
-     */
-    protected abstract Stream<T> toProcess();
-
-    /**
-     * Ignite instance.
-     *
-     * @return Ignite instance.
-     */
-    protected static Ignite ignite() {
-        return Ignition.localIgnite();
-    }
-
-    /**
-     * Get cache used for training.
-     *
-     * @return Cache used for training.
-     */
-    protected IgniteCache<GroupTrainerCacheKey<K>, V> cache() {
-        return ignite().getOrCreateCache(cacheName);
-    }
-
-    /**
-     * Get affinity function for cache used in group training.
-     *
-     * @return Affinity function for cache used in group training.
-     */
-    protected Affinity<GroupTrainerCacheKey> affinity() {
-        return ignite().affinity(cacheName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java
deleted file mode 100644
index 75f8179..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ConstModel.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import org.apache.ignite.ml.Model;
-
-/**
- * Model which outputs given constant.
- *
- * @param <T> Type of constant.
- */
-public class ConstModel<T> implements Model<T, T> {
-    /**
-     * Constant to be returned by this model.
-     */
-    private T c;
-
-    /**
-     * Create instance of this class specified by input parameters.
-     *
-     * @param c Constant to be returned by this model.
-     */
-    public ConstModel(T c) {
-        this.c = c;
-    }
-
-    /** {@inheritDoc} */
-    @Override public T apply(T val) {
-        return c;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java
deleted file mode 100644
index fb34bf7..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainer.java
+++ /dev/null
@@ -1,208 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.ml.Model;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.Trainer;
-import org.apache.ignite.ml.trainers.group.chain.ComputationsChain;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/**
- * Class encapsulating synchronous distributed group training.
- * Training is performed by following scheme:
- * 1. For specified set of keys distributed initialization is done. For each key some initialization result is returned.
- * 2. All initialization results are processed locally and reduced into some object of type I.
- * 3. While 'shouldContinue' condition is true, training loop step is executed.
- * 4. After loop is finished, data from each key from final key set is collected.
- * 5. Data collected on previous step is transformed into a model which is returned as final result.
- * Note that all methods returning functions, suppliers etc should return values with minimal dependencies because they are serialized
- * with all dependent objects.
- *
- * @param <LC> Type of local context of the training.
- * @param <K> Type of data in {@link GroupTrainerCacheKey} keys on which the training is done.
- * @param <V> Type of cache values on which the training is done.
- * @param <IN> Type of data returned after initializing of distributed context.
- * @param <R> Type of result returned after training from each node.
- * @param <I> Type of data which is fed into each training loop step and returned from it.
- * @param <M> Type of model returned after training.
- * @param <T> Type of input to this trainer.
- * @param <G> Type of distributed context which is needed for forming final result which is send from each node to trainer for final model creation.
- */
-abstract class GroupTrainer<LC extends HasTrainingUUID, K, V, IN extends Serializable, R extends Serializable, I extends Serializable, M extends Model, T extends GroupTrainerInput<K>, G> implements Trainer<M, T> {
-    /**
-     * Cache on which training is performed. For example it can be cache of neural networks.
-     */
-    protected IgniteCache<GroupTrainerCacheKey<K>, V> cache;
-
-    /**
-     * Ignite instance.
-     */
-    protected Ignite ignite;
-
-    /**
-     * Construct an instance of this class.
-     *
-     * @param cache Cache on which training is performed.
-     * @param ignite Ignite instance.
-     */
-    GroupTrainer(
-        IgniteCache<GroupTrainerCacheKey<K>, V> cache,
-        Ignite ignite) {
-        this.cache = cache;
-        this.ignite = ignite;
-    }
-
-    /** {@inheritDoc} */
-    @Override public final M train(T data) {
-        UUID trainingUUID = UUID.randomUUID();
-        LC locCtx = initialLocalContext(data, trainingUUID);
-
-        GroupTrainingContext<K, V, LC> ctx = new GroupTrainingContext<>(locCtx, cache, ignite);
-        ComputationsChain<LC, K, V, T, T> chain = (i, c) -> i;
-        IgniteFunction<GroupTrainerCacheKey<K>, ResultAndUpdates<IN>> distributedInitializer
-            = distributedInitializer(data);
-
-        init(data, trainingUUID);
-
-        M res = chain.
-            thenDistributedForKeys(distributedInitializer, (t, lc) -> data.initialKeys(trainingUUID),
-                reduceDistributedInitData()).
-            thenLocally(this::locallyProcessInitData).
-            thenWhile(this::shouldContinue, trainingLoopStep()).
-            thenDistributedForEntries(this::extractContextForFinalResultCreation, finalResultsExtractor(),
-                this::finalResultKeys, finalResultsReducer()).
-            thenLocally(this::mapFinalResult).
-            process(data, ctx);
-
-        cleanup(locCtx);
-
-        return res;
-    }
-
-    /**
-     * Create initial local context from data given as input to trainer.
-     *
-     * @param data Data given as input to this trainer.
-     * @param trainingUUID UUID of this training.
-     * @return Initial local context.
-     */
-    protected abstract LC initialLocalContext(T data, UUID trainingUUID);
-
-    /** Override in subclasses if needed. */
-    protected void init(T data, UUID trainingUUID) {
-    }
-
-    /**
-     * Get function for initialization for each of keys specified in initial key set.
-     *
-     * @param data Data given to this trainer as input.
-     * @return Function for initialization for each of keys specified in initial key set.
-     */
-    protected abstract IgniteFunction<GroupTrainerCacheKey<K>, ResultAndUpdates<IN>> distributedInitializer(T data);
-
-    /**
-     * Get reducer to reduce data collected from initialization of each key specified in initial key set.
-     *
-     * @return Reducer to reduce data collected from initialization of each key specified in initial key set.
-     */
-    protected abstract IgniteFunction<List<IN>, IN> reduceDistributedInitData();
-
-    /**
-     * Transform data from initialization step into data which is fed as input to first step of training loop.
-     *
-     * @param data Data from initialization step.
-     * @param locCtx Local context.
-     * @return Data which is fed as input to first step of training loop.
-     */
-    protected abstract I locallyProcessInitData(IN data, LC locCtx);
-
-    /**
-     * Training loop step.
-     *
-     * @return Result of training loop step.
-     */
-    protected abstract ComputationsChain<LC, K, V, I, I> trainingLoopStep();
-
-    /**
-     * Condition specifying if training loop should continue.
-     *
-     * @param data First time, data returned by locallyProcessInitData then data returned by last step of loop.
-     * @param locCtx Local context.
-     * @return Boolean value indicating if training loop should continue.
-     */
-    protected abstract boolean shouldContinue(I data, LC locCtx);
-
-    /**
-     * Extract context for final result creation. Each key from the final keys set will be processed with
-     * finalResultsExtractor. While entry data (i.e. key and value) for each key varies, some data can be common for all
-     * processed entries. This data is called context.
-     *
-     * @param data Data returned from last training loop step.
-     * @param locCtx Local context.
-     * @return Context.
-     */
-    protected abstract IgniteSupplier<G> extractContextForFinalResultCreation(I data, LC locCtx);
-
-    /**
-     * Keys for final result creation.
-     *
-     * @param data Data returned from the last training loop step.
-     * @param locCtx Local context.
-     * @return Stream of keys for final result creation.
-     */
-    protected abstract IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> finalResultKeys(I data, LC locCtx);
-
-    /**
-     * Get function for extracting final result from each key specified in finalResultKeys.
-     *
-     * @return Function for extracting final result from each key specified in finalResultKeys.
-     */
-    protected abstract IgniteFunction<EntryAndContext<K, V, G>, ResultAndUpdates<R>> finalResultsExtractor();
-
-    /**
-     * Get function for reducing final results.
-     *
-     * @return Function for reducing final results.
-     */
-    protected abstract IgniteFunction<List<R>, R> finalResultsReducer();
-
-    /**
-     * Map final result to model which is returned by trainer.
-     *
-     * @param res Final result.
-     * @param locCtx Local context.
-     * @return Model resulted from training.
-     */
-    protected abstract M mapFinalResult(R res, LC locCtx);
-
-    /**
-     * Performs cleanups of temporary objects created by this trainer.
-     *
-     * @param locCtx Local context.
-     */
-    protected abstract void cleanup(LC locCtx);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java
deleted file mode 100644
index b192f42..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerBaseProcessorTask.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteException;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.compute.ComputeJob;
-import org.apache.ignite.compute.ComputeJobResult;
-import org.apache.ignite.compute.ComputeTaskAdapter;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.jetbrains.annotations.Nullable;
-
-/**
- * Base task for group trainer.
- *
- * @param <K> Type of cache keys of cache used for training.
- * @param <V> Type of cache values of cache used for training.
- * @param <C> Type of context (common part of data needed for computation).
- * @param <T> Type of arguments of workers.
- * @param <R> Type of computation result.
- */
-public abstract class GroupTrainerBaseProcessorTask<K, V, C, T, R extends Serializable> extends ComputeTaskAdapter<Void, R> {
-    /**
-     * Context supplier.
-     */
-    protected final IgniteSupplier<C> ctxSupplier;
-
-    /**
-     * UUID of training.
-     */
-    protected final UUID trainingUUID;
-
-    /**
-     * Worker.
-     */
-    protected IgniteFunction<T, ResultAndUpdates<R>> worker;
-
-    /**
-     * Reducer used for reducing of computations on specified keys.
-     */
-    protected final IgniteFunction<List<R>, R> reducer;
-
-    /**
-     * Name of cache on which training is done.
-     */
-    protected final String cacheName;
-
-    /**
-     * Supplier of keys on which worker should be executed.
-     */
-    protected final IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysSupplier;
-
-    /**
-     * Ignite instance.
-     */
-    protected final Ignite ignite;
-
-    /**
-     * Construct an instance of this class with specified parameters.
-     *
-     * @param trainingUUID UUID of training.
-     * @param ctxSupplier Supplier of context.
-     * @param worker Function calculated on each of specified keys.
-     * @param keysSupplier Supplier of keys on which training is done.
-     * @param reducer Reducer used for reducing results of computation performed on each of specified keys.
-     * @param cacheName Name of cache on which training is done.
-     * @param ignite Ignite instance.
-     */
-    public GroupTrainerBaseProcessorTask(UUID trainingUUID,
-        IgniteSupplier<C> ctxSupplier,
-        IgniteFunction<T, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysSupplier,
-        IgniteFunction<List<R>, R> reducer,
-        String cacheName,
-        Ignite ignite) {
-        this.trainingUUID = trainingUUID;
-        this.ctxSupplier = ctxSupplier;
-        this.worker = worker;
-        this.keysSupplier = keysSupplier;
-        this.reducer = reducer;
-        this.cacheName = cacheName;
-        this.ignite = ignite;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
-        @Nullable Void arg) throws IgniteException {
-        Map<ComputeJob, ClusterNode> res = new HashMap<>();
-
-        for (ClusterNode node : subgrid) {
-            BaseLocalProcessorJob<K, V, T, R> job = createJob();
-            res.put(job, node);
-        }
-
-        return res;
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public R reduce(List<ComputeJobResult> results) throws IgniteException {
-        return reducer.apply(results.stream().map(res -> (R)res.getData()).filter(Objects::nonNull).collect(Collectors.toList()));
-    }
-
-    /**
-     * Create job for execution on subgrid.
-     *
-     * @return Job for execution on subgrid.
-     */
-    protected abstract BaseLocalProcessorJob<K, V, T, R> createJob();
-
-    /**
-     * Get affinity function of cache on which training is done.
-     *
-     * @return Affinity function of cache on which training is done.
-     */
-    protected Affinity<GroupTrainerCacheKey> affinity() {
-        return ignite.affinity(cacheName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java
deleted file mode 100644
index 5e4cb76..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerCacheKey.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.UUID;
-import org.apache.ignite.cache.affinity.AffinityKeyMapped;
-
-/**
- * Class used as a key for caches on which {@link GroupTrainer} works.
- * Structurally it is a triple: (nodeLocalEntityIndex, trainingUUID, data);
- * nodeLocalEntityIndex is used to map key to node;
- * trainingUUID is id of training;
- * data is some custom data stored in this key, for example if we want to store three neural networks on one node
- * for training with training UUID == trainingUUID, we can use keys
- * (1, trainingUUID, networkIdx1), (1, trainingUUID, networkIdx2), (1, trainingUUID, networkIdx3).
- *
- * @param <K> Type of data part of this key.
- */
-public class GroupTrainerCacheKey<K> {
-    /**
-     * Part of key for key-to-node affinity.
-     */
-    @AffinityKeyMapped
-    private Long nodeLocEntityIdx;
-
-    /**
-     * UUID of training.
-     */
-    private UUID trainingUUID;
-
-    /**
-     * Data.
-     */
-    K data;
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param nodeLocEntityIdx Part of key for key-to-node affinity.
-     * @param data Data.
-     * @param trainingUUID Training UUID.
-     */
-    public GroupTrainerCacheKey(long nodeLocEntityIdx, K data, UUID trainingUUID) {
-        this.nodeLocEntityIdx = nodeLocEntityIdx;
-        this.trainingUUID = trainingUUID;
-        this.data = data;
-    }
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param nodeLocEntityIdx Part of key for key-to-node affinity.
-     * @param data Data.
-     * @param trainingUUID Training UUID.
-     */
-    public GroupTrainerCacheKey(int nodeLocEntityIdx, K data, UUID trainingUUID) {
-        this((long)nodeLocEntityIdx, data, trainingUUID);
-    }
-
-    /**
-     * Get part of key used for key-to-node affinity.
-     *
-     * @return Part of key used for key-to-node affinity.
-     */
-    public Long nodeLocalEntityIndex() {
-        return nodeLocEntityIdx;
-    }
-
-    /**
-     * Get UUID of training.
-     *
-     * @return UUID of training.
-     */
-    public UUID trainingUUID() {
-        return trainingUUID;
-    }
-
-    /**
-     * Get data.
-     *
-     * @return Data.
-     */
-    public K data() {
-        return data;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        GroupTrainerCacheKey<?> key = (GroupTrainerCacheKey<?>)o;
-
-        if (nodeLocEntityIdx != null ? !nodeLocEntityIdx.equals(key.nodeLocEntityIdx) : key.nodeLocEntityIdx != null)
-            return false;
-        if (trainingUUID != null ? !trainingUUID.equals(key.trainingUUID) : key.trainingUUID != null)
-            return false;
-        return data != null ? data.equals(key.data) : key.data == null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = nodeLocEntityIdx != null ? nodeLocEntityIdx.hashCode() : 0;
-        res = 31 * res + (trainingUUID != null ? trainingUUID.hashCode() : 0);
-        res = 31 * res + (data != null ? data.hashCode() : 0);
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java
deleted file mode 100644
index daa396f..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerEntriesProcessorTask.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-
-/**
- * Task for processing entries of cache used for training.
- *
- * @param <K> Type of cache keys of cache used for training.
- * @param <V> Type of cache values of cache used for training.
- * @param <C> Type of context (common part of data needed for computation).
- * @param <R> Type of computation result.
- */
-public class GroupTrainerEntriesProcessorTask<K, V, C, R extends Serializable>
-    extends GroupTrainerBaseProcessorTask<K, V, C, EntryAndContext<K, V, C>, R> {
-    /**
-     * Construct instance of this class with given parameters.
-     *
-     * @param trainingUUID UUID of training.
-     * @param ctxSupplier Supplier of context.
-     * @param worker Function calculated on each of specified keys.
-     * @param keysSupplier Supplier of keys on which training is done.
-     * @param reducer Reducer used for reducing results of computation performed on each of specified keys.
-     * @param cacheName Name of cache on which training is done.
-     * @param ignite Ignite instance.
-     */
-    public GroupTrainerEntriesProcessorTask(UUID trainingUUID,
-        IgniteSupplier<C> ctxSupplier,
-        IgniteFunction<EntryAndContext<K, V, C>, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysSupplier,
-        IgniteFunction<List<R>, R> reducer,
-        String cacheName,
-        Ignite ignite) {
-        super(trainingUUID, ctxSupplier, worker, keysSupplier, reducer, cacheName, ignite);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected BaseLocalProcessorJob<K, V, EntryAndContext<K, V, C>, R> createJob() {
-        return new LocalEntriesProcessorJob<>(ctxSupplier, worker, keysSupplier, reducer, trainingUUID, cacheName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java
deleted file mode 100644
index ae75f16..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerInput.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-
-/**
- * Interface for {@link GroupTrainer} inputs.
- *
- * @param <K> Types of cache keys used for group training.
- */
-public interface GroupTrainerInput<K> {
-    /**
-     * Get supplier of stream of keys used for initialization of {@link GroupTrainer}.
-     *
-     * @param trainingUUID UUID of training.
-     * @return Supplier of stream of keys used for initialization of {@link GroupTrainer}.
-     */
-    IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> initialKeys(UUID trainingUUID);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java
deleted file mode 100644
index 7ac18f8..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainerKeysProcessorTask.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.KeyAndContext;
-
-/**
- * Task for processing entries of cache used for training.
- *
- * @param <K> Type of cache keys of cache used for training.
- * @param <C> Type of context (common part of data needed for computation).
- * @param <R> Type of computation result.
- */
-public class GroupTrainerKeysProcessorTask<K, C, R extends Serializable> extends GroupTrainerBaseProcessorTask<K, Object, C, KeyAndContext<K, C>, R> {
-    /**
-     * Construct instance of this class with specified parameters.
-     *
-     * @param trainingUUID UUID of training.
-     * @param ctxSupplier Context supplier.
-     * @param worker Function calculated on each of specified keys.
-     * @param keysSupplier Supplier of keys on which computations should be done.
-     * @param reducer Reducer used for reducing results of computation performed on each of specified keys.
-     * @param cacheName Name of cache on which training is done.
-     * @param ignite Ignite instance.
-     */
-    public GroupTrainerKeysProcessorTask(UUID trainingUUID,
-        IgniteSupplier<C> ctxSupplier,
-        IgniteFunction<KeyAndContext<K, C>, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysSupplier,
-        IgniteFunction<List<R>, R> reducer,
-        String cacheName,
-        Ignite ignite) {
-        super(trainingUUID, ctxSupplier, worker, keysSupplier, reducer, cacheName, ignite);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected BaseLocalProcessorJob<K, Object, KeyAndContext<K, C>, R> createJob() {
-        return new LocalKeysProcessorJob<>(ctxSupplier, worker, keysSupplier, reducer, trainingUUID, cacheName);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java
deleted file mode 100644
index cbd04b2..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/GroupTrainingContext.java
+++ /dev/null
@@ -1,98 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/**
- * Context for group training.
- *
- * @param <K> Type of keys of cache used for group training.
- * @param <V> Type of values of cache used for group training.
- * @param <L> Type of local context used for training.
- */
-public class GroupTrainingContext<K, V, L extends HasTrainingUUID> {
-    /**
-     * Local context.
-     */
-    private L locCtx;
-
-    /**
-     * Cache used for training.
-     */
-    private IgniteCache<GroupTrainerCacheKey<K>, V> cache;
-
-    /**
-     * Ignite instance.
-     */
-    private Ignite ignite;
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param locCtx Local context.
-     * @param cache Information about cache used for training.
-     * @param ignite Ignite instance.
-     */
-    public GroupTrainingContext(L locCtx, IgniteCache<GroupTrainerCacheKey<K>, V> cache, Ignite ignite) {
-        this.locCtx = locCtx;
-        this.cache = cache;
-        this.ignite = ignite;
-    }
-
-    /**
-     * Construct new training context with same parameters but with new cache.
-     *
-     * @param newCache New cache.
-     * @param <K1> Type of keys of new cache.
-     * @param <V1> Type of values of new cache.
-     * @return New training context with same parameters but with new cache.
-     */
-    public <K1, V1> GroupTrainingContext<K1, V1, L> withCache(IgniteCache<GroupTrainerCacheKey<K1>, V1> newCache) {
-        return new GroupTrainingContext<>(locCtx, newCache, ignite);
-    }
-
-    /**
-     * Get local context.
-     *
-     * @return Local context.
-     */
-    public L localContext() {
-        return locCtx;
-    }
-
-    /**
-     * Get cache used for training.
-     *
-     * @return Cache used for training.
-     */
-    public IgniteCache<GroupTrainerCacheKey<K>, V> cache() {
-        return cache;
-    }
-
-    /**
-     * Get ignite instance.
-     *
-     * @return Ignite instance.
-     */
-    public Ignite ignite() {
-        return ignite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java
deleted file mode 100644
index d035aa5..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalEntriesProcessorJob.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-
-/**
- * {@link BaseLocalProcessorJob} specified to entry processing.
- *
- * @param <K> Type of cache used for group training.
- * @param <V> Type of values used for group training.
- * @param <C> Type of context.
- * @param <R> Type of result returned by worker.
- */
-public class LocalEntriesProcessorJob<K, V, C, R extends Serializable> extends BaseLocalProcessorJob<K, V, EntryAndContext<K, V, C>, R> {
-    /**
-     * Supplier of context for worker.
-     */
-    private final IgniteSupplier<C> ctxSupplier;
-
-    /**
-     * Construct an instance of this class.
-     *
-     * @param ctxSupplier Supplier for context for worker.
-     * @param worker Worker.
-     * @param keySupplier Supplier of keys.
-     * @param reducer Reducer.
-     * @param trainingUUID UUID for training.
-     * @param cacheName Name of cache used for training.
-     */
-    public LocalEntriesProcessorJob(IgniteSupplier<C> ctxSupplier,
-        IgniteFunction<EntryAndContext<K, V, C>, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keySupplier,
-        IgniteFunction<List<R>, R> reducer,
-        UUID trainingUUID, String cacheName) {
-        super(worker, keySupplier, reducer, trainingUUID, cacheName);
-        this.ctxSupplier = ctxSupplier;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Stream<EntryAndContext<K, V, C>> toProcess() {
-        C ctx = ctxSupplier.get();
-
-        return selectLocalEntries().map(e -> new EntryAndContext<>(e, ctx));
-    }
-
-    /**
-     * Select entries for processing by worker.
-     *
-     * @return Entries for processing by worker.
-     */
-    private Stream<Map.Entry<GroupTrainerCacheKey<K>, V>> selectLocalEntries() {
-        Set<GroupTrainerCacheKey<K>> keys = keySupplier.get().
-            filter(k -> Objects.requireNonNull(affinity().mapKeyToNode(k)).isLocal()).
-            filter(k -> k.trainingUUID().equals(trainingUUID)).
-            collect(Collectors.toSet());
-
-        return cache().getAll(keys).entrySet().stream();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java
deleted file mode 100644
index cad53c9..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/LocalKeysProcessorJob.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Objects;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.KeyAndContext;
-
-/**
- * {@link BaseLocalProcessorJob} specified to keys processing.
- *
- * @param <K> Type of cache used for group training.
- * @param <V> Type of values used for group training.
- * @param <C> Type of context.
- * @param <R> Type of result returned by worker.
- */
-public class LocalKeysProcessorJob<K, V, C, R extends Serializable> extends BaseLocalProcessorJob<K, V, KeyAndContext<K, C>, R> {
-    /**
-     * Supplier of worker context.
-     */
-    private final IgniteSupplier<C> ctxSupplier;
-
-    /**
-     * Construct instance of this class with given arguments.
-     *
-     * @param worker Worker.
-     * @param keySupplier Supplier of keys.
-     * @param reducer Reducer.
-     * @param trainingUUID UUID of training.
-     * @param cacheName Name of cache used for training.
-     */
-    public LocalKeysProcessorJob(IgniteSupplier<C> ctxSupplier,
-        IgniteFunction<KeyAndContext<K, C>, ResultAndUpdates<R>> worker,
-        IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keySupplier,
-        IgniteFunction<List<R>, R> reducer,
-        UUID trainingUUID, String cacheName) {
-        super(worker, keySupplier, reducer, trainingUUID, cacheName);
-        this.ctxSupplier = ctxSupplier;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Stream<KeyAndContext<K, C>> toProcess() {
-        C ctx = ctxSupplier.get();
-
-        return selectLocalKeys().map(k -> new KeyAndContext<>(k, ctx));
-    }
-
-    /**
-     * Get subset of keys provided by keySupplier which are mapped to node on which code is executed.
-     *
-     * @return Subset of keys provided by keySupplier which are mapped to node on which code is executed.
-     */
-    private Stream<GroupTrainerCacheKey<K>> selectLocalKeys() {
-        return keySupplier.get().
-            filter(k -> Objects.requireNonNull(affinity().mapKeyToNode(k)).isLocal()).
-            filter(k -> k.trainingUUID().equals(trainingUUID));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java
deleted file mode 100644
index 0ab6d32..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/Metaoptimizer.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.List;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-
-/**
- * Class encapsulating data transformations in group training in {@link MetaoptimizerGroupTrainer}, which is adapter of
- * {@link GroupTrainer}.
- *
- * @param <LC> Local context of {@link GroupTrainer}.
- * @param <X> Type of data which is processed in training loop step.
- * @param <Y> Type of data returned by training loop step data processor.
- * @param <I> Type of data to which data returned by distributed initialization is mapped.
- * @param <D> Type of data returned by initialization.
- * @param <O> Type of data to which data returned by data processor is mapped.
- */
-public interface Metaoptimizer<LC, X, Y, I, D, O> {
-    /**
-     * Get function used to reduce distributed initialization results.
-     *
-     * @return Function used to reduce distributed initialization results.
-     */
-    IgniteFunction<List<D>, D> initialReducer();
-
-    /**
-     * Maps data returned by distributed initialization to data consumed by training loop step.
-     *
-     * @param data Data returned by distributed initialization.
-     * @param locCtx Local context.
-     * @return Mapping of data returned by distributed initialization to data consumed by training loop step.
-     */
-    I locallyProcessInitData(D data, LC locCtx);
-
-    /**
-     * Preprocess data for {@link MetaoptimizerGroupTrainer#dataProcessor()}.
-     *
-     * @return Preprocessed data for {@link MetaoptimizerGroupTrainer#dataProcessor()}.
-     */
-    default IgniteFunction<X, X> distributedPreprocessor() {
-        return x -> x;
-    }
-
-    /**
-     * Get function used to map values returned by {@link MetaoptimizerGroupTrainer#dataProcessor()}.
-     *
-     * @return Function used to map values returned by {@link MetaoptimizerGroupTrainer#dataProcessor()}.
-     */
-    IgniteFunction<Y, O> distributedPostprocessor();
-
-    /**
-     * Get binary operator used for reducing results returned by distributedPostprocessor.
-     *
-     * @return Binary operator used for reducing results returned by distributedPostprocessor.
-     */
-    IgniteFunction<List<O>, O> postProcessReducer();
-
-    /**
-     * Transform data returned by distributed part of training loop step into input fed into distributed part of training
-     * loop step.
-     *
-     * @param input Type of output of distributed part of training loop step.
-     * @param locCtx Local context.
-     * @return Result of transform data returned by distributed part of training loop step into input fed into distributed part of training
-     * loop step.
-     */
-    I localProcessor(O input, LC locCtx);
-
-    /**
-     * Returns value of predicate 'should training loop continue given previous step output and local context'.
-     *
-     * @param input Input of previous step.
-     * @param locCtx Local context.
-     * @return Value of predicate 'should training loop continue given previous step output and local context'.
-     */
-    boolean shouldContinue(I input, LC locCtx);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java
deleted file mode 100644
index 08e1f47..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerDistributedStep.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.DistributedEntryProcessingStep;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/**
- * Distributed step based on {@link Metaoptimizer}.
- *
- * @param <L> Type of local context.
- * @param <K> Type of data in {@link GroupTrainerCacheKey}.
- * @param <V> Type of values of cache on which training is done.
- * @param <G> Type of distributed context.
- * @param <I> Type of data to which data returned by distributed initialization is mapped (see {@link Metaoptimizer}).
- * @param <O> Type of data to which data returned by data processor is mapped (see {@link Metaoptimizer}).
- * @param <X> Type of data which is processed in training loop step (see {@link Metaoptimizer}).
- * @param <Y> Type of data returned by training loop step data processor (see {@link Metaoptimizer}).
- * @param <D> Type of data returned by initialization (see {@link Metaoptimizer}).
- */
-class MetaoptimizerDistributedStep<L extends HasTrainingUUID, K, V, G, I extends Serializable, O extends Serializable,
-    X, Y, D extends Serializable> implements DistributedEntryProcessingStep<L, K, V, G, I, O> {
-    /**
-     * {@link Metaoptimizer}.
-     */
-    private final Metaoptimizer<L, X, Y, I, D, O> metaoptimizer;
-
-    /**
-     * {@link MetaoptimizerGroupTrainer} for which this distributed step is used.
-     */
-    private final MetaoptimizerGroupTrainer<L, K, V, D, ?, I, ?, ?, G, O, X, Y> trainer;
-
-    /**
-     * Construct instance of this class with given parameters.
-     *
-     * @param metaoptimizer Metaoptimizer.
-     * @param trainer {@link MetaoptimizerGroupTrainer} for which this distributed step is used.
-     */
-    public MetaoptimizerDistributedStep(Metaoptimizer<L, X, Y, I, D, O> metaoptimizer,
-        MetaoptimizerGroupTrainer<L, K, V, D, ?, I, ?, ?, G, O, X, Y> trainer) {
-        this.metaoptimizer = metaoptimizer;
-        this.trainer = trainer;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSupplier<G> remoteContextSupplier(I input, L locCtx) {
-        return trainer.remoteContextExtractor(input, locCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFunction<EntryAndContext<K, V, G>, ResultAndUpdates<O>> worker() {
-        IgniteFunction<X, ResultAndUpdates<Y>> dataProcessor = trainer.dataProcessor();
-        IgniteFunction<X, X> preprocessor = metaoptimizer.distributedPreprocessor();
-        IgniteFunction<Y, O> postprocessor = metaoptimizer.distributedPostprocessor();
-        IgniteFunction<EntryAndContext<K, V, G>, X> ctxExtractor = trainer.trainingLoopStepDataExtractor();
-
-        return entryAndCtx -> {
-            X apply = ctxExtractor.apply(entryAndCtx);
-            preprocessor.apply(apply);
-            ResultAndUpdates<Y> res = dataProcessor.apply(apply);
-            O postprocessRes = postprocessor.apply(res.result());
-
-            return ResultAndUpdates.of(postprocessRes).setUpdates(res.updates());
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keys(I input, L locCtx) {
-        return trainer.keysToProcessInTrainingLoop(locCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFunction<List<O>, O> reducer() {
-        return metaoptimizer.postProcessReducer();
-    }
-}


[31/50] [abbrv] ignite git commit: IGNITE-8042: .NET: Thin client: authentication support - fix naming and inspections

Posted by ag...@apache.org.
IGNITE-8042: .NET: Thin client: authentication support - fix naming and inspections


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

Branch: refs/heads/ignite-7708
Commit: 9be3357c427f7839ce347831517327a572c0c18d
Parents: 44c4656
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Apr 13 12:44:17 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Apr 13 12:44:17 2018 +0300

----------------------------------------------------------------------
 .../Client/ClientConnectionTest.cs                    | 12 ++++++------
 .../Client/IgniteClientConfiguration.cs               |  4 ++--
 .../IgniteClientConfigurationSection.xsd              |  2 +-
 .../Apache.Ignite.Core/Impl/Client/ClientSocket.cs    | 14 +++++++-------
 4 files changed, 16 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9be3357c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index cb30f40..0a6b1a7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -98,11 +98,11 @@ namespace Apache.Ignite.Core.Tests.Client
 
                 cliCfg.Password = "ignite";
 
-                cliCfg.Username = null;
+                cliCfg.UserName = null;
                 ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
                 Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null"));
 
-                cliCfg.Username = "";
+                cliCfg.UserName = "";
                 ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
                 Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be empty"));
             }
@@ -118,12 +118,12 @@ namespace Apache.Ignite.Core.Tests.Client
             {
                 var cliCfg = SecureClientConfig();
 
-                cliCfg.Username = "invalid";
+                cliCfg.UserName = "invalid";
 
                 var ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
                 Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed);
 
-                cliCfg.Username = "ignite";
+                cliCfg.UserName = "ignite";
                 cliCfg.Password = "invalid";
 
                 ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
@@ -165,7 +165,7 @@ namespace Apache.Ignite.Core.Tests.Client
 
                 var cliCfg = SecureClientConfig();
 
-                cliCfg.Username = "my_User";
+                cliCfg.UserName = "my_User";
                 cliCfg.Password = "my_Password";
 
                 using (var cli = Ignition.StartClient(cliCfg))
@@ -532,7 +532,7 @@ namespace Apache.Ignite.Core.Tests.Client
             return new IgniteClientConfiguration()
             {
                 Host = "localhost",
-                Username = "ignite",
+                UserName = "ignite",
                 Password = "ignite"
             };
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9be3357c/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
index 3252495..80f26cf 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
@@ -91,7 +91,7 @@ namespace Apache.Ignite.Core.Client
             BinaryProcessor = cfg.BinaryProcessor;
             SslStreamFactory = cfg.SslStreamFactory;
 
-            Username = cfg.Username;
+            UserName = cfg.UserName;
             Password = cfg.Password;
         }
 
@@ -151,7 +151,7 @@ namespace Apache.Ignite.Core.Client
         /// <summary>
         /// Username to be used to connect to secured cluster.
         /// </summary>
-        public string Username { get; set; }
+        public string UserName { get; set; }
 
         /// <summary>
         /// Password to be used to connect to secured cluster.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9be3357c/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
index 7e6caff..b9a04b8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
@@ -237,7 +237,7 @@
                     <xs:documentation>Socket operation timeout. Zero or negative for infinite timeout.</xs:documentation>
                 </xs:annotation>
             </xs:attribute>
-            <xs:attribute name="username" type="xs:string">
+            <xs:attribute name="userName" type="xs:string">
                 <xs:annotation>
                     <xs:documentation>Username to be used to connect to secured cluster.</xs:documentation>
                 </xs:annotation>

http://git-wip-us.apache.org/repos/asf/ignite/blob/9be3357c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index bce681f..11d7942 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -124,11 +124,11 @@ namespace Apache.Ignite.Core.Impl.Client
         /// Validate configuration.
         /// </summary>
         /// <param name="cfg">Configuration.</param>
-        private void Validate(IgniteClientConfiguration cfg)
+        private static void Validate(IgniteClientConfiguration cfg)
         {
-            if (cfg.Username != null)
+            if (cfg.UserName != null)
             {
-                if (cfg.Username.Length == 0)
+                if (cfg.UserName.Length == 0)
                     throw new IgniteClientException("IgniteClientConfiguration.Username cannot be empty.");
 
                 if (cfg.Password == null)
@@ -140,8 +140,8 @@ namespace Apache.Ignite.Core.Impl.Client
                 if (cfg.Password.Length == 0)
                     throw new IgniteClientException("IgniteClientConfiguration.Password cannot be empty.");
 
-                if (cfg.Username == null)
-                    throw new IgniteClientException("IgniteClientConfiguration.Username cannot be null when Password is set.");
+                if (cfg.UserName == null)
+                    throw new IgniteClientException("IgniteClientConfiguration.UserName cannot be null when Password is set.");
             }
         }
 
@@ -262,7 +262,7 @@ namespace Apache.Ignite.Core.Impl.Client
         /// </summary>
         private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProtocolVersion version)
         {
-            bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.Username != null;
+            bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.UserName != null;
 
             // Send request.
             int messageLen;
@@ -284,7 +284,7 @@ namespace Apache.Ignite.Core.Impl.Client
                 {
                     var writer = BinaryUtils.Marshaller.StartMarshal(stream);
 
-                    writer.WriteString(clientConfiguration.Username);
+                    writer.WriteString(clientConfiguration.UserName);
                     writer.WriteString(clientConfiguration.Password);
 
                     BinaryUtils.Marshaller.FinishMarshal(writer);


[10/50] [abbrv] ignite git commit: IGNITE-7871 Check local join future on error. - Fixes #3793.

Posted by ag...@apache.org.
IGNITE-7871 Check local join future on error. - Fixes #3793.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 0e73fa2c10dcd96ff98279018bdd3f8b36568008
Parents: 74d2545
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Wed Apr 11 14:12:50 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Wed Apr 11 14:12:50 2018 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/latch/ExchangeLatchManager.java     | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0e73fa2c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
index c205cb1..404f88f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
@@ -104,7 +104,8 @@ public class ExchangeLatchManager {
 
             // First coordinator initialization.
             ctx.discovery().localJoinFuture().listen(f -> {
-                this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE);
+                if (f.error() == null)
+                    this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE);
             });
 
             ctx.event().addDiscoveryEventListener((e, cache) -> {


[45/50] [abbrv] ignite git commit: IGNITE-7691 .NET: Add attribute-based Scale and Precision tests

Posted by ag...@apache.org.
IGNITE-7691 .NET: Add attribute-based Scale and Precision tests


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

Branch: refs/heads/ignite-7708
Commit: 8cb69174338b3f62c11092ca89baa736f0b24989
Parents: 0991437
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Sun Apr 15 01:34:30 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Sun Apr 15 01:34:30 2018 +0300

----------------------------------------------------------------------
 .../Cache/Query/CacheQueriesCodeConfigurationTest.cs        | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8cb69174/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
index a05923d..1d03c68 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/Query/CacheQueriesCodeConfigurationTest.cs
@@ -118,6 +118,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             Assert.AreEqual(typeof(AttributeTest), qe.ValueType);
 
             var fields = qe.Fields.ToArray();
+            var idxField = fields.Single(x => x.Name == "IndexedField1");
 
             CollectionAssert.AreEquivalent(new[]
             {
@@ -126,7 +127,7 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             }, fields.Select(x => x.Name));
 
             Assert.IsTrue(fields.Single(x => x.Name == "SqlField").NotNull);
-            Assert.IsFalse(fields.Single(x => x.Name == "IndexedField1").NotNull);
+            Assert.IsFalse(idxField.NotNull);
 
             var idx = qe.Indexes.ToArray();
 
@@ -144,6 +145,9 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             Assert.AreEqual(-1, idx[1].InlineSize);
             Assert.AreEqual(513, idx[2].InlineSize);
             Assert.AreEqual(-1, idx[3].InlineSize);
+            
+            Assert.AreEqual(3, idxField.Precision);
+            Assert.AreEqual(4, idxField.Scale);
         }
 
         /// <summary>
@@ -320,7 +324,8 @@ namespace Apache.Ignite.Core.Tests.Cache.Query
             [QuerySqlField(NotNull = true)]
             public double SqlField { get; set; }
 
-            [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true, IndexInlineSize = 513)]
+            [QuerySqlField(IsIndexed = true, Name = "IndexedField1", IsDescending = true, IndexInlineSize = 513,
+                DefaultValue = 42, Precision = 3, Scale = 4)]
             public int IndexedField { get; set; }
 
             [QueryTextField]


[07/50] [abbrv] ignite git commit: IGNITE-7830: Knn Lin Reg with new datasets

Posted by ag...@apache.org.
IGNITE-7830: Knn Lin Reg with new datasets

this closes #3583


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

Branch: refs/heads/ignite-7708
Commit: a4653b7c1287a039206bf22e9d85125bb15bc412
Parents: 6557fe6
Author: zaleslaw <za...@gmail.com>
Authored: Wed Apr 11 12:31:48 2018 +0300
Committer: YuriBabak <y....@gmail.com>
Committed: Wed Apr 11 12:31:48 2018 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/ml/knn/KNNUtils.java |  59 ++++++++
 .../KNNClassificationTrainer.java               |  23 +--
 .../ml/knn/regression/KNNRegressionModel.java   |  87 +++++++++++
 .../ml/knn/regression/KNNRegressionTrainer.java |  40 ++++++
 .../ignite/ml/knn/regression/package-info.java  |  22 +++
 .../apache/ignite/ml/knn/KNNRegressionTest.java | 143 +++++++++++++++++++
 .../org/apache/ignite/ml/knn/KNNTestSuite.java  |   1 +
 7 files changed, 354 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
new file mode 100644
index 0000000..88fa70f
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
@@ -0,0 +1,59 @@
+/*
+ * 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.ml.knn;
+
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.PartitionDataBuilder;
+import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.structures.LabeledDataset;
+import org.apache.ignite.ml.structures.LabeledVector;
+import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Helper class for KNNRegression.
+ */
+public class KNNUtils {
+    /**
+     * Builds dataset.
+     *
+     * @param datasetBuilder Dataset builder.
+     * @param featureExtractor Feature extractor.
+     * @param lbExtractor Label extractor.
+     * @return Dataset.
+     */
+    @Nullable public static <K, V> Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> buildDataset(DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
+        PartitionDataBuilder<K, V, KNNPartitionContext, LabeledDataset<Double, LabeledVector>> partDataBuilder
+            = new LabeledDatasetPartitionDataBuilderOnHeap<>(
+            featureExtractor,
+            lbExtractor
+        );
+
+        Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset = null;
+
+        if (datasetBuilder != null) {
+            dataset = datasetBuilder.build(
+                (upstream, upstreamSize) -> new KNNPartitionContext(),
+                partDataBuilder
+            );
+        }
+        return dataset;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java
index 357047f..c0c8e65 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationTrainer.java
@@ -17,14 +17,9 @@
 
 package org.apache.ignite.ml.knn.classification;
 
-import org.apache.ignite.ml.dataset.Dataset;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
-import org.apache.ignite.ml.dataset.PartitionDataBuilder;
-import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
+import org.apache.ignite.ml.knn.KNNUtils;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.structures.LabeledDataset;
-import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap;
-import org.apache.ignite.ml.structures.LabeledVector;
 import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
 
 /**
@@ -41,20 +36,6 @@ public class KNNClassificationTrainer implements SingleLabelDatasetTrainer<KNNCl
      */
     @Override public <K, V> KNNClassificationModel fit(DatasetBuilder<K, V> datasetBuilder,
         IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
-        PartitionDataBuilder<K, V, KNNPartitionContext, LabeledDataset<Double, LabeledVector>> partDataBuilder
-            = new LabeledDatasetPartitionDataBuilderOnHeap<>(
-            featureExtractor,
-            lbExtractor
-        );
-
-        Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset = null;
-
-        if (datasetBuilder != null) {
-            dataset = datasetBuilder.build(
-                (upstream, upstreamSize) -> new KNNPartitionContext(),
-                partDataBuilder
-            );
-        }
-        return new KNNClassificationModel<>(dataset);
+        return new KNNClassificationModel<>(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
new file mode 100644
index 0000000..cabc143
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
@@ -0,0 +1,87 @@
+/*
+ * 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.ml.knn.regression;
+
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
+import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
+import org.apache.ignite.ml.math.Vector;
+import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException;
+import org.apache.ignite.ml.structures.LabeledDataset;
+import org.apache.ignite.ml.structures.LabeledVector;
+
+import java.util.List;
+
+/**
+ * This class provides kNN Multiple Linear Regression or Locally [weighted] regression (Simple and Weighted versions).
+ *
+ * <p> This is an instance-based learning method. </p>
+ *
+ * <ul>
+ *     <li>Local means using nearby points (i.e. a nearest neighbors approach).</li>
+ *     <li>Weighted means we value points based upon how far away they are.</li>
+ *     <li>Regression means approximating a function.</li>
+ * </ul>
+ */
+public class KNNRegressionModel<K,V> extends KNNClassificationModel<K,V> {
+    /**
+     * Builds the model via prepared dataset.
+     * @param dataset Specially prepared object to run algorithm over it.
+     */
+    public KNNRegressionModel(Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset) {
+        super(dataset);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Double apply(Vector v) {
+        List<LabeledVector> neighbors = findKNearestNeighbors(v);
+
+        return predictYBasedOn(neighbors, v);
+    }
+
+    /** */
+    private double predictYBasedOn(List<LabeledVector> neighbors, Vector v) {
+        switch (stgy) {
+            case SIMPLE:
+                return simpleRegression(neighbors);
+            case WEIGHTED:
+                return weightedRegression(neighbors, v);
+            default:
+                throw new UnsupportedOperationException("Strategy " + stgy.name() + " is not supported");
+        }
+    }
+
+    /** */
+    private double weightedRegression(List<LabeledVector> neighbors, Vector v) {
+        double sum = 0.0;
+        double div = 0.0;
+        for (LabeledVector<Vector, Double> neighbor : neighbors) {
+            double distance = distanceMeasure.compute(v, neighbor.features());
+            sum += neighbor.label() * distance;
+            div += distance;
+        }
+        return sum / div;
+    }
+
+    /** */
+    private double simpleRegression(List<LabeledVector> neighbors) {
+        double sum = 0.0;
+        for (LabeledVector<Vector, Double> neighbor : neighbors)
+            sum += neighbor.label();
+        return sum / (double)k;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java
new file mode 100644
index 0000000..2d13cd5
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionTrainer.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.ml.knn.regression;
+
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.knn.KNNUtils;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+
+/**
+ * kNN algorithm trainer to solve regression task.
+ */
+public class KNNRegressionTrainer{
+    /**
+     * Trains model based on the specified data.
+     *
+     * @param datasetBuilder Dataset builder.
+     * @param featureExtractor Feature extractor.
+     * @param lbExtractor Label extractor.
+     * @return Model.
+     */
+    public <K, V> KNNRegressionModel fit(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
+        return new KNNRegressionModel<>(KNNUtils.buildDataset(datasetBuilder, featureExtractor, lbExtractor));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/package-info.java
new file mode 100644
index 0000000..82e7192
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/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 helper classes for kNN regression algorithms.
+ */
+package org.apache.ignite.ml.knn.regression;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
new file mode 100644
index 0000000..66dbca9
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.ml.knn;
+
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
+import org.apache.ignite.ml.knn.classification.KNNStrategy;
+import org.apache.ignite.ml.knn.regression.KNNRegressionModel;
+import org.apache.ignite.ml.knn.regression.KNNRegressionTrainer;
+import org.apache.ignite.ml.math.Vector;
+import org.apache.ignite.ml.math.distances.EuclideanDistance;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.junit.Assert;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Tests for {@link KNNRegressionTrainer}.
+ */
+public class KNNRegressionTest extends BaseKNNTest {
+    /** */
+    private double[] y;
+
+    /** */
+    private double[][] x;
+
+    /** */
+    public void testSimpleRegressionWithOneNeighbour() {
+        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
+
+        Map<Integer, double[]> data = new HashMap<>();
+        data.put(0, new double[] {11.0, 0, 0, 0, 0, 0});
+        data.put(1, new double[] {12.0, 2.0, 0, 0, 0, 0});
+        data.put(2, new double[] {13.0, 0, 3.0, 0, 0, 0});
+        data.put(3, new double[] {14.0, 0, 0, 4.0, 0, 0});
+        data.put(4, new double[] {15.0, 0, 0, 0, 5.0, 0});
+        data.put(5, new double[] {16.0, 0, 0, 0, 0, 6.0});
+
+        KNNRegressionTrainer trainer = new KNNRegressionTrainer();
+
+        KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
+            new LocalDatasetBuilder<>(data, 2),
+            (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+            (k, v) -> v[0]
+        ).withK(1)
+            .withDistanceMeasure(new EuclideanDistance())
+            .withStrategy(KNNStrategy.SIMPLE);
+
+        Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0});
+        System.out.println(knnMdl.apply(vector));
+        Assert.assertEquals(15, knnMdl.apply(vector), 1E-12);
+    }
+
+    /** */
+    public void testLongly() {
+
+        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
+
+        Map<Integer, double[]> data = new HashMap<>();
+        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+
+        KNNRegressionTrainer trainer = new KNNRegressionTrainer();
+
+        KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
+            new LocalDatasetBuilder<>(data, 2),
+            (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+            (k, v) -> v[0]
+        ).withK(3)
+            .withDistanceMeasure(new EuclideanDistance())
+            .withStrategy(KNNStrategy.SIMPLE);
+
+        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
+        System.out.println(knnMdl.apply(vector));
+        Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
+    }
+
+    /** */
+    public void testLonglyWithWeightedStrategy() {
+        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
+
+        Map<Integer, double[]> data = new HashMap<>();
+        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+
+        KNNRegressionTrainer trainer = new KNNRegressionTrainer();
+
+        KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
+            new LocalDatasetBuilder<>(data, 2),
+            (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+            (k, v) -> v[0]
+        ).withK(3)
+            .withDistanceMeasure(new EuclideanDistance())
+            .withStrategy(KNNStrategy.SIMPLE);
+
+        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
+        System.out.println(knnMdl.apply(vector));
+        Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a4653b7c/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java
index 95ebec5..55ef24e 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNTestSuite.java
@@ -26,6 +26,7 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     KNNClassificationTest.class,
+    KNNRegressionTest.class,
     LabeledDatasetTest.class
 })
 public class KNNTestSuite {


[28/50] [abbrv] ignite git commit: IGNITE-7983: NPE fixed in transactions

Posted by ag...@apache.org.
IGNITE-7983: NPE fixed in transactions

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 08a700db31604d33959e3b5aa3f709dbf66a878b
Parents: adaedb4
Author: Andrey Kuznetsov <st...@gmail.com>
Authored: Thu Apr 12 21:23:28 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Apr 12 21:23:28 2018 +0300

----------------------------------------------------------------------
 .../processors/cache/distributed/near/GridNearTxLocal.java  | 9 +++++++++
 1 file changed, 9 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/08a700db/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 33f84f0..fc8a9a3 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
@@ -3916,6 +3916,15 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
                         throw new GridClosureException(e);
                     }
 
+                    if (isRollbackOnly()) {
+                        if (timedOut())
+                            throw new GridClosureException(new IgniteTxTimeoutCheckedException(
+                                "Transaction has been timed out: " + GridNearTxLocal.this));
+                        else
+                            throw new GridClosureException(new IgniteTxRollbackCheckedException(
+                                "Transaction has been rolled back: " + GridNearTxLocal.this));
+                    }
+
                     return map;
                 }
             },


[46/50] [abbrv] ignite git commit: IGNITE-8129: MTCGA: setup default SSL context in JdbcthinConnectionSSLTest (because sometimes default SSL context may be setup by build system). This closes #3795.

Posted by ag...@apache.org.
IGNITE-8129: MTCGA: setup default SSL context in JdbcthinConnectionSSLTest (because sometimes default SSL context may be setup by build system). This closes #3795.


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

Branch: refs/heads/ignite-7708
Commit: af2b74215e8b0fc119537c6ba0a35977e0579b24
Parents: 8cb6917
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Mon Apr 16 11:28:39 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Apr 16 11:28:39 2018 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinConnectionSSLTest.java    | 20 ++++++++++----------
 1 file changed, 10 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/af2b7421/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java
index cc71f51..355a198 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSSLTest.java
@@ -164,7 +164,14 @@ public class JdbcThinConnectionSSLTest extends JdbcThinAbstractSelfTest {
      * @throws Exception If failed.
      */
     public void testDefaultContext() throws Exception {
+        // Store exists default SSL context to restore after test.
+        final SSLContext dfltSslCtx = SSLContext.getDefault();
+
+        // Setup default context
+        SSLContext.setDefault(getTestSslContextFactory().create());
+
         setSslCtxFactoryToCli = true;
+
         // Factory return default SSL context
         sslCtxFactory = new Factory<SSLContext>() {
             @Override public SSLContext create() {
@@ -177,23 +184,16 @@ public class JdbcThinConnectionSSLTest extends JdbcThinAbstractSelfTest {
             }
         };
 
-        System.setProperty("javax.net.ssl.keyStore", CLI_KEY_STORE_PATH);
-        System.setProperty("javax.net.ssl.keyStorePassword", "123456");
-        System.setProperty("javax.net.ssl.trustStore", TRUST_KEY_STORE_PATH);
-        System.setProperty("javax.net.ssl.trustStorePassword", "123456");
-
         startGrids(1);
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1/?sslMode=require")) {
             checkConnection(conn);
         }
         finally {
-            System.getProperties().remove("javax.net.ssl.keyStore");
-            System.getProperties().remove("javax.net.ssl.keyStorePassword");
-            System.getProperties().remove("javax.net.ssl.trustStore");
-            System.getProperties().remove("javax.net.ssl.trustStorePassword");
-
             stopAllGrids();
+
+            // Restore SSL context.
+            SSLContext.setDefault(dfltSslCtx);
         }
     }
 


[15/50] [abbrv] ignite git commit: IGNITE-8148: JDBC thin: semicolon as delimiter for properties. This closes #3794.

Posted by ag...@apache.org.
IGNITE-8148: JDBC thin: semicolon as delimiter for properties. This closes #3794.


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

Branch: refs/heads/ignite-7708
Commit: 14402e4030cf0257b72c636894f140f346266299
Parents: 5a29276
Author: devozerov <vo...@gridgain.com>
Authored: Wed Apr 11 16:44:33 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Apr 11 16:44:33 2018 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   | 233 +++++++++++++++----
 .../jdbc/thin/ConnectionPropertiesImpl.java     | 161 +++++++++----
 2 files changed, 300 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/14402e40/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
index 14b91b2..ed0b324 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinConnectionSelfTest.java
@@ -185,86 +185,136 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * Test invalid socket buffer sizes with semicolon.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSocketBuffersSemicolon() throws Exception {
+        final int dfltDufSize = 64 * 1024;
+
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;socketSendBuffer=-1",
+            "Property cannot be lower than 0 [name=socketSendBuffer, value=-1]");
+
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;socketReceiveBuffer=-1",
+            "Property cannot be lower than 0 [name=socketReceiveBuffer, value=-1]");
+
+        // Note that SO_* options are hints, so we check that value is equals to either what we set or to default.
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;socketSendBuffer=1024")) {
+            assertEquals(1024, io(conn).connectionProperties().getSocketSendBuffer());
+            assertEquals(dfltDufSize, io(conn).connectionProperties().getSocketReceiveBuffer());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;socketReceiveBuffer=1024")) {
+            assertEquals(dfltDufSize, io(conn).connectionProperties().getSocketSendBuffer());
+            assertEquals(1024, io(conn).connectionProperties().getSocketReceiveBuffer());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;" +
+            "socketSendBuffer=1024;socketReceiveBuffer=2048")) {
+            assertEquals(1024, io(conn).connectionProperties().getSocketSendBuffer());
+            assertEquals(2048, io(conn).connectionProperties().getSocketReceiveBuffer());
+        }
+    }
+
+    /**
      * Test SQL hints.
      *
      * @throws Exception If failed.
      */
     public void testSqlHints() throws Exception {
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, false, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true")) {
-            assertTrue(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, true, false, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?enforceJoinOrder=true")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertTrue(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, true, false, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?collocated=true")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertTrue(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, false, true, false, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?replicatedOnly=true")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertTrue(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, false, false, true, false, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?lazy=true")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertTrue(io(conn).connectionProperties().isLazy());
-            assertFalse(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, false, false, false, true, false);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?skipReducerOnUpdate=true")) {
-            assertFalse(io(conn).connectionProperties().isDistributedJoins());
-            assertFalse(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertFalse(io(conn).connectionProperties().isCollocated());
-            assertFalse(io(conn).connectionProperties().isReplicatedOnly());
-            assertFalse(io(conn).connectionProperties().isLazy());
-            assertTrue(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, false, false, false, false, false, true);
         }
 
         try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1?distributedJoins=true&" +
             "enforceJoinOrder=true&collocated=true&replicatedOnly=true&lazy=true&skipReducerOnUpdate=true")) {
-            assertTrue(io(conn).connectionProperties().isDistributedJoins());
-            assertTrue(io(conn).connectionProperties().isEnforceJoinOrder());
-            assertTrue(io(conn).connectionProperties().isCollocated());
-            assertTrue(io(conn).connectionProperties().isReplicatedOnly());
-            assertTrue(io(conn).connectionProperties().isLazy());
-            assertTrue(io(conn).connectionProperties().isSkipReducerOnUpdate());
+            assertHints(conn, true, true, true, true, true, true);
         }
     }
 
     /**
+     * Test SQL hints with semicolon.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSqlHintsSemicolon() throws Exception {
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true")) {
+            assertHints(conn, true, false, false, false, false, false);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;enforceJoinOrder=true")) {
+            assertHints(conn, false, true, false, false, false, false);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;collocated=true")) {
+            assertHints(conn, false, false, true, false, false, false);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;replicatedOnly=true")) {
+            assertHints(conn, false, false, false, true, false, false);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;lazy=true")) {
+            assertHints(conn, false, false, false, false, true, false);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;skipReducerOnUpdate=true")) {
+            assertHints(conn, false, false, false, false, false, true);
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;distributedJoins=true;" +
+            "enforceJoinOrder=true;collocated=true;replicatedOnly=true;lazy=true;skipReducerOnUpdate=true")) {
+            assertHints(conn, true, true, true, true, true, true);
+        }
+    }
+
+    /**
+     * Assert hints.
+     *
+     * @param conn Connection.
+     * @param distributedJoins Distributed joins.
+     * @param enforceJoinOrder Enforce join order.
+     * @param collocated Co-located.
+     * @param replicatedOnly Replicated only.
+     * @param lazy Lazy.
+     * @param skipReducerOnUpdate Skip reducer on update.
+     * @throws Exception If failed.
+     */
+    private void assertHints(Connection conn, boolean distributedJoins, boolean enforceJoinOrder, boolean collocated,
+        boolean replicatedOnly, boolean lazy, boolean skipReducerOnUpdate)throws Exception {
+        assertEquals(distributedJoins, io(conn).connectionProperties().isDistributedJoins());
+        assertEquals(enforceJoinOrder, io(conn).connectionProperties().isEnforceJoinOrder());
+        assertEquals(collocated, io(conn).connectionProperties().isCollocated());
+        assertEquals(replicatedOnly, io(conn).connectionProperties().isReplicatedOnly());
+        assertEquals(lazy, io(conn).connectionProperties().isLazy());
+        assertEquals(skipReducerOnUpdate, io(conn).connectionProperties().isSkipReducerOnUpdate());
+    }
+
+    /**
      * Test TCP no delay property handling.
      *
      * @throws Exception If failed.
@@ -304,6 +354,41 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * Test TCP no delay property handling with semicolon.
+     *
+     * @throws Exception If failed.
+     */
+    public void testTcpNoDelaySemicolon() throws Exception {
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=0",
+            "Invalid property value. [name=tcpNoDelay, val=0, choices=[true, false]]");
+
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=1",
+            "Invalid property value. [name=tcpNoDelay, val=1, choices=[true, false]]");
+
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=false1",
+            "Invalid property value. [name=tcpNoDelay, val=false1, choices=[true, false]]");
+
+        assertInvalid("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=true1",
+            "Invalid property value. [name=tcpNoDelay, val=true1, choices=[true, false]]");
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=true")) {
+            assertTrue(io(conn).connectionProperties().isTcpNoDelay());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=True")) {
+            assertTrue(io(conn).connectionProperties().isTcpNoDelay());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=false")) {
+            assertFalse(io(conn).connectionProperties().isTcpNoDelay());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;tcpNoDelay=False")) {
+            assertFalse(io(conn).connectionProperties().isTcpNoDelay());
+        }
+    }
+
+    /**
      * Test autoCloseServerCursor property handling.
      *
      * @throws Exception If failed.
@@ -340,6 +425,38 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * Test autoCloseServerCursor property handling with semicolon.
+     *
+     * @throws Exception If failed.
+     */
+    public void testAutoCloseServerCursorPropertySemicolon() throws Exception {
+        String url = "jdbc:ignite:thin://127.0.0.1;autoCloseServerCursor";
+
+        String err = "Invalid property value. [name=autoCloseServerCursor";
+
+        assertInvalid(url + "=0", err);
+        assertInvalid(url + "=1", err);
+        assertInvalid(url + "=false1", err);
+        assertInvalid(url + "=true1", err);
+
+        try (Connection conn = DriverManager.getConnection(url + "=true")) {
+            assertTrue(io(conn).connectionProperties().isAutoCloseServerCursor());
+        }
+
+        try (Connection conn = DriverManager.getConnection(url + "=True")) {
+            assertTrue(io(conn).connectionProperties().isAutoCloseServerCursor());
+        }
+
+        try (Connection conn = DriverManager.getConnection(url + "=false")) {
+            assertFalse(io(conn).connectionProperties().isAutoCloseServerCursor());
+        }
+
+        try (Connection conn = DriverManager.getConnection(url + "=False")) {
+            assertFalse(io(conn).connectionProperties().isAutoCloseServerCursor());
+        }
+    }
+
+    /**
      * Test schema property in URL.
      *
      * @throws Exception If failed.
@@ -362,6 +479,25 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
     }
 
     /**
+     * Test schema property in URL with semicolon.
+     *
+     * @throws Exception If failed.
+     */
+    public void testSchemaSemicolon() throws Exception {
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=public")) {
+            assertEquals("Invalid schema", "PUBLIC", conn.getSchema());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=\"" + DEFAULT_CACHE_NAME + '"')) {
+            assertEquals("Invalid schema", DEFAULT_CACHE_NAME, conn.getSchema());
+        }
+
+        try (Connection conn = DriverManager.getConnection("jdbc:ignite:thin://127.0.0.1;schema=_not_exist_schema_")) {
+            assertEquals("Invalid schema", "_NOT_EXIST_SCHEMA_", conn.getSchema());
+        }
+    }
+
+    /**
      * Get client socket for connection.
      *
      * @param conn Connection.
@@ -1010,6 +1146,7 @@ public class JdbcThinConnectionSelfTest extends JdbcThinAbstractSelfTest {
             // Invalid parameter value
             GridTestUtils.assertThrows(log,
                 new Callable<Object>() {
+                    @SuppressWarnings("MagicConstant")
                     @Override public Object call() throws Exception {
                         conn.setTransactionIsolation(-1);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/14402e40/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
index 5d77005..86dc298 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
@@ -23,8 +23,6 @@ import java.sql.SQLException;
 import java.util.Arrays;
 import java.util.Properties;
 import java.util.StringTokenizer;
-import javax.naming.RefAddr;
-import javax.naming.Reference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.configuration.ClientConnectorConfiguration;
 import org.apache.ignite.internal.processors.odbc.SqlStateCode;
@@ -44,6 +42,9 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
     /** Default socket buffer size. */
     private static final int DFLT_SOCK_BUFFER_SIZE = 64 * 1024;
 
+    /** Property: schema. */
+    private static final String PROP_SCHEMA = "schema";
+
     /** Connection URL. */
     private String url;
 
@@ -51,7 +52,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
     private HostAndPortRange [] addrs;
 
     /** Schema name. Hidden property. Is used to set default schema name part of the URL. */
-    private StringProperty schema = new StringProperty("schema",
+    private StringProperty schema = new StringProperty(PROP_SCHEMA,
         "Schema name of the connection", "PUBLIC", null, false, null);
 
     /** Distributed joins property. */
@@ -487,21 +488,113 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
 
         String nakedUrl = url.substring(JdbcThinUtils.URL_PREFIX.length()).trim();
 
-        int pathPartEndPos = nakedUrl.indexOf('?');
+        parseUrl0(nakedUrl, props);
+    }
+
+    /**
+     * Parse naked URL (i.e. without {@link JdbcThinUtils#URL_PREFIX}).
+     *
+     * @param url Naked URL.
+     * @param props Properties.
+     * @throws SQLException If failed.
+     */
+    private void parseUrl0(String url, Properties props) throws SQLException {
+        // Determine mode - semicolon or ampersand.
+        int semicolonPos = url.indexOf(";");
+        int slashPos = url.indexOf("/");
+        int queryPos = url.indexOf("?");
+
+        boolean semicolonMode;
+
+        if (semicolonPos == -1 && slashPos == -1 && queryPos == -1)
+            // No special char -> any mode could be used, choose semicolon for simplicity.
+            semicolonMode = true;
+        else {
+            if (semicolonPos != -1) {
+                // Use semicolon mode if it appears earlier than slash or query.
+                semicolonMode =
+                    (slashPos == -1 || semicolonPos < slashPos) && (queryPos == -1 || semicolonPos < queryPos);
+            }
+            else
+                // Semicolon is not found.
+                semicolonMode = false;
+        }
+
+        if (semicolonMode)
+            parseUrlWithSemicolon(url, props);
+        else
+            parseUrlWithQuery(url, props);
+    }
+
+    /**
+     * Parse URL in semicolon mode.
+     *
+     * @param url Naked URL
+     * @param props Properties.
+     * @throws SQLException If failed.
+     */
+    private void parseUrlWithSemicolon(String url, Properties props) throws SQLException {
+        int pathPartEndPos = url.indexOf(';');
 
         if (pathPartEndPos == -1)
-            pathPartEndPos = nakedUrl.length();
+            pathPartEndPos = url.length();
 
-        String pathPart = nakedUrl.substring(0, pathPartEndPos);
+        String pathPart = url.substring(0, pathPartEndPos);
 
         String paramPart = null;
 
-        if (pathPartEndPos > 0 && pathPartEndPos < nakedUrl.length())
-            paramPart = nakedUrl.substring(pathPartEndPos + 1, nakedUrl.length());
+        if (pathPartEndPos > 0 && pathPartEndPos < url.length())
+            paramPart = url.substring(pathPartEndPos + 1, url.length());
+
+        parseEndpoints(pathPart);
+
+        if (!F.isEmpty(paramPart))
+            parseParameters(paramPart, props, ";");
+    }
+
+    /**
+     * Parse URL in query mode.
+     *
+     * @param url Naked URL
+     * @param props Properties.
+     * @throws SQLException If failed.
+     */
+    private void parseUrlWithQuery(String url, Properties props) throws SQLException {
+        int pathPartEndPos = url.indexOf('?');
+
+        if (pathPartEndPos == -1)
+            pathPartEndPos = url.length();
+
+        String pathPart = url.substring(0, pathPartEndPos);
+
+        String paramPart = null;
+
+        if (pathPartEndPos > 0 && pathPartEndPos < url.length())
+            paramPart = url.substring(pathPartEndPos + 1, url.length());
 
         String[] pathParts = pathPart.split("/");
 
-        String [] endpoints = pathParts[0].split(",");
+        parseEndpoints(pathParts[0]);
+
+        if (pathParts.length > 2) {
+            throw new SQLException("Invalid URL format (only schema name is allowed in URL path parameter " +
+                "'host:port[/schemaName]'): " + this.url, SqlStateCode.CLIENT_CONNECTION_FAILED);
+        }
+
+        setSchema(pathParts.length == 2 ? pathParts[1] : null);
+
+        if (!F.isEmpty(paramPart))
+            parseParameters(paramPart, props, "&");
+    }
+
+    /**
+     * Parse endpoints.
+     *
+     * @param endpointStr Endpoint string.
+     * @throws SQLException If failed.
+     */
+    private void parseEndpoints(String endpointStr) throws SQLException {
+        String [] endpoints = endpointStr.split(",");
 
         if (endpoints.length > 0)
             addrs = new HostAndPortRange[endpoints.length];
@@ -519,16 +612,6 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
 
         if (F.isEmpty(addrs) || F.isEmpty(addrs[0].host()))
             throw new SQLException("Host name is empty", SqlStateCode.CLIENT_CONNECTION_FAILED);
-
-        if (pathParts.length > 2) {
-            throw new SQLException("Invalid URL format (only schema name is allowed in URL path parameter " +
-                "'host:port[/schemaName]'): " + url, SqlStateCode.CLIENT_CONNECTION_FAILED);
-        }
-
-        setSchema(pathParts.length == 2 ? pathParts[1] : null);
-
-        if (!F.isEmpty(paramPart))
-            parseParameters(paramPart, props);
     }
 
     /**
@@ -536,10 +619,11 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
      *
      * @param paramStr Parameters string.
      * @param props Properties.
+     * @param delimChar Delimiter character.
      * @throws SQLException If failed.
      */
-    private void parseParameters(String paramStr, Properties props) throws SQLException {
-        StringTokenizer st = new StringTokenizer(paramStr, "&");
+    private void parseParameters(String paramStr, Properties props, String delimChar) throws SQLException {
+        StringTokenizer st = new StringTokenizer(paramStr, delimChar);
 
         boolean insideBrace = false;
 
@@ -553,8 +637,8 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
                 int eqSymPos = token.indexOf('=');
 
                 if (eqSymPos < 0) {
-                    throw new SQLException("Invalid parameter format " +
-                        "(URL properties format: key0=value0&key1=value1&... etc. pair: " + token);
+                    throw new SQLException("Invalid parameter format (should be \"key1=val1" + delimChar +
+                        "key2=val2" + delimChar + "...\"): " + token);
                 }
 
                 if (eqSymPos == token.length())
@@ -570,7 +654,7 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
                 }
             }
             else
-                val += "&" + token;
+                val += delimChar + token;
 
             if (val.endsWith("}")) {
                 insideBrace = false;
@@ -587,22 +671,24 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
                 if (key.isEmpty() || val.isEmpty())
                     throw new SQLException("Invalid parameter format (key and value cannot be empty): " + token);
 
-                props.setProperty(PROP_PREFIX + key, val);
+                if (PROP_SCHEMA.equalsIgnoreCase(key))
+                    setSchema(val);
+                else
+                    props.setProperty(PROP_PREFIX + key, val);
             }
         }
     }
 
-
     /**
      * @return Driver's properties info array.
      */
     public DriverPropertyInfo[] getDriverPropertyInfo() {
-        DriverPropertyInfo[] dpis = new DriverPropertyInfo[propsArray.length];
+        DriverPropertyInfo[] infos = new DriverPropertyInfo[propsArray.length];
 
         for (int i = 0; i < propsArray.length; ++i)
-            dpis[i] = propsArray[i].getDriverPropertyInfo();
+            infos[i] = propsArray[i].getDriverPropertyInfo();
 
-        return dpis;
+        return infos;
     }
 
     /**
@@ -741,23 +827,6 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
         }
 
         /**
-         * @param ref Reference object.
-         * @throws SQLException On error.
-         */
-        void init(Reference ref) throws SQLException {
-            RefAddr refAddr = ref.get(name);
-
-            if (refAddr != null) {
-                String str = (String) refAddr.getContent();
-
-                if (validator != null)
-                    validator.validate(str);
-
-                init(str);
-            }
-        }
-
-        /**
          * @param str String representation of the
          * @throws SQLException on error.
          */


[17/50] [abbrv] ignite git commit: IGNITE-6892 OOM should be covered by failure handling

Posted by ag...@apache.org.
IGNITE-6892 OOM should be covered by failure handling

Signed-off-by: Andrey Gura <ag...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: d1be9b85507eb3358327e93b81031f92e660531b
Parents: 32fc6c3
Author: Aleksey Plekhanov <pl...@gmail.com>
Authored: Wed Apr 11 18:24:51 2018 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Wed Apr 11 18:24:51 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   8 +
 .../org/apache/ignite/internal/IgnitionEx.java  |  50 +++-
 .../discovery/GridDiscoveryManager.java         |   3 +
 .../processors/cache/WalStateManager.java       |   8 +-
 .../continuous/GridContinuousProcessor.java     |   3 +
 .../datastreamer/DataStreamProcessor.java       |   3 +
 .../processors/failure/FailureProcessor.java    |  11 +
 .../internal/processors/job/GridJobWorker.java  |   8 +-
 .../service/GridServiceProcessor.java           |  15 +-
 .../thread/IgniteStripedThreadPoolExecutor.java |   8 +-
 .../ignite/thread/IgniteThreadFactory.java      |  30 ++-
 .../ignite/thread/IgniteThreadPoolExecutor.java |  12 +-
 .../ignite/thread/OomExceptionHandler.java      |  44 ++++
 .../ignite/failure/OomFailureHandlerTest.java   | 255 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 15 files changed, 430 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 662338c..437f49f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -863,6 +863,14 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_BPLUS_TREE_LOCK_RETRIES = "IGNITE_BPLUS_TREE_LOCK_RETRIES";
 
     /**
+     * Amount of memory reserved in the heap at node start, which can be dropped to increase the chances of success when
+     * handling OutOfMemoryError.
+     *
+     * Default is {@code 64kb}.
+     */
+    public static final String IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE = "IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE";
+
+    /**
      * The threshold of uneven distribution above which partition distribution will be logged.
      *
      * The default is '50', that means: warn about nodes with 50+% difference.

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 10a0752..b3c3ee8 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
@@ -20,6 +20,7 @@ package org.apache.ignite.internal;
 import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.lang.management.ManagementFactory;
 import java.lang.reflect.Constructor;
 import java.net.MalformedURLException;
@@ -88,6 +89,7 @@ import org.apache.ignite.internal.util.typedef.CA;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -1764,6 +1766,13 @@ public class IgnitionEx {
 
             validateThreadPoolSize(cfg.getPublicThreadPoolSize(), "public");
 
+            UncaughtExceptionHandler oomeHnd = new UncaughtExceptionHandler() {
+                @Override public void uncaughtException(Thread t, Throwable e) {
+                    if (grid != null && X.hasCause(e, OutOfMemoryError.class))
+                        grid.context().failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                }
+            };
+
             execSvc = new IgniteThreadPoolExecutor(
                 "pub",
                 cfg.getIgniteInstanceName(),
@@ -1771,7 +1780,8 @@ public class IgnitionEx {
                 cfg.getPublicThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.PUBLIC_POOL);
+                GridIoPolicy.PUBLIC_POOL,
+                oomeHnd);
 
             execSvc.allowCoreThreadTimeOut(true);
 
@@ -1784,7 +1794,8 @@ public class IgnitionEx {
                 cfg.getServiceThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.SERVICE_POOL);
+                GridIoPolicy.SERVICE_POOL,
+                oomeHnd);
 
             svcExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1797,7 +1808,8 @@ public class IgnitionEx {
                 cfg.getSystemThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.SYSTEM_POOL);
+                GridIoPolicy.SYSTEM_POOL,
+                oomeHnd);
 
             sysExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1828,7 +1840,8 @@ public class IgnitionEx {
                 cfg.getManagementThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.MANAGEMENT_POOL);
+                GridIoPolicy.MANAGEMENT_POOL,
+                oomeHnd);
 
             mgmtExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1844,7 +1857,8 @@ public class IgnitionEx {
                 cfg.getPeerClassLoadingThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.P2P_POOL);
+                GridIoPolicy.P2P_POOL,
+                oomeHnd);
 
             p2pExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1879,7 +1893,8 @@ public class IgnitionEx {
             callbackExecSvc = new IgniteStripedThreadPoolExecutor(
                 cfg.getAsyncCallbackPoolSize(),
                 cfg.getIgniteInstanceName(),
-                "callback");
+                "callback",
+                oomeHnd);
 
             if (myCfg.getConnectorConfiguration() != null) {
                 validateThreadPoolSize(myCfg.getConnectorConfiguration().getThreadPoolSize(), "connector");
@@ -1890,7 +1905,9 @@ public class IgnitionEx {
                     myCfg.getConnectorConfiguration().getThreadPoolSize(),
                     myCfg.getConnectorConfiguration().getThreadPoolSize(),
                     DFLT_THREAD_KEEP_ALIVE_TIME,
-                    new LinkedBlockingQueue<Runnable>()
+                    new LinkedBlockingQueue<Runnable>(),
+                    GridIoPolicy.UNDEFINED,
+                    oomeHnd
                 );
 
                 restExecSvc.allowCoreThreadTimeOut(true);
@@ -1905,7 +1922,8 @@ public class IgnitionEx {
                 myCfg.getUtilityCacheThreadPoolSize(),
                 myCfg.getUtilityCacheKeepAliveTime(),
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.UTILITY_CACHE_POOL);
+                GridIoPolicy.UTILITY_CACHE_POOL,
+                oomeHnd);
 
             utilityCacheExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1916,7 +1934,8 @@ public class IgnitionEx {
                 1,
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.AFFINITY_POOL);
+                GridIoPolicy.AFFINITY_POOL,
+                oomeHnd);
 
             affExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1930,7 +1949,8 @@ public class IgnitionEx {
                     cpus * 2,
                     3000L,
                     new LinkedBlockingQueue<Runnable>(1000),
-                    GridIoPolicy.IDX_POOL
+                    GridIoPolicy.IDX_POOL,
+                    oomeHnd
                 );
             }
 
@@ -1943,7 +1963,8 @@ public class IgnitionEx {
                 cfg.getQueryThreadPoolSize(),
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.QUERY_POOL);
+                GridIoPolicy.QUERY_POOL,
+                oomeHnd);
 
             qryExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1954,7 +1975,8 @@ public class IgnitionEx {
                 2,
                 DFLT_THREAD_KEEP_ALIVE_TIME,
                 new LinkedBlockingQueue<Runnable>(),
-                GridIoPolicy.SCHEMA_POOL);
+                GridIoPolicy.SCHEMA_POOL,
+                oomeHnd);
 
             schemaExecSvc.allowCoreThreadTimeOut(true);
 
@@ -1970,7 +1992,9 @@ public class IgnitionEx {
                         execCfg.getSize(),
                         execCfg.getSize(),
                         DFLT_THREAD_KEEP_ALIVE_TIME,
-                        new LinkedBlockingQueue<Runnable>());
+                        new LinkedBlockingQueue<Runnable>(),
+                        GridIoPolicy.UNDEFINED,
+                        oomeHnd);
 
                     customExecSvcs.put(execCfg.getName(), exec);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 400bb5f..77c9657 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -130,6 +130,7 @@ import org.apache.ignite.spi.discovery.DiscoverySpiNodeAuthenticator;
 import org.apache.ignite.spi.discovery.DiscoverySpiOrderSupport;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.thread.IgniteThread;
+import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -924,6 +925,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             segChkThread = new IgniteThread(segChkWrk);
 
+            segChkThread.setUncaughtExceptionHandler(new OomExceptionHandler(ctx));
+
             segChkThread.start();
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
index 0ac699f..64a6819 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/WalStateManager.java
@@ -38,6 +38,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.thread.OomExceptionHandler;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
@@ -473,7 +474,12 @@ public class WalStateManager extends GridCacheSharedManagerAdapter {
                                     // not-yet-flushed dirty pages have been logged.
                                     WalStateChangeWorker worker = new WalStateChangeWorker(msg, cpFut);
 
-                                    new IgniteThread(worker).start();
+                                    IgniteThread thread = new IgniteThread(worker);
+
+                                    thread.setUncaughtExceptionHandler(new OomExceptionHandler(
+                                        cctx.kernalContext()));
+
+                                    thread.start();
                                 }
                                 else {
                                     // Disable: not-yet-flushed operations are not logged, so wait for them

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index cebe4b1..2d48b7d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -88,6 +88,7 @@ import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.JoiningNodeDiscoveryData;
 import org.apache.ignite.thread.IgniteThread;
+import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
@@ -1727,6 +1728,8 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     }
                 });
 
+                checker.setUncaughtExceptionHandler(new OomExceptionHandler(ctx));
+
                 bufCheckThreads.put(routineId, checker);
 
                 checker.start();

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 8b984c0..e63d7d4 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
@@ -44,6 +44,7 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.marshaller.Marshaller;
 import org.apache.ignite.stream.StreamReceiver;
 import org.apache.ignite.thread.IgniteThread;
+import org.apache.ignite.thread.OomExceptionHandler;
 import org.jetbrains.annotations.Nullable;
 
 import java.util.Collection;
@@ -125,6 +126,8 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
             }
         });
 
+        flusher.setUncaughtExceptionHandler(new OomExceptionHandler(ctx));
+
         flusher.start();
 
         if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
index 615fb9f..0234e84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/failure/FailureProcessor.java
@@ -19,12 +19,14 @@ package org.apache.ignite.internal.processors.failure;
 
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.failure.FailureContext;
 import org.apache.ignite.failure.FailureHandler;
 import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
 /**
@@ -40,6 +42,9 @@ public class FailureProcessor extends GridProcessorAdapter {
     /** Failure context. */
     private volatile FailureContext failureCtx;
 
+    /** Reserve buffer, which can be dropped to handle OOME. */
+    private volatile byte[] reserveBuf;
+
     /**
      * @param ctx Context.
      */
@@ -56,6 +61,9 @@ public class FailureProcessor extends GridProcessorAdapter {
         if (hnd == null)
             hnd = getDefaultFailureHandler();
 
+        reserveBuf = new byte[IgniteSystemProperties.getInteger(
+            IgniteSystemProperties.IGNITE_FAILURE_HANDLER_RESERVE_BUFFER_SIZE, 64 * 1024)];
+
         assert hnd != null;
 
         this.hnd = hnd;
@@ -102,6 +110,9 @@ public class FailureProcessor extends GridProcessorAdapter {
         U.error(ignite.log(), "Critical failure. Will be handled accordingly to configured handler [hnd=" +
             hnd.getClass() + ", failureCtx=" + failureCtx + ']', failureCtx.error());
 
+        if (reserveBuf != null && X.hasCause(failureCtx.error(), OutOfMemoryError.class))
+            reserveBuf = null;
+
         boolean invalidated = hnd.onFailure(ignite, failureCtx);
 
         if (invalidated) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
index 6d2e621..f7c07f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
@@ -36,6 +36,8 @@ import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.compute.ComputeJobMasterLeaveAware;
 import org.apache.ignite.compute.ComputeUserUndeclaredException;
 import org.apache.ignite.events.JobEvent;
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
 import org.apache.ignite.igfs.IgfsOutOfSpaceException;
 import org.apache.ignite.internal.GridInternalException;
 import org.apache.ignite.internal.GridJobContextImpl;
@@ -603,9 +605,13 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject {
                         X.hasCause(e, ClusterTopologyCheckedException.class))
                         // Should be throttled, because GridServiceProxy continuously retry getting service.
                         LT.error(log, e, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']');
-                    else
+                    else {
                         U.error(log, "Failed to execute job [jobId=" + ses.getJobId() + ", ses=" + ses + ']', e);
 
+                        if (X.hasCause(e, OutOfMemoryError.class))
+                            ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+                    }
+
                     ex = e;
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 ff68e72..63f5027 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.service;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -103,6 +104,7 @@ import org.apache.ignite.services.ServiceConfiguration;
 import org.apache.ignite.services.ServiceDeploymentException;
 import org.apache.ignite.services.ServiceDescriptor;
 import org.apache.ignite.thread.IgniteThreadFactory;
+import org.apache.ignite.thread.OomExceptionHandler;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
 import java.util.concurrent.ConcurrentHashMap;
@@ -112,7 +114,6 @@ import static org.apache.ignite.IgniteSystemProperties.getString;
 import static org.apache.ignite.configuration.DeploymentMode.ISOLATED;
 import static org.apache.ignite.configuration.DeploymentMode.PRIVATE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_SERVICES_COMPATIBILITY_MODE;
-import static org.apache.ignite.internal.processors.cache.GridCacheUtils.UTILITY_CACHE_NAME;
 import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
 import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
 import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
@@ -154,8 +155,12 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     /** Busy lock. */
     private volatile GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
+    /** Uncaught exception handler for thread pools. */
+    private final UncaughtExceptionHandler oomeHnd = new OomExceptionHandler(ctx);
+
     /** Thread factory. */
-    private ThreadFactory threadFactory = new IgniteThreadFactory(ctx.igniteInstanceName(), "service");
+    private ThreadFactory threadFactory = new IgniteThreadFactory(ctx.igniteInstanceName(), "service",
+        oomeHnd);
 
     /** Thread local for service name. */
     private ThreadLocal<String> svcName = new ThreadLocal<>();
@@ -175,7 +180,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
     public GridServiceProcessor(GridKernalContext ctx) {
         super(ctx);
 
-        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), "srvc-deploy"));
+        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(),
+            "srvc-deploy", oomeHnd));
 
         String servicesCompatibilityMode = getString(IGNITE_SERVICES_COMPATIBILITY_MODE);
 
@@ -373,7 +379,8 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
 
         busyLock = new GridSpinBusyLock();
 
-        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(), "srvc-deploy"));
+        depExe = Executors.newSingleThreadExecutor(new IgniteThreadFactory(ctx.igniteInstanceName(),
+            "srvc-deploy", oomeHnd));
 
         start();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java
index 3cd7484..418812f 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteStripedThreadPoolExecutor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.thread;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
@@ -45,10 +46,11 @@ public class IgniteStripedThreadPoolExecutor implements ExecutorService {
      * @param igniteInstanceName Node name.
      * @param threadNamePrefix Thread name prefix.
      */
-    public IgniteStripedThreadPoolExecutor(int concurrentLvl, String igniteInstanceName, String threadNamePrefix) {
+    public IgniteStripedThreadPoolExecutor(int concurrentLvl, String igniteInstanceName, String threadNamePrefix,
+        UncaughtExceptionHandler eHnd) {
         execs = new ExecutorService[concurrentLvl];
 
-        ThreadFactory factory = new IgniteThreadFactory(igniteInstanceName, threadNamePrefix);
+        ThreadFactory factory = new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, eHnd);
 
         for (int i = 0; i < concurrentLvl; i++)
             execs[i] = Executors.newSingleThreadExecutor(factory);
@@ -173,4 +175,4 @@ public class IgniteStripedThreadPoolExecutor implements ExecutorService {
     @Override public String toString() {
         return S.toString(IgniteStripedThreadPoolExecutor.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 062c973..23bf14d 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
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.thread;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.NotNull;
@@ -41,6 +41,9 @@ public class IgniteThreadFactory implements ThreadFactory {
     /** */
     private final byte plc;
 
+    /** Exception handler. */
+    private final UncaughtExceptionHandler eHnd;
+
     /**
      * Constructs new thread factory for given grid. All threads will belong
      * to the same default thread group.
@@ -49,7 +52,19 @@ public class IgniteThreadFactory implements ThreadFactory {
      * @param threadName Thread name.
      */
     public IgniteThreadFactory(String igniteInstanceName, String threadName) {
-        this(igniteInstanceName, threadName, GridIoPolicy.UNDEFINED);
+        this(igniteInstanceName, threadName, null);
+    }
+
+    /**
+     * Constructs new thread factory for given grid. All threads will belong
+     * to the same default thread group.
+     *
+     * @param igniteInstanceName Ignite instance name.
+     * @param threadName Thread name.
+     * @param eHnd Uncaught exception handler.
+     */
+    public IgniteThreadFactory(String igniteInstanceName, String threadName, UncaughtExceptionHandler eHnd) {
+        this(igniteInstanceName, threadName, GridIoPolicy.UNDEFINED, eHnd);
     }
 
     /**
@@ -59,16 +74,23 @@ public class IgniteThreadFactory implements ThreadFactory {
      * @param igniteInstanceName Ignite instance name.
      * @param threadName Thread name.
      * @param plc {@link GridIoPolicy} for thread pool.
+     * @param eHnd Uncaught exception handler.
      */
-    public IgniteThreadFactory(String igniteInstanceName, String threadName, byte plc) {
+    public IgniteThreadFactory(String igniteInstanceName, String threadName, byte plc, UncaughtExceptionHandler eHnd) {
         this.igniteInstanceName = igniteInstanceName;
         this.threadName = threadName;
         this.plc = plc;
+        this.eHnd = eHnd;
     }
 
     /** {@inheritDoc} */
     @Override public Thread newThread(@NotNull Runnable r) {
-        return new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet(), -1, plc);
+        Thread thread = new IgniteThread(igniteInstanceName, threadName, r, idxGen.incrementAndGet(), -1, plc);
+
+        if (eHnd != null)
+            thread.setUncaughtExceptionHandler(eHnd);
+
+        return thread;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
index 83c64c3..fed77ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadPoolExecutor.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.thread;
 
+import java.lang.Thread.UncaughtExceptionHandler;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadFactory;
@@ -53,7 +54,8 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
             maxPoolSize,
             keepAliveTime,
             workQ,
-            GridIoPolicy.UNDEFINED);
+            GridIoPolicy.UNDEFINED,
+            null);
     }
 
     /**
@@ -68,6 +70,7 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
      * @param workQ The queue to use for holding tasks before they are executed. This queue will hold only
      *      runnable tasks submitted by the {@link #execute(Runnable)} method.
      * @param plc {@link GridIoPolicy} for thread pool.
+     * @param eHnd Uncaught exception handler for thread pool.
      */
     public IgniteThreadPoolExecutor(
         String threadNamePrefix,
@@ -76,14 +79,15 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
         int maxPoolSize,
         long keepAliveTime,
         BlockingQueue<Runnable> workQ,
-        byte plc) {
+        byte plc,
+        UncaughtExceptionHandler eHnd) {
         super(
             corePoolSize,
             maxPoolSize,
             keepAliveTime,
             TimeUnit.MILLISECONDS,
             workQ,
-            new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, plc)
+            new IgniteThreadFactory(igniteInstanceName, threadNamePrefix, plc, eHnd)
         );
     }
 
@@ -114,4 +118,4 @@ public class IgniteThreadPoolExecutor extends ThreadPoolExecutor {
             new AbortPolicy()
         );
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java b/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java
new file mode 100644
index 0000000..3a62ad8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/thread/OomExceptionHandler.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.thread;
+
+import org.apache.ignite.failure.FailureContext;
+import org.apache.ignite.failure.FailureType;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.util.typedef.X;
+
+/**
+ * OOM exception handler for system threads.
+ */
+public class OomExceptionHandler implements Thread.UncaughtExceptionHandler {
+    /** Context. */
+    private final GridKernalContext ctx;
+
+    /**
+     * @param ctx Context.
+     */
+    public OomExceptionHandler(GridKernalContext ctx) {
+        this.ctx = ctx;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void uncaughtException(Thread t, Throwable e) {
+        if (X.hasCause(e, OutOfMemoryError.class))
+            ctx.failure().process(new FailureContext(FailureType.CRITICAL_ERROR, e));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java b/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java
new file mode 100644
index 0000000..2af94b8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/failure/OomFailureHandlerTest.java
@@ -0,0 +1,255 @@
+/*
+ * 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.failure;
+
+import javax.cache.processor.EntryProcessor;
+import javax.cache.processor.EntryProcessorException;
+import javax.cache.processor.MutableEntry;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.events.Event;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.services.Service;
+import org.apache.ignite.services.ServiceContext;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ * Out of memory error failure handler test.
+ */
+public class OomFailureHandlerTest extends AbstractFailureHandlerTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration()
+            .setName(DEFAULT_CACHE_NAME)
+            .setCacheMode(CacheMode.PARTITIONED)
+            .setBackups(0)
+        );
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * Test OOME in IgniteCompute.
+     */
+    public void testComputeOomError() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        try {
+            IgniteFuture<Boolean> res = ignite0.compute(ignite0.cluster().forNodeId(ignite1.cluster().localNode().id()))
+                .callAsync(new IgniteCallable<Boolean>() {
+                    @Override public Boolean call() throws Exception {
+                        throw new OutOfMemoryError();
+                    }
+                });
+
+            res.get();
+        }
+        catch (Throwable ignore) {
+            // Expected.
+        }
+
+        assertFailureState(ignite0, ignite1);
+    }
+
+    /**
+     * Test OOME in EntryProcessor.
+     */
+    public void testEntryProcessorOomError() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.getOrCreateCache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, Integer> cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+
+        Integer key = primaryKey(cache1);
+
+        cache1.put(key, key);
+
+        try {
+            IgniteFuture fut = cache0.invokeAsync(key, new EntryProcessor<Integer, Integer, Object>() {
+                @Override public Object process(MutableEntry<Integer, Integer> entry,
+                    Object... arguments) throws EntryProcessorException {
+                    throw new OutOfMemoryError();
+                }
+            });
+
+            fut.get();
+        }
+        catch (Throwable ignore) {
+            // Expected.
+        }
+
+        assertFailureState(ignite0, ignite1);
+    }
+
+    /**
+     * Test OOME in service method invocation.
+     */
+    public void testServiceInvokeOomError() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        IgniteCache<Integer, Integer> cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+
+        Integer key = primaryKey(cache1);
+
+        ignite0.services().deployKeyAffinitySingleton("fail-invoke-service", new FailServiceImpl(false),
+            DEFAULT_CACHE_NAME, key);
+
+        FailService svc = ignite0.services().serviceProxy("fail-invoke-service", FailService.class, false);
+
+        try {
+            svc.fail();
+        }
+        catch (Throwable ignore) {
+            // Expected.
+        }
+
+        assertFailureState(ignite0, ignite1);
+    }
+
+    /**
+     * Test OOME in service execute.
+     */
+    public void testServiceExecuteOomError() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        IgniteCache<Integer, Integer> cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+
+        Integer key = primaryKey(cache1);
+
+        ignite0.services().deployKeyAffinitySingleton("fail-execute-service", new FailServiceImpl(true),
+            DEFAULT_CACHE_NAME, key);
+
+        assertFailureState(ignite0, ignite1);
+    }
+
+    /**
+     * Test OOME in event listener.
+     */
+    public void testEventListenerOomError() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        IgniteEx ignite1 = startGrid(1);
+
+        IgniteCache<Integer, Integer> cache0 = ignite0.getOrCreateCache(DEFAULT_CACHE_NAME);
+        IgniteCache<Integer, Integer> cache1 = ignite1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+
+        ignite1.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                throw new OutOfMemoryError();
+            }
+        }, EventType.EVT_CACHE_OBJECT_PUT);
+
+        Integer key = primaryKey(cache1);
+
+        try {
+            cache0.put(key, key);
+        }
+        catch (Throwable ignore) {
+            // Expected.
+        }
+
+        assertFailureState(ignite0, ignite1);
+    }
+
+    /**
+     * @param igniteWork Working ignite instance.
+     * @param igniteFail Failed ignite instance.
+     */
+    private static void assertFailureState(Ignite igniteWork, Ignite igniteFail) throws IgniteInterruptedCheckedException {
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return dummyFailureHandler(igniteFail).failure();
+            }
+        }, 5000L));
+
+        assertFalse(dummyFailureHandler(igniteWork).failure());
+    }
+
+    /**
+     *
+     */
+    private interface FailService extends Service {
+        /**
+         * Fail.
+         */
+        void fail();
+    }
+
+    /**
+     *
+     */
+    private static class FailServiceImpl implements FailService {
+        /** Fail on execute. */
+        private final boolean failOnExec;
+
+        /**
+         * @param failOnExec Fail on execute.
+         */
+        private FailServiceImpl(boolean failOnExec) {
+            this.failOnExec = failOnExec;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void fail() {
+            throw new OutOfMemoryError();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+            if (failOnExec)
+                throw new OutOfMemoryError();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/d1be9b85/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 c4b7d92..c388f1d 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
@@ -22,6 +22,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
 import org.apache.ignite.failure.FailureHandlerTriggeredTest;
 import org.apache.ignite.failure.IoomFailureHandlerTest;
+import org.apache.ignite.failure.OomFailureHandlerTest;
 import org.apache.ignite.failure.StopNodeFailureHandlerTest;
 import org.apache.ignite.failure.StopNodeOrHaltFailureHandlerTest;
 import org.apache.ignite.internal.ClassSetTest;
@@ -199,6 +200,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(StopNodeFailureHandlerTest.class);
         suite.addTestSuite(StopNodeOrHaltFailureHandlerTest.class);
         suite.addTestSuite(IoomFailureHandlerTest.class);
+        suite.addTestSuite(OomFailureHandlerTest.class);
 
         return suite;
     }


[12/50] [abbrv] ignite git commit: IGNITE-8106 Collect suppressed exceptions from causes. - Fixes #3735.

Posted by ag...@apache.org.
IGNITE-8106 Collect suppressed exceptions from causes. - Fixes #3735.

Signed-off-by: Alexey Kuznetsov <ak...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 98ef925933f392d419f70b2fcf51e3655b08b290
Parents: a3eb1f5
Author: Ilya Kasnacheev <il...@gmail.com>
Authored: Wed Apr 11 19:32:52 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Apr 11 19:32:52 2018 +0700

----------------------------------------------------------------------
 .../cluster/GridChangeStateCommandHandler.java  |  3 +-
 .../apache/ignite/internal/util/typedef/X.java  | 37 +++++++++++++++-----
 .../visor/util/VisorExceptionWrapper.java       | 11 +++---
 .../communication/tcp/TcpCommunicationSpi.java  |  2 +-
 .../ignite/GridSuppressedExceptionSelfTest.java | 23 +++++++++++-
 5 files changed, 59 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/98ef9259/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
index 7bb13d9..619be34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cluster/GridChangeStateCommandHandler.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandle
 import org.apache.ignite.internal.processors.rest.request.GridRestChangeStateRequest;
 import org.apache.ignite.internal.processors.rest.request.GridRestRequest;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
@@ -78,7 +79,7 @@ public class GridChangeStateCommandHandler extends GridRestCommandHandlerAdapter
 
             sb.a(e.getMessage()).a("\n").a("suppressed: \n");
 
-            for (Throwable t:e.getSuppressed())
+            for (Throwable t : X.getSuppressedList(e))
                 sb.a(t.getMessage()).a("\n");
 
             res.setError(sb.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/98ef9259/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
index 395de23..1a43daa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
@@ -469,14 +469,12 @@ public final class X {
         if (t == null || cls == null)
             return false;
 
-        if (t.getSuppressed() != null) {
-            for (Throwable th : t.getSuppressed()) {
-                if (cls.isAssignableFrom(th.getClass()))
-                    return true;
+        for (Throwable th : t.getSuppressed()) {
+            if (cls.isAssignableFrom(th.getClass()))
+                return true;
 
-                if (hasSuppressed(th, cls))
-                    return true;
-            }
+            if (hasSuppressed(th, cls))
+                return true;
         }
 
         return false;
@@ -749,6 +747,29 @@ public final class X {
     }
 
     /**
+     * Collects suppressed exceptions from throwable and all it causes.
+     *
+     * @param t Throwable.
+     * @return List of suppressed throwables.
+     */
+    public static List<Throwable> getSuppressedList(@Nullable Throwable t) {
+        List<Throwable> result = new ArrayList<>();
+
+        if (t == null)
+            return result;
+
+        do {
+            for (Throwable suppressed : t.getSuppressed()) {
+                result.add(suppressed);
+
+                result.addAll(getSuppressedList(suppressed));
+            }
+        } while ((t = t.getCause()) != null);
+
+        return result;
+    }
+
+    /**
      * A way to get the entire nested stack-trace of an throwable.
      *
      * The result of this method is highly dependent on the JDK version
@@ -889,4 +910,4 @@ public final class X {
             return dflt;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/98ef9259/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
index 15e9557..ba52c5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorExceptionWrapper.java
@@ -17,7 +17,8 @@
 
 package org.apache.ignite.internal.visor.util;
 
-import org.apache.ignite.internal.util.typedef.F;
+import java.util.List;
+import org.apache.ignite.internal.util.typedef.X;
 
 /**
  * Exception wrapper for safe for transferring to Visor.
@@ -56,12 +57,10 @@ public class VisorExceptionWrapper extends Throwable {
         if (cause.getCause() != null)
             initCause(new VisorExceptionWrapper(cause.getCause()));
 
-        Throwable[] suppressed = cause.getSuppressed();
+        List<Throwable> suppressed = X.getSuppressedList(cause);
 
-        if (!F.isEmpty(suppressed)) {
-            for (Throwable sup : suppressed)
-                addSuppressed(new VisorExceptionWrapper(sup));
-        }
+        for (Throwable sup : suppressed)
+            addSuppressed(new VisorExceptionWrapper(sup));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/98ef9259/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 9e7b592..df37dff 100755
--- 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
@@ -3476,7 +3476,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                     ctx.failNode(node.id(), "TcpCommunicationSpi failed to establish connection to node [" +
                         "rmtNode=" + node +
                         ", errs=" + errs +
-                        ", connectErrs=" + Arrays.toString(errs.getSuppressed()) + ']');
+                        ", connectErrs=" + X.getSuppressedList(errs) + ']');
                 }
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/98ef9259/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java
index 6e32249..55e54fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridSuppressedExceptionSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite;
 
 import java.io.IOException;
+import java.util.List;
 import junit.framework.TestCase;
 import org.apache.ignite.internal.util.typedef.X;
 
@@ -70,6 +71,26 @@ public class GridSuppressedExceptionSelfTest extends TestCase {
     /**
      * @throws Exception If failed.
      */
+    public void testXGetSuppressedList() throws Exception {
+        IgniteCheckedException me = prepareMultiException();
+
+        assertEquals(3, X.getSuppressedList(me).size());
+
+        RuntimeException e = new RuntimeException();
+        e.addSuppressed(me);
+
+        List<Throwable> suppresseds = X.getSuppressedList(e);
+
+        assertEquals(4, suppresseds.size());
+
+        assertEquals("Test message.", suppresseds.get(0).getMessage());
+        for (int i = 1; i <= 3; i++)
+            assertEquals("Demo exception.", suppresseds.get(1).getMessage());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testXCause() throws Exception {
         IgniteCheckedException me = prepareMultiException();
 
@@ -116,4 +137,4 @@ public class GridSuppressedExceptionSelfTest extends TestCase {
         else
             generateException(calls - 1, cause);
     }
-}
\ No newline at end of file
+}


[03/50] [abbrv] ignite git commit: IGNITE-7871 Implemented additional synchronization phase for correct partition counters update

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
new file mode 100644
index 0000000..bad1b61
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/LatchAckMessage.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+/**
+ * Message is used to send acks for {@link Latch} instances management.
+ */
+public class LatchAckMessage implements Message {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Latch id. */
+    private String latchId;
+
+    /** Latch topology version. */
+    private AffinityTopologyVersion topVer;
+
+    /** Flag indicates that ack is final. */
+    private boolean isFinal;
+
+    /**
+     * Constructor.
+     *
+     * @param latchId Latch id.
+     * @param topVer Latch topology version.
+     * @param isFinal Final acknowledgement flag.
+     */
+    public LatchAckMessage(String latchId, AffinityTopologyVersion topVer, boolean isFinal) {
+        this.latchId = latchId;
+        this.topVer = topVer;
+        this.isFinal = isFinal;
+    }
+
+    /**
+     * Empty constructor for marshalling purposes.
+     */
+    public LatchAckMessage() {
+    }
+
+    /**
+     * @return Latch id.
+     */
+    public String latchId() {
+        return latchId;
+    }
+
+    /**
+     * @return Latch topology version.
+     */
+    public AffinityTopologyVersion topVer() {
+        return topVer;
+    }
+
+    /**
+     * @return {@code} if ack is final.
+     */
+    public boolean isFinal() {
+        return isFinal;
+    }
+
+    /** {@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.writeBoolean("isFinal", isFinal))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeString("latchId", latchId))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeMessage("topVer", topVer))
+                    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:
+                isFinal = reader.readBoolean("isFinal");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                latchId = reader.readString("latchId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                topVer = reader.readMessage("topVer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+        }
+
+        return reader.afterMessageRead(LatchAckMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public short directType() {
+        return 135;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 7785605..33f84f0 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
@@ -3525,6 +3525,11 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
 
                 U.error(log, "Failed to prepare transaction: " + this, e);
             }
+            catch (Throwable t) {
+                fut.onDone(t);
+
+                throw t;
+            }
 
             if (err != null)
                 fut.rollbackOnError(err);
@@ -3544,6 +3549,11 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou
 
                         U.error(log, "Failed to prepare transaction: " + this, e);
                     }
+                    catch (Throwable t) {
+                        fut.onDone(t);
+
+                        throw t;
+                    }
 
                     if (err != null)
                         fut.rollbackOnError(err);

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
index 5cfd92d..68ec83d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheOffheapManager.java
@@ -189,7 +189,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             freeList.saveMetadata();
 
             long updCntr = store.updateCounter();
-            int size = store.fullSize();
+            long size = store.fullSize();
             long rmvId = globalRemoveId().get();
 
             PageMemoryEx pageMem = (PageMemoryEx)grp.dataRegion().pageMemory();
@@ -318,7 +318,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
                                 partMetaId,
                                 updCntr,
                                 rmvId,
-                                size,
+                                (int)size, // TODO: Partition size may be long
                                 cntrsPageId,
                                 state == null ? -1 : (byte)state.ordinal(),
                                 pageCnt
@@ -549,7 +549,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
             final int grpId,
             final int partId,
             final int currAllocatedPageCnt,
-            final int partSize
+            final long partSize
     ) {
         if (part != null) {
             boolean reserved = part.reserve();
@@ -1301,7 +1301,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         }
 
         /** {@inheritDoc} */
-        @Override public int fullSize() {
+        @Override public long fullSize() {
             try {
                 CacheDataStore delegate0 = init0(true);
 
@@ -1313,7 +1313,7 @@ public class GridCacheOffheapManager extends IgniteCacheOffheapManagerImpl imple
         }
 
         /** {@inheritDoc} */
-        @Override public int cacheSize(int cacheId) {
+        @Override public long cacheSize(int cacheId) {
             try {
                 CacheDataStore delegate0 = init0(true);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 9bfaaf3..945ef48 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -490,7 +490,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter implement
     @Override public AffinityTopologyVersion topologyVersion() {
         AffinityTopologyVersion res = topVer;
 
-        if (res.equals(AffinityTopologyVersion.NONE)) {
+        if (res == null || res.equals(AffinityTopologyVersion.NONE)) {
             if (system()) {
                 AffinityTopologyVersion topVer = cctx.tm().lockedTopologyVersion(Thread.currentThread().getId(), this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 fbdeca1..9fb8777 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
@@ -545,10 +545,10 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @param topVer Topology version.
      * @return Future that will be completed when all ongoing transactions are finished.
      */
-    public IgniteInternalFuture<Boolean> finishTxs(AffinityTopologyVersion topVer) {
+    public IgniteInternalFuture<Boolean> finishLocalTxs(AffinityTopologyVersion topVer) {
         GridCompoundFuture<IgniteInternalTx, Boolean> res =
             new CacheObjectsReleaseFuture<>(
-                "Tx",
+                "LocalTx",
                 topVer,
                 new IgniteReducer<IgniteInternalTx, Boolean>() {
                     @Override public boolean collect(IgniteInternalTx e) {
@@ -561,8 +561,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
                 });
 
         for (IgniteInternalTx tx : txs()) {
-            if (needWaitTransaction(tx, topVer))
+            if (needWaitTransaction(tx, topVer)) {
                 res.add(tx.finishFuture());
+            }
         }
 
         res.markInitialized();
@@ -571,6 +572,29 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Creates a future that will wait for finishing all tx updates on backups after all local transactions are finished.
+     *
+     * NOTE:
+     * As we send finish request to backup nodes after transaction successfully completed on primary node
+     * it's important to ensure that all updates from primary to backup are finished or at least remote transaction has created on backup node.
+     *
+     * @param finishLocalTxsFuture Local transactions finish future.
+     * @param topVer Topology version.
+     * @return Future that will be completed when all ongoing transactions are finished.
+     */
+    public IgniteInternalFuture<?> finishAllTxs(IgniteInternalFuture<?> finishLocalTxsFuture, AffinityTopologyVersion topVer) {
+        final GridCompoundFuture finishAllTxsFuture = new CacheObjectsReleaseFuture("AllTx", topVer);
+
+        // After finishing all local updates, wait for finishing all tx updates on backups.
+        finishLocalTxsFuture.listen(future -> {
+            finishAllTxsFuture.add(cctx.mvcc().finishRemoteTxs(topVer));
+            finishAllTxsFuture.markInitialized();
+        });
+
+        return finishAllTxsFuture;
+    }
+
+    /**
      * @param tx Transaction.
      * @param topVer Exchange version.
      * @return {@code True} if need wait transaction for exchange.
@@ -1834,12 +1858,12 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
      * @return Finish future for related remote transactions.
      */
     @SuppressWarnings("unchecked")
-    public IgniteInternalFuture<?> remoteTxFinishFuture(GridCacheVersion nearVer) {
-        GridCompoundFuture<Void, Void> fut = new GridCompoundFuture<>();
+    public IgniteInternalFuture<IgniteInternalTx> remoteTxFinishFuture(GridCacheVersion nearVer) {
+        GridCompoundFuture<IgniteInternalTx, IgniteInternalTx> fut = new GridCompoundFuture<>();
 
         for (final IgniteInternalTx tx : txs()) {
             if (!tx.local() && nearVer.equals(tx.nearXidVersion()))
-                fut.add((IgniteInternalFuture) tx.finishFuture());
+                fut.add(tx.finishFuture());
         }
 
         fut.markInitialized();

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
index 7263656..702b188 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheDhtLocalPartitionAfterRemoveSelfTest.java
@@ -76,7 +76,7 @@ public class CacheDhtLocalPartitionAfterRemoveSelfTest extends GridCommonAbstrac
             cache = grid(g).cache(DEFAULT_CACHE_NAME);
 
             for (GridDhtLocalPartition p : dht(cache).topology().localPartitions()) {
-                int size = p.dataStore().fullSize();
+                long size = p.dataStore().fullSize();
 
                 assertTrue("Unexpected size: " + size, size <= 32);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java
index 468bbc8..6c570d7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheGroupsTest.java
@@ -86,6 +86,7 @@ import org.apache.ignite.internal.util.lang.GridIterator;
 import org.apache.ignite.internal.util.lang.GridPlainCallable;
 import org.apache.ignite.internal.util.lang.gridfunc.ContainsPredicate;
 import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java
new file mode 100644
index 0000000..52cd033
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteExchangeLatchManagerCoordinatorFailTest.java
@@ -0,0 +1,244 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.datastructures;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.collect.Lists;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager;
+import org.apache.ignite.internal.util.future.GridCompoundFuture;
+import org.apache.ignite.lang.IgniteBiClosure;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+/**
+ * Tests for {@link ExchangeLatchManager} functionality when latch coordinator is failed.
+ */
+public class IgniteExchangeLatchManagerCoordinatorFailTest extends GridCommonAbstractTest {
+    /** */
+    private static final String LATCH_NAME = "test";
+
+    /** 5 nodes. */
+    private final AffinityTopologyVersion latchTopVer = new AffinityTopologyVersion(5, 0);
+
+    /** Wait before latch creation. */
+    private final IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean> beforeCreate = (mgr, syncLatch) -> {
+        try {
+            syncLatch.countDown();
+            syncLatch.await();
+
+            Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer);
+
+            distributedLatch.countDown();
+
+            distributedLatch.await();
+        } catch (Exception e) {
+            log.error("Unexpected exception", e);
+
+            return false;
+        }
+
+        return true;
+    };
+
+    /** Wait before latch count down. */
+    private final IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean> beforeCountDown = (mgr, syncLatch) -> {
+        try {
+            Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer);
+
+            syncLatch.countDown();
+            syncLatch.await();
+
+            distributedLatch.countDown();
+
+            distributedLatch.await();
+        } catch (Exception e) {
+            log.error("Unexpected exception ", e);
+
+            return false;
+        }
+
+        return true;
+    };
+
+    /** Wait after all operations are successful. */
+    private final IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean> all = (mgr, syncLatch) -> {
+        try {
+            Latch distributedLatch = mgr.getOrCreate(LATCH_NAME, latchTopVer);
+
+            distributedLatch.countDown();
+
+            syncLatch.countDown();
+
+            distributedLatch.await();
+
+            syncLatch.await();
+        } catch (Exception e) {
+            log.error("Unexpected exception ", e);
+
+            return false;
+        }
+
+        return true;
+    };
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Test scenarios description:
+     *
+     * We have existing coordinator and 4 other nodes.
+     * Each node do following operations:
+     * 1) Create latch
+     * 2) Countdown latch
+     * 3) Await latch
+     *
+     * While nodes do the operations we shutdown coordinator and next oldest node become new coordinator.
+     * We should check that new coordinator properly restored latch and all nodes finished latch completion successfully after that.
+     *
+     * Each node before coordinator shutdown can be in 3 different states:
+     *
+     * State {@link #beforeCreate} - Node didn't create latch yet.
+     * State {@link #beforeCountDown} - Node created latch but didn't count down it yet.
+     * State {@link #all} - Node created latch and count downed it.
+     *
+     * We should check important cases when future coordinator is in one of these states, and other 3 nodes have 3 different states.
+     */
+
+    /**
+     * Scenario 1:
+     *
+     * Node 1 state -> {@link #beforeCreate}
+     * Node 2 state -> {@link #beforeCountDown}
+     * Node 3 state -> {@link #all}
+     * Node 4 state -> {@link #beforeCreate}
+     */
+    public void testCoordinatorFail1() throws Exception {
+        List<IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean>> nodeStates = Lists.newArrayList(
+            beforeCreate,
+            beforeCountDown,
+            all,
+            beforeCreate
+        );
+
+        doTestCoordinatorFail(nodeStates);
+    }
+
+    /**
+     * Scenario 2:
+     *
+     * Node 1 state -> {@link #beforeCountDown}
+     * Node 2 state -> {@link #beforeCountDown}
+     * Node 3 state -> {@link #all}
+     * Node 4 state -> {@link #beforeCreate}
+     */
+    public void testCoordinatorFail2() throws Exception {
+        List<IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean>> nodeStates = Lists.newArrayList(
+            beforeCountDown,
+            beforeCountDown,
+            all,
+            beforeCreate
+        );
+
+        doTestCoordinatorFail(nodeStates);
+    }
+
+    /**
+     * Scenario 3:
+     *
+     * Node 1 state -> {@link #all}
+     * Node 2 state -> {@link #beforeCountDown}
+     * Node 3 state -> {@link #all}
+     * Node 4 state -> {@link #beforeCreate}
+     */
+    public void testCoordinatorFail3() throws Exception {
+        List<IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean>> nodeStates = Lists.newArrayList(
+            all,
+            beforeCountDown,
+            all,
+            beforeCreate
+        );
+
+        doTestCoordinatorFail(nodeStates);
+    }
+
+    /**
+     * Test latch coordinator fail with specified scenarios.
+     *
+     * @param nodeScenarios Node scenarios.
+     * @throws Exception If failed.
+     */
+    private void doTestCoordinatorFail(List<IgniteBiClosure<ExchangeLatchManager, CountDownLatch, Boolean>> nodeScenarios) throws Exception {
+        IgniteEx crd = (IgniteEx) startGridsMultiThreaded(5);
+        crd.cluster().active(true);
+
+        // Latch to synchronize node states.
+        CountDownLatch syncLatch = new CountDownLatch(5);
+
+        GridCompoundFuture finishAllLatches = new GridCompoundFuture();
+
+        AtomicBoolean hasErrors = new AtomicBoolean();
+
+        for (int node = 1; node < 5; node++) {
+            IgniteEx grid = grid(node);
+            ExchangeLatchManager latchMgr = grid.context().cache().context().exchange().latch();
+            final int stateIdx = node - 1;
+
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(() -> {
+                boolean success = nodeScenarios.get(stateIdx).apply(latchMgr, syncLatch);
+                if (!success)
+                    hasErrors.set(true);
+            }, 1, "latch-runner-" + node);
+
+            finishAllLatches.add(fut);
+        }
+
+        finishAllLatches.markInitialized();
+
+        // Wait while all nodes reaches their states.
+        while (syncLatch.getCount() != 1) {
+            Thread.sleep(10);
+
+            if (hasErrors.get())
+                throw new Exception("All nodes should complete latches without errors");
+        }
+
+        crd.close();
+
+        // Resume progress for all nodes.
+        syncLatch.countDown();
+
+        // Wait for distributed latch completion.
+        finishAllLatches.get(5000);
+
+        Assert.assertFalse("All nodes should complete latches without errors", hasErrors.get());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java
new file mode 100644
index 0000000..63d772a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidationTest.java
@@ -0,0 +1,316 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionConcurrency;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+/**
+ *
+ */
+public class GridCachePartitionsStateValidationTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** */
+    private boolean clientMode;
+
+    /** {@inheritDoc */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME)
+                .setBackups(1)
+                .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        cfg.setCommunicationSpi(new SingleMessageInterceptorCommunicationSpi(2));
+
+        if (clientMode)
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        clientMode = false;
+    }
+
+    /**
+     * Test that partitions state validation works correctly.
+     *
+     * @throws Exception If failed.
+     */
+    public void testValidationIfPartitionCountersAreInconsistent() throws Exception {
+        IgniteEx ignite = (IgniteEx) startGrids(2);
+        ignite.cluster().active(true);
+
+        awaitPartitionMapExchange();
+
+        // Modify update counter for some partition.
+        for (GridDhtLocalPartition partition : ignite.cachex(CACHE_NAME).context().topology().localPartitions()) {
+            partition.updateCounter(100500L);
+            break;
+        }
+
+        // Trigger exchange.
+        startGrid(2);
+
+        awaitPartitionMapExchange();
+
+        // Nothing should happen (just log error message) and we're still able to put data to corrupted cache.
+        ignite.cache(CACHE_NAME).put(0, 0);
+
+        stopAllGrids();
+    }
+
+    /**
+     * Test that all nodes send correct {@link GridDhtPartitionsSingleMessage} with consistent update counters.
+     *
+     * @throws Exception If failed.
+     */
+    public void testPartitionCountersConsistencyOnExchange() throws Exception {
+        IgniteEx ignite = (IgniteEx) startGrids(4);
+        ignite.cluster().active(true);
+
+        awaitPartitionMapExchange();
+
+        final String atomicCacheName = "atomic-cache";
+        final String txCacheName = "tx-cache";
+
+        clientMode = true;
+
+        Ignite client = startGrid(4);
+
+        clientMode = false;
+
+        IgniteCache atomicCache = client.getOrCreateCache(new CacheConfiguration<>(atomicCacheName)
+            .setAtomicityMode(CacheAtomicityMode.ATOMIC)
+            .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        IgniteCache txCache = client.getOrCreateCache(new CacheConfiguration<>(txCacheName)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        for (int it = 0; it < 10; it++) {
+            SingleMessageInterceptorCommunicationSpi spi = (SingleMessageInterceptorCommunicationSpi) ignite.configuration().getCommunicationSpi();
+            spi.clear();
+
+            // Stop load future.
+            final AtomicBoolean stop = new AtomicBoolean();
+
+            // Run atomic load.
+            IgniteInternalFuture atomicLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> {
+                int k = 0;
+
+                while (!stop.get()) {
+                    k++;
+                    try {
+                        atomicCache.put(k, k);
+                    } catch (Exception ignored) {}
+                }
+            }, 1, "atomic-load");
+
+            // Run tx load.
+            IgniteInternalFuture txLoadFuture = GridTestUtils.runMultiThreadedAsync(() -> {
+                final int txOps = 5;
+
+                while (!stop.get()) {
+                    List<Integer> randomKeys = Stream.generate(() -> ThreadLocalRandom.current().nextInt(5))
+                        .limit(txOps)
+                        .sorted()
+                        .collect(Collectors.toList());
+
+                    try (Transaction tx = ignite.transactions().txStart(TransactionConcurrency.OPTIMISTIC, TransactionIsolation.READ_COMMITTED)) {
+                        for (Integer key : randomKeys)
+                            txCache.put(key, key);
+
+                        tx.commit();
+                    }
+                    catch (Exception ignored) { }
+                }
+            }, 4, "tx-load");
+
+            // Wait for some data.
+            Thread.sleep(1000);
+
+            // Prevent sending full message.
+            spi.blockFullMessage();
+
+            // Trigger exchange.
+            IgniteInternalFuture nodeStopFuture = GridTestUtils.runAsync(() -> stopGrid(3));
+
+            try {
+                spi.waitUntilAllSingleMessagesAreSent();
+
+                List<GridDhtPartitionsSingleMessage> interceptedMessages = spi.getMessages();
+
+                // Associate each message with existing node UUID.
+                Map<UUID, GridDhtPartitionsSingleMessage> messagesMap = new HashMap<>();
+                for (int i = 0; i < interceptedMessages.size(); i++)
+                    messagesMap.put(grid(i + 1).context().localNodeId(), interceptedMessages.get(i));
+
+                GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(ignite.context().cache().context());
+
+                // Validate partition update counters. If counters are not consistent, exception will be thrown.
+                validator.validatePartitionsUpdateCounters(ignite.cachex(atomicCacheName).context().topology(), messagesMap, Collections.emptySet());
+                validator.validatePartitionsUpdateCounters(ignite.cachex(txCacheName).context().topology(), messagesMap, Collections.emptySet());
+
+            } finally {
+                // Stop load and resume exchange.
+                spi.unblockFullMessage();
+
+                stop.set(true);
+
+                atomicLoadFuture.get();
+                txLoadFuture.get();
+                nodeStopFuture.get();
+            }
+
+            // Return grid to initial state.
+            startGrid(3);
+
+            awaitPartitionMapExchange();
+        }
+    }
+
+    /**
+     * SPI which intercepts single messages during exchange.
+     */
+    private static class SingleMessageInterceptorCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        private static final List<GridDhtPartitionsSingleMessage> messages = new CopyOnWriteArrayList<>();
+
+        /** Future completes when {@link #singleMessagesThreshold} messages are sent to coordinator. */
+        private static final GridFutureAdapter allSingleMessagesSent = new GridFutureAdapter();
+
+        /** A number of single messages we're waiting for send. */
+        private final int singleMessagesThreshold;
+
+        /** Latch which blocks full message sending. */
+        private volatile CountDownLatch blockFullMsgLatch;
+
+        /**
+         * Constructor.
+         */
+        private SingleMessageInterceptorCommunicationSpi(int singleMessagesThreshold) {
+            this.singleMessagesThreshold = singleMessagesThreshold;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
+            if (((GridIoMessage) msg).message() instanceof GridDhtPartitionsSingleMessage) {
+                GridDhtPartitionsSingleMessage singleMsg = (GridDhtPartitionsSingleMessage) ((GridIoMessage) msg).message();
+
+                // We're interesting for only exchange messages and when node is stopped.
+                if (singleMsg.exchangeId() != null && singleMsg.exchangeId().isLeft() && !singleMsg.client()) {
+                    messages.add(singleMsg);
+
+                    if (messages.size() == singleMessagesThreshold)
+                        allSingleMessagesSent.onDone();
+                }
+            }
+
+            try {
+                if (((GridIoMessage) msg).message() instanceof GridDhtPartitionsFullMessage) {
+                    if (blockFullMsgLatch != null)
+                        blockFullMsgLatch.await();
+                }
+            }
+            catch (Exception ignored) { }
+
+            super.sendMessage(node, msg, ackC);
+        }
+
+        /** */
+        public void clear() {
+            messages.clear();
+            allSingleMessagesSent.reset();
+        }
+
+        /** */
+        public List<GridDhtPartitionsSingleMessage> getMessages() {
+            return Collections.unmodifiableList(messages);
+        }
+
+        /** */
+        public void blockFullMessage() {
+            blockFullMsgLatch = new CountDownLatch(1);
+        }
+
+        /** */
+        public void unblockFullMessage() {
+            blockFullMsgLatch.countDown();
+        }
+
+        /** */
+        public void waitUntilAllSingleMessagesAreSent() throws IgniteCheckedException {
+            allSingleMessagesSent.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java
new file mode 100644
index 0000000..9ed8d54
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsStateValidatorSelfTest.java
@@ -0,0 +1,158 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+import org.mockito.Matchers;
+import org.mockito.Mockito;
+
+/**
+ * Test correct behaviour of {@link GridDhtPartitionsStateValidator} class.
+ */
+public class GridCachePartitionsStateValidatorSelfTest extends GridCommonAbstractTest {
+    /** Mocks and stubs. */
+    private final UUID localNodeId = UUID.randomUUID();
+    /** */
+    private GridCacheSharedContext cctxMock;
+    /** */
+    private GridDhtPartitionTopology topologyMock;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        // Prepare mocks.
+        cctxMock = Mockito.mock(GridCacheSharedContext.class);
+        Mockito.when(cctxMock.localNodeId()).thenReturn(localNodeId);
+
+        topologyMock = Mockito.mock(GridDhtPartitionTopology.class);
+        Mockito.when(topologyMock.partitionState(Matchers.any(), Matchers.anyInt())).thenReturn(GridDhtPartitionState.OWNING);
+        Mockito.when(topologyMock.groupId()).thenReturn(0);
+        Mockito.when(topologyMock.partitions()).thenReturn(3);
+
+        List<GridDhtLocalPartition> localPartitions = Lists.newArrayList(
+                partitionMock(0, 1, 1),
+                partitionMock(1, 2, 2),
+                partitionMock(2, 3, 3)
+        );
+        Mockito.when(topologyMock.localPartitions()).thenReturn(localPartitions);
+        Mockito.when(topologyMock.currentLocalPartitions()).thenReturn(localPartitions);
+    }
+
+    /**
+     * @return Partition mock with specified {@code id}, {@code updateCounter} and {@code size}.
+     */
+    private GridDhtLocalPartition partitionMock(int id, long updateCounter, long size) {
+        GridDhtLocalPartition partitionMock = Mockito.mock(GridDhtLocalPartition.class);
+        Mockito.when(partitionMock.id()).thenReturn(id);
+        Mockito.when(partitionMock.updateCounter()).thenReturn(updateCounter);
+        Mockito.when(partitionMock.fullSize()).thenReturn(size);
+        return partitionMock;
+    }
+
+    /**
+     * @return Message containing specified {@code countersMap}.
+     */
+    private GridDhtPartitionsSingleMessage fromUpdateCounters(Map<Integer, T2<Long, Long>> countersMap) {
+        GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage();
+        msg.addPartitionUpdateCounters(0, countersMap);
+        return msg;
+    }
+
+    /**
+     * @return Message containing specified {@code sizesMap}.
+     */
+    private GridDhtPartitionsSingleMessage fromCacheSizes(Map<Integer, Long> sizesMap) {
+        GridDhtPartitionsSingleMessage msg = new GridDhtPartitionsSingleMessage();
+        msg.addPartitionSizes(0, sizesMap);
+        return msg;
+    }
+
+    /**
+     * Test partition update counters validation.
+     */
+    public void testPartitionCountersValidation() {
+        UUID remoteNode = UUID.randomUUID();
+        UUID ignoreNode = UUID.randomUUID();
+
+        // For partitions 0 and 2 (zero counter) we have inconsistent update counters.
+        Map<Integer, T2<Long, Long>> updateCountersMap = new HashMap<>();
+        updateCountersMap.put(0, new T2<>(2L, 2L));
+        updateCountersMap.put(1, new T2<>(2L, 2L));
+
+        // Form single messages map.
+        Map<UUID, GridDhtPartitionsSingleMessage> messages = new HashMap<>();
+        messages.put(remoteNode, fromUpdateCounters(updateCountersMap));
+        messages.put(ignoreNode, fromUpdateCounters(updateCountersMap));
+
+        GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock);
+
+        // (partId, (nodeId, updateCounter))
+        Map<Integer, Map<UUID, Long>> result = validator.validatePartitionsUpdateCounters(topologyMock, messages, Sets.newHashSet(ignoreNode));
+
+        // Check that validation result contains all necessary information.
+        Assert.assertEquals(2, result.size());
+        Assert.assertTrue(result.containsKey(0));
+        Assert.assertTrue(result.containsKey(2));
+        Assert.assertTrue(result.get(0).get(localNodeId) == 1L);
+        Assert.assertTrue(result.get(0).get(remoteNode) == 2L);
+        Assert.assertTrue(result.get(2).get(localNodeId) == 3L);
+        Assert.assertTrue(result.get(2).get(remoteNode) == 0L);
+    }
+
+    /**
+     * Test partition cache sizes validation.
+     */
+    public void testPartitionCacheSizesValidation() {
+        UUID remoteNode = UUID.randomUUID();
+        UUID ignoreNode = UUID.randomUUID();
+
+        // For partitions 0 and 2 we have inconsistent cache sizes.
+        Map<Integer, Long> cacheSizesMap = new HashMap<>();
+        cacheSizesMap.put(0, 2L);
+        cacheSizesMap.put(1, 2L);
+        cacheSizesMap.put(2, 2L);
+
+        // Form single messages map.
+        Map<UUID, GridDhtPartitionsSingleMessage> messages = new HashMap<>();
+        messages.put(remoteNode, fromCacheSizes(cacheSizesMap));
+        messages.put(ignoreNode, fromCacheSizes(cacheSizesMap));
+
+        GridDhtPartitionsStateValidator validator = new GridDhtPartitionsStateValidator(cctxMock);
+
+        // (partId, (nodeId, cacheSize))
+        Map<Integer, Map<UUID, Long>> result = validator.validatePartitionsSizes(topologyMock, messages, Sets.newHashSet(ignoreNode));
+
+        // Check that validation result contains all necessary information.
+        Assert.assertEquals(2, result.size());
+        Assert.assertTrue(result.containsKey(0));
+        Assert.assertTrue(result.containsKey(2));
+        Assert.assertTrue(result.get(0).get(localNodeId) == 1L);
+        Assert.assertTrue(result.get(0).get(remoteNode) == 2L);
+        Assert.assertTrue(result.get(2).get(localNodeId) == 3L);
+        Assert.assertTrue(result.get(2).get(remoteNode) == 2L);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java
new file mode 100644
index 0000000..03ea0f7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticOnPartitionExchangeTest.java
@@ -0,0 +1,322 @@
+/*
+ * 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.transactions;
+
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.TreeSet;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheAffinityManager;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxPrepareRequest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishRequest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.typedef.T1;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.transactions.Transaction;
+import org.apache.ignite.transactions.TransactionIsolation;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
+import static org.apache.ignite.testframework.GridTestUtils.waitForCondition;
+import static org.apache.ignite.transactions.TransactionConcurrency.OPTIMISTIC;
+import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
+import static org.apache.ignite.transactions.TransactionIsolation.READ_COMMITTED;
+import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ;
+import static org.apache.ignite.transactions.TransactionIsolation.SERIALIZABLE;
+
+/**
+ *
+ */
+public class TxOptimisticOnPartitionExchangeTest extends GridCommonAbstractTest {
+    /** Nodes count. */
+    private static final int NODES_CNT = 3;
+
+    /** Tx size. */
+    private static final int TX_SIZE = 20 * NODES_CNT;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** Logger started. */
+    private static volatile boolean msgInterception;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(NODES_CNT);
+
+        awaitPartitionMapExchange();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi(log()));
+
+        cfg.setCacheConfiguration(defaultCacheConfiguration()
+            .setName(CACHE_NAME)
+            .setAtomicityMode(TRANSACTIONAL)
+            .setWriteSynchronizationMode(FULL_SYNC)
+            .setCacheMode(PARTITIONED)
+            .setBackups(1));
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConsistencyOnPartitionExchange() throws Exception {
+        doTest(SERIALIZABLE, true);
+        doTest(READ_COMMITTED, true);
+        doTest(SERIALIZABLE, false);
+        doTest(READ_COMMITTED, false);
+    }
+
+    /**
+     * @param isolation {@link TransactionIsolation}.
+     * @param txInitiatorPrimary False If the transaction does not use the keys of the node that initiated it.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void doTest(final TransactionIsolation isolation, boolean txInitiatorPrimary) throws Exception {
+        final CountDownLatch txStarted = new CountDownLatch(1);
+
+        final IgniteCache cache = ignite(0).cache(CACHE_NAME);
+
+        final Map<Integer, Integer> txValues = new TreeMap<>();
+
+        ClusterNode node = ignite(0).cluster().node();
+
+        GridCacheAffinityManager affinity = ((IgniteCacheProxy)cache).context().affinity();
+
+        for (int i = 0; txValues.size() < TX_SIZE; i++) {
+            if (!txInitiatorPrimary && node.equals(affinity.primaryByKey(i, NONE)))
+                continue;
+
+            txValues.put(i, i);
+        }
+
+        TestCommunicationSpi.init();
+
+        msgInterception = true;
+
+        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() {
+                try (Transaction tx = ignite(0).transactions().txStart(OPTIMISTIC, isolation)) {
+                    info(">>> TX started.");
+
+                    txStarted.countDown();
+
+                    cache.putAll(txValues);
+
+                    tx.commit();
+
+                    info(">>> TX committed.");
+                }
+
+                return null;
+            }
+        });
+
+        txStarted.await();
+
+        try {
+            info(">>> Grid starting.");
+
+            IgniteEx ignite = startGrid(NODES_CNT);
+
+            info(">>> Grid started.");
+
+            fut.get();
+
+            awaitPartitionMapExchange();
+
+            msgInterception = false;
+
+            IgniteCache<Object, Object> cacheStartedNode = ignite.cache(CACHE_NAME);
+
+            try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                Set<Object> keys = cacheStartedNode.getAll(txValues.keySet()).keySet();
+
+                assertEquals(txValues.keySet(), new TreeSet<>(keys));
+
+                tx.commit();
+            }
+        }
+        finally {
+            msgInterception = false;
+
+            stopGrid(NODES_CNT);
+        }
+    }
+
+    /**
+     *
+     */
+    @SuppressWarnings("ConstantConditions")
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** Partition single message sent from added node. */
+        private static volatile CountDownLatch partSingleMsgSentFromAddedNode;
+
+        /** Partition supply message sent count. */
+        private static final AtomicInteger partSupplyMsgSentCnt = new AtomicInteger();
+
+        /** Logger. */
+        private IgniteLogger log;
+
+        /**
+         * @param log Logger.
+         */
+        public TestCommunicationSpi(IgniteLogger log) {
+            this.log = log;
+        }
+
+        /**
+         *
+         */
+        public static void init() {
+            partSingleMsgSentFromAddedNode = new CountDownLatch(1);
+
+            partSupplyMsgSentCnt.set(0);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(
+            final ClusterNode node,
+            final Message msg,
+            final IgniteInClosure<IgniteException> ackC
+        ) throws IgniteSpiException {
+            if (msgInterception) {
+                if (msg instanceof GridIoMessage) {
+                    final Message msg0 = ((GridIoMessage)msg).message();
+
+                    String locNodeId = ((IgniteEx)ignite).context().localNodeId().toString();
+
+                    int nodeIdx = Integer.parseInt(locNodeId.substring(locNodeId.length() - 3));
+
+                    if (nodeIdx == 0) {
+                        if (msg0 instanceof GridNearTxPrepareRequest || msg0 instanceof GridDhtTxPrepareRequest) {
+                            GridTestUtils.runAsync(new Callable<Void>() {
+                                @Override public Void call() throws Exception {
+                                    partSingleMsgSentFromAddedNode.await();
+
+                                    sendMessage(node, msg, ackC, true);
+
+                                    return null;
+                                }
+                            });
+
+                            return;
+
+                        }
+                        else if (msg0 instanceof GridNearTxFinishRequest || msg0 instanceof GridDhtTxFinishRequest) {
+                            GridTestUtils.runAsync(new Callable<Void>() {
+                                @Override public Void call() throws Exception {
+                                    final T1<Integer> i = new T1<>(0);
+
+                                    while (waitForCondition(new GridAbsPredicate() {
+                                        @Override public boolean apply() {
+                                            return partSupplyMsgSentCnt.get() > i.get();
+                                        }
+                                    }, i.get() == 0 ? 5_000 : 500))
+                                        i.set(partSupplyMsgSentCnt.get());
+
+                                    sendMessage(node, msg, ackC, true);
+
+                                    return null;
+                                }
+                            });
+
+                            return;
+                        }
+                    }
+                    else if (nodeIdx == NODES_CNT && msg0 instanceof GridDhtPartitionsSingleMessage)
+                        partSingleMsgSentFromAddedNode.countDown();
+
+                    if (msg0 instanceof GridDhtPartitionSupplyMessage)
+                        partSupplyMsgSentCnt.incrementAndGet();
+                }
+            }
+
+            sendMessage(node, msg, ackC, msgInterception);
+        }
+
+        /**
+         * @param node Node.
+         * @param msg Message.
+         * @param ackC Ack closure.
+         * @param logMsg Log Messages.
+         */
+        private void sendMessage(
+            final ClusterNode node,
+            final Message msg,
+            final IgniteInClosure<IgniteException> ackC,
+            boolean logMsg
+        ) throws IgniteSpiException {
+            if (logMsg) {
+                String id = node.id().toString();
+                String locNodeId = ((IgniteEx)ignite).context().localNodeId().toString();
+
+                Message msg0 = ((GridIoMessage)msg).message();
+
+                log.info(
+                    String.format(">>> Output msg[type=%s, fromNode= %s, toNode=%s]",
+                        msg0.getClass().getSimpleName(),
+                        locNodeId.charAt(locNodeId.length() - 1),
+                        id.charAt(id.length() - 1)
+                    )
+                );
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 bb397f7..0612615 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
@@ -133,6 +133,8 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteCrossCacheT
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheAtomicNearCacheSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheGlobalLoadTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionsStateValidationTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionsStateValidatorSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheGetStoreErrorSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheNearTxExceptionSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.GridCacheReplicatedTxExceptionSelfTest;
@@ -292,6 +294,8 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheSystemTransactionsSelfTest.class);
         suite.addTestSuite(CacheDeferredDeleteSanitySelfTest.class);
         suite.addTestSuite(CacheDeferredDeleteQueueTest.class);
+        suite.addTestSuite(GridCachePartitionsStateValidatorSelfTest.class);
+        suite.addTestSuite(GridCachePartitionsStateValidationTest.class);
 
         suite.addTest(IgniteCacheTcpClientDiscoveryTestSuite.suite());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
index f8add30..415479d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite6.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessim
 import org.apache.ignite.internal.processors.cache.WalModeChangeAdvancedSelfTest;
 import org.apache.ignite.internal.processors.cache.WalModeChangeCoordinatorNotAffinityNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.WalModeChangeSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.IgniteExchangeLatchManagerCoordinatorFailTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheExchangeMergeTest;
 import org.apache.ignite.internal.processors.cache.distributed.CachePartitionStateTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCachePartitionEvictionDuringReadThroughSelfTest;
@@ -40,6 +41,7 @@ import org.apache.ignite.internal.processors.cache.distributed.IgniteOptimisticT
 import org.apache.ignite.internal.processors.cache.distributed.IgniteOptimisticTxSuspendResumeTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgnitePessimisticTxSuspendResumeTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheAssignmentNodeRestartsTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticOnPartitionExchangeTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNearCacheTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutNoDeadlockDetectionTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxRollbackOnTimeoutTest;
@@ -93,6 +95,10 @@ public class IgniteCacheTestSuite6 extends TestSuite {
         suite.addTestSuite(PartitionedTransactionalOptimisticCacheGetsDistributionTest.class);
         suite.addTestSuite(PartitionedTransactionalPessimisticCacheGetsDistributionTest.class);
 
+        suite.addTestSuite(TxOptimisticOnPartitionExchangeTest.class);
+
+        suite.addTestSuite(IgniteExchangeLatchManagerCoordinatorFailTest.class);
+
         return suite;
     }
 }


[48/50] [abbrv] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-7708

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/ad805704/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
----------------------------------------------------------------------


[22/50] [abbrv] ignite git commit: IGNITE-7824: Wrong warning message fixed. - Fixes #3788.

Posted by ag...@apache.org.
IGNITE-7824: Wrong warning message fixed. - Fixes #3788.

Signed-off-by: Nikolay Izhikov <ni...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 706e7f81b44ca9b3a01ffd673e5db61578123b64
Parents: df6356d
Author: NSAmelchev <ns...@gmail.com>
Authored: Thu Apr 12 13:21:15 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Thu Apr 12 13:21:15 2018 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/706e7f81/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 1cb07b9..795601e 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
@@ -1439,7 +1439,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             long safeToUse = ram - Math.max(4L << 30, (long)(ram * 0.2));
 
             if (total > safeToUse) {
-                U.quietAndWarn(log, "Nodes started on local machine require more than 20% of physical RAM what can " +
+                U.quietAndWarn(log, "Nodes started on local machine require more than 80% of physical RAM what can " +
                     "lead to significant slowdown due to swapping (please decrease JVM heap size, data region " +
                     "size or checkpoint buffer size) [required=" + (total >> 20) + "MB, available=" +
                     (ram >> 20) + "MB]");


[44/50] [abbrv] ignite git commit: IGNITE-2766 - Opportunistically reopen cache after client reconnect - Fixes #3417

Posted by ag...@apache.org.
IGNITE-2766 - Opportunistically reopen cache after client reconnect - Fixes #3417

Signed-off-by: Valentin Kulichenko <va...@gmail.com>


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

Branch: refs/heads/ignite-7708
Commit: 0991437a3f4d38e68483a8bcadd3daf614b7b2dc
Parents: c6ab036
Author: Ilya Kasnacheev <il...@gmail.com>
Authored: Fri Apr 13 14:48:10 2018 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Apr 13 14:48:10 2018 -0700

----------------------------------------------------------------------
 .../cache/GatewayProtectedCacheProxy.java       | 676 ++++++++-----------
 .../processors/cache/GridCacheGateway.java      |   7 +
 .../processors/cache/IgniteCacheProxyImpl.java  |  31 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   6 +-
 .../IgniteCacheQueryCacheDestroySelfTest.java   |   4 +
 .../ClientReconnectAfterClusterRestartTest.java |  33 +-
 6 files changed, 316 insertions(+), 441 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
index 27fc395..2e8120b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GatewayProtectedCacheProxy.java
@@ -36,6 +36,7 @@ import javax.cache.integration.CompletionListener;
 import javax.cache.processor.EntryProcessor;
 import javax.cache.processor.EntryProcessorResult;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheEntry;
 import org.apache.ignite.cache.CacheEntryProcessor;
 import org.apache.ignite.cache.CacheMetrics;
@@ -48,6 +49,9 @@ import org.apache.ignite.cache.query.QueryMetrics;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.internal.AsyncSupportAdapter;
+import org.apache.ignite.internal.GridKernalState;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.lang.IgniteClosure;
@@ -138,15 +142,13 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public GatewayProtectedCacheProxy<K, V> withExpiryPolicy(ExpiryPolicy plc) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return new GatewayProtectedCacheProxy<>(delegate, opCtx.withExpiryPolicy(plc), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -157,9 +159,7 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public GatewayProtectedCacheProxy<K, V> skipStore() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             boolean skip = opCtx.skipStore();
@@ -170,15 +170,13 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
             return new GatewayProtectedCacheProxy<>(delegate, opCtx.setSkipStore(true), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public GatewayProtectedCacheProxy<K, V> withNoRetries() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             boolean noRetries = opCtx.noRetries();
@@ -189,15 +187,13 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
             return new GatewayProtectedCacheProxy<>(delegate, opCtx.setNoRetries(true), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public GatewayProtectedCacheProxy<K, V> withPartitionRecover() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             boolean recovery = opCtx.recovery();
@@ -208,7 +204,7 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
             return new GatewayProtectedCacheProxy<>(delegate, opCtx.setRecovery(true), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -219,23 +215,19 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public <K1, V1> GatewayProtectedCacheProxy<K1, V1> keepBinary() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return new GatewayProtectedCacheProxy<>((IgniteCacheProxy<K1, V1>) delegate, opCtx.keepBinary(), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public GatewayProtectedCacheProxy<K, V> withDataCenterId(byte dataCenterId) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             Byte prevDataCenterId = opCtx.dataCenterId();
@@ -246,91 +238,79 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
             return new GatewayProtectedCacheProxy<>(delegate, opCtx.setDataCenterId(dataCenterId), lock);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void loadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.loadCache(p, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> loadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.loadCacheAsync(p, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void localLoadCache(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.localLoadCache(p, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> localLoadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localLoadCacheAsync(p, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V getAndPutIfAbsent(K key, V val) throws CacheException, TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndPutIfAbsent(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAndPutIfAbsentAsync(K key, V val) throws CacheException, TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndPutIfAbsentAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -346,1093 +326,937 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public boolean isLocalLocked(K key, boolean byCurrThread) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.isLocalLocked(key, byCurrThread);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <R> QueryCursor<R> query(Query<R> qry) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.query(qry);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public FieldsQueryCursor<List<?>> query(SqlFieldsQuery qry) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.query(qry);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public List<FieldsQueryCursor<List<?>>> queryMultipleStatements(SqlFieldsQuery qry) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.queryMultipleStatements(qry);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> QueryCursor<R> query(Query<T> qry, IgniteClosure<T, R> transformer) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.query(qry, transformer);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Iterable<Entry<K, V>> localEntries(CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localEntries(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public QueryMetrics queryMetrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.queryMetrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void resetQueryMetrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.resetQueryMetrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Collection<? extends QueryDetailMetrics> queryDetailMetrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.queryDetailMetrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void resetQueryDetailMetrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.resetQueryDetailMetrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void localEvict(Collection<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.localEvict(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V localPeek(K key, CachePeekMode... peekModes) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localPeek(key, peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public int size(CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.size(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Integer> sizeAsync(CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.sizeAsync(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public long sizeLong(CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.sizeLong(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Long> sizeLongAsync(CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.sizeLongAsync(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public long sizeLong(int partition, CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.sizeLong(partition, peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Long> sizeLongAsync(int partition, CachePeekMode... peekModes) throws CacheException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.sizeLongAsync(partition, peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public int localSize(CachePeekMode... peekModes) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localSize(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public long localSizeLong(CachePeekMode... peekModes) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localSizeLong(peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public long localSizeLong(int partition, CachePeekMode... peekModes) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localSizeLong(partition, peekModes);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Map<? extends K, ? extends EntryProcessor<K, V, T>> map, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAll(map, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Map<? extends K, ? extends EntryProcessor<K, V, T>> map, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAllAsync(map, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V get(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.get(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAsync(K key) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public CacheEntry<K, V> getEntry(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getEntry(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<CacheEntry<K, V>> getEntryAsync(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getEntryAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Map<K, V> getAll(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAll(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Map<K, V>> getAllAsync(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAllAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Collection<CacheEntry<K, V>> getEntries(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getEntries(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Collection<CacheEntry<K, V>>> getEntriesAsync(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getEntriesAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Map<K, V> getAllOutTx(Set<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAllOutTx(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Map<K, V>> getAllOutTxAsync(Set<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAllOutTxAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean containsKey(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.containsKey(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void loadAll(Set<? extends K> keys, boolean replaceExisting, CompletionListener completionListener) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.loadAll(keys, replaceExisting, completionListener);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> containsKeyAsync(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.containsKeyAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean containsKeys(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.containsKeys(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> containsKeysAsync(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.containsKeysAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void put(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.put(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> putAsync(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.putAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V getAndPut(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndPut(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAndPutAsync(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndPutAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void putAll(Map<? extends K, ? extends V> map) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.putAll(map);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> putAllAsync(Map<? extends K, ? extends V> map) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.putAllAsync(map);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean putIfAbsent(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.putIfAbsent(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> putIfAbsentAsync(K key, V val) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.putIfAbsentAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.remove(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> removeAsync(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.removeAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean remove(K key, V oldVal) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.remove(key, oldVal);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> removeAsync(K key, V oldVal) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.removeAsync(key, oldVal);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V getAndRemove(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndRemove(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAndRemoveAsync(K key) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndRemoveAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean replace(K key, V oldVal, V newVal) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.replace(key, oldVal, newVal);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.replaceAsync(key, oldVal, newVal);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public boolean replace(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.replace(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> replaceAsync(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.replaceAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public V getAndReplace(K key, V val) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndReplace(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAndReplaceAsync(K key, V val) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.getAndReplaceAsync(key, val);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void removeAll(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.removeAll(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> removeAllAsync(Set<? extends K> keys) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.removeAllAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void removeAll() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.removeAll();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> removeAllAsync() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.removeAllAsync();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void clear() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.clear();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> clearAsync() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.clearAsync();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void clear(K key) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.clear(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> clearAsync(K key) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.clearAsync(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void clearAll(Set<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.clearAll(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Void> clearAllAsync(Set<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.clearAllAsync(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void localClear(K key) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.localClear(key);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void localClearAll(Set<? extends K> keys) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.localClearAll(keys);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> T invoke(K key, EntryProcessor<K, V, T> entryProcessor, Object... arguments) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invoke(key, entryProcessor, arguments);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteFuture<T> invokeAsync(K key, EntryProcessor<K, V, T> entryProcessor, Object... arguments) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAsync(key, entryProcessor, arguments);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> T invoke(K key, CacheEntryProcessor<K, V, T> entryProcessor, Object... arguments) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invoke(key, entryProcessor, arguments);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteFuture<T> invokeAsync(K key, CacheEntryProcessor<K, V, T> entryProcessor, Object... arguments) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAsync(key, entryProcessor, arguments);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys, EntryProcessor<K, V, T> entryProcessor, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAll(keys, entryProcessor, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys, EntryProcessor<K, V, T> entryProcessor, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAllAsync(keys, entryProcessor, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys, CacheEntryProcessor<K, V, T> entryProcessor, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAll(keys, entryProcessor, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys, CacheEntryProcessor<K, V, T> entryProcessor, Object... args) throws TransactionException {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.invokeAllAsync(keys, entryProcessor, args);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -1443,43 +1267,37 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public void registerCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.registerCacheEntryListener(cacheEntryListenerConfiguration);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void deregisterCacheEntryListener(CacheEntryListenerConfiguration<K, V> cacheEntryListenerConfiguration) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.deregisterCacheEntryListener(cacheEntryListenerConfiguration);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Iterator<Entry<K, V>> iterator() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.iterator();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -1550,99 +1368,85 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
 
     /** {@inheritDoc} */
     @Override public CacheMetrics metrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.metrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public CacheMetrics metrics(ClusterGroup grp) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.metrics(grp);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public CacheMetrics localMetrics() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localMetrics();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public CacheMetricsMXBean mxBean() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.mxBean();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public CacheMetricsMXBean localMxBean() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.localMxBean();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public Collection<Integer> lostPartitions() {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             return delegate.lostPartitions();
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void enableStatistics(boolean enabled) {
-        GridCacheGateway<K, V> gate = gate();
-
-        CacheOperationContext prev = onEnter(gate, opCtx);
+        CacheOperationGate opGate = onEnter();
 
         try {
             delegate.enableStatistics(enabled);
         }
         finally {
-            onLeave(gate, prev);
+            onLeave(opGate);
         }
     }
 
@@ -1662,26 +1466,49 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
      *
      * @param gate Cache gateway.
      */
-    private void checkProxyIsValid(@Nullable GridCacheGateway<K, V> gate) {
+    private GridCacheGateway<K, V> checkProxyIsValid(@Nullable GridCacheGateway<K, V> gate, boolean tryRestart) {
         if (isProxyClosed())
             throw new IllegalStateException("Cache has been closed: " + context().name());
 
-        if (delegate instanceof IgniteCacheProxyImpl)
+        boolean isCacheProxy = delegate instanceof IgniteCacheProxyImpl;
+
+        if (isCacheProxy)
             ((IgniteCacheProxyImpl) delegate).checkRestart();
 
         if (gate == null)
             throw new IllegalStateException("Gateway is unavailable. Probably cache has been destroyed, but proxy is not closed.");
+
+        if (isCacheProxy && tryRestart && gate.isStopped() &&
+            context().kernalContext().gateway().getState() == GridKernalState.STARTED) {
+            IgniteCacheProxyImpl proxyImpl = (IgniteCacheProxyImpl) delegate;
+
+            try {
+                IgniteInternalCache<K, V> cache = context().kernalContext().cache().<K, V>publicJCache(context().name()).internalProxy();
+
+                GridFutureAdapter<Void> fut = proxyImpl.opportunisticRestart();
+
+                if (fut == null)
+                    proxyImpl.onRestarted(cache.context(), cache.context().cache());
+                else
+                    new IgniteFutureImpl<>(fut).get();
+
+                return gate();
+            } catch (IgniteCheckedException ice) {
+                // Opportunity didn't work out.
+            }
+        }
+
+        return gate;
     }
 
     /**
-     * @param gate Cache gateway.
-     * @param opCtx Cache operation context to guard.
      * @return Previous projection set on this thread.
      */
-    private CacheOperationContext onEnter(@Nullable GridCacheGateway<K, V> gate, CacheOperationContext opCtx) {
-        checkProxyIsValid(gate);
+    private CacheOperationGate onEnter() {
+        GridCacheGateway<K, V> gate = checkProxyIsValid(gate(), true);
 
-        return lock ? gate.enter(opCtx) : gate.enterNoLock(opCtx);
+        return new CacheOperationGate(gate,
+            lock ? gate.enter(opCtx) : gate.enterNoLock(opCtx));
     }
 
     /**
@@ -1690,7 +1517,7 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
      */
     private boolean onEnterIfNoStop(@Nullable GridCacheGateway<K, V> gate) {
         try {
-            checkProxyIsValid(gate);
+            checkProxyIsValid(gate, false);
         }
         catch (Exception e) {
             return false;
@@ -1700,14 +1527,13 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
     }
 
     /**
-     * @param gate Cache gateway.
-     * @param opCtx Operation context to guard.
+     * @param opGate Operation context to guard.
      */
-    private void onLeave(GridCacheGateway<K, V> gate, CacheOperationContext opCtx) {
+    private void onLeave(CacheOperationGate opGate) {
         if (lock)
-            gate.leave(opCtx);
+            opGate.gate.leave(opGate.prev);
         else
-            gate.leaveNoLock(opCtx);
+            opGate.gate.leaveNoLock(opGate.prev);
     }
 
     /**
@@ -1774,4 +1600,28 @@ public class GatewayProtectedCacheProxy<K, V> extends AsyncSupportAdapter<Ignite
     @Override public int hashCode() {
         return delegate.hashCode();
     }
+
+    /**
+     * Holder for gate being entered and operation context to restore.
+     */
+    private class CacheOperationGate {
+        /**
+         * Gate being entered in this operation.
+         */
+        public final GridCacheGateway<K, V> gate;
+
+        /**
+         * Operation context to restore after current operation completes.
+         */
+        public final CacheOperationContext prev;
+
+        /**
+         * @param gate Gate being entered in this operation.
+         * @param prev Operation context to restore after current operation completes.
+         */
+        public CacheOperationGate(GridCacheGateway<K, V> gate, CacheOperationContext prev) {
+            this.gate = gate;
+            this.prev = prev;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
index b9a4b25..658ca2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
@@ -256,6 +256,13 @@ public class GridCacheGateway<K, V> {
     /**
      *
      */
+    public boolean isStopped() {
+        return !checkState(false, false);
+    }
+
+    /**
+     *
+     */
     public void stopped() {
         state.set(State.STOPPED);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
index be4b0db..68e5b85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxyImpl.java
@@ -1824,8 +1824,10 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
      * Throws {@code IgniteCacheRestartingException} if proxy is restarting.
      */
     public void checkRestart() {
-        if (isRestarting())
-            throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(restartFut.get()), "Cache is restarting: " +
+        GridFutureAdapter<Void> currentFut = this.restartFut.get();
+
+        if (currentFut != null)
+            throw new IgniteCacheRestartingException(new IgniteFutureImpl<>(currentFut), "Cache is restarting: " +
                     context().name());
     }
 
@@ -1833,13 +1835,13 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
      * @return True if proxy is restarting, false in other case.
      */
     public boolean isRestarting() {
-        return restartFut != null && restartFut.get() != null;
+        return restartFut.get() != null;
     }
 
     /**
      * Restarts this cache proxy.
      */
-    public void restart() {
+    public boolean restart() {
         GridFutureAdapter<Void> restartFut = new GridFutureAdapter<>();
 
         final GridFutureAdapter<Void> curFut = this.restartFut.get();
@@ -1855,6 +1857,27 @@ public class IgniteCacheProxyImpl<K, V> extends AsyncSupportAdapter<IgniteCache<
                         curFut.onDone();
                 }
             });
+
+        return changed;
+    }
+
+    /**
+     * If proxy is already being restarted, returns future to wait on, else restarts this cache proxy.
+     *
+     * @return Future to wait on, or null.
+     */
+    public GridFutureAdapter<Void> opportunisticRestart() {
+        GridFutureAdapter<Void> restartFut = new GridFutureAdapter<>();
+
+        while (true) {
+            if (this.restartFut.compareAndSet(null, restartFut))
+                return null;
+
+            GridFutureAdapter<Void> curFut = this.restartFut.get();
+
+            if (curFut != null)
+                return curFut;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 16c5d3a..b22a397 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -1616,8 +1616,6 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                             onDisconnected();
 
-                            notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes());
-
                             UUID newId = UUID.randomUUID();
 
                             U.quietAndWarn(log, "Local node will try to reconnect to cluster with new id due " +
@@ -1716,8 +1714,6 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 }
 
                                 onDisconnected();
-
-                                notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes());
                             }
 
                             UUID newId = UUID.randomUUID();
@@ -1820,6 +1816,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
             delayDiscoData.clear();
 
+            notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes());
+
             IgniteClientDisconnectedCheckedException err =
                 new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " +
                     "client node disconnected.");

http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
index dea491c..d0d392b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/IgniteCacheQueryCacheDestroySelfTest.java
@@ -48,6 +48,10 @@ public class IgniteCacheQueryCacheDestroySelfTest extends GridCommonAbstractTest
     /** */
     public static final int GRID_CNT = 3;
 
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
     /**
      * The main test code.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/0991437a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java
index 392cdc7..505d373 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/ClientReconnectAfterClusterRestartTest.java
@@ -17,8 +17,10 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import javax.cache.CacheException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -31,9 +33,8 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgnitePredicate;
-import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jetbrains.annotations.NotNull;
 
@@ -119,6 +120,8 @@ public class ClientReconnectAfterClusterRestartTest extends GridCommonAbstractTe
 
             checkTopology(2);
 
+            IgniteCache<Long, BinaryObject> cache = client.getOrCreateCache(CACHE_PARAMS).withKeepBinary();
+
             client.events().localListen(new IgnitePredicate<Event>() {
 
                 @Override public boolean apply(Event event) {
@@ -161,27 +164,17 @@ public class ClientReconnectAfterClusterRestartTest extends GridCommonAbstractTe
 
             startGrid(0);
 
-            assert GridTestUtils.waitForCondition(new GridAbsPredicate() {
-                @Override public boolean apply() {
-                    try {
-                        checkTopology(2);
-
-                        return true;
-                    } catch (Exception ex) {
-                        return false;
-                    }
-                }
-            }, 30_000);
+            try {
+                assertNull(cache.get(1L));
+            } catch (CacheException ce) {
+                IgniteClientDisconnectedException icde = (IgniteClientDisconnectedException)ce.getCause();
 
-            info("Pre-insert");
+                icde.reconnectFuture().get();
 
-            streamer = client.dataStreamer("PPRB_PARAMS");
-            streamer.allowOverwrite(true);
-            streamer.keepBinary(true);
-            streamer.perNodeBufferSize(10000);
-            streamer.perNodeParallelOperations(100);
+                assertNull(cache.get(1L));
+            }
 
-            IgniteCache<Long, BinaryObject> cache = client.getOrCreateCache(CACHE_PARAMS).withKeepBinary();
+            info("Pre-insert");
 
             builder = client.binary().builder("PARAMS");
             builder.setField("ID", 2L);


[05/50] [abbrv] ignite git commit: IGNITE-7222 .NET: Ignore missing IgniteConfiguration.CommunicationFailureResolver

Posted by ag...@apache.org.
IGNITE-7222 .NET: Ignore missing IgniteConfiguration.CommunicationFailureResolver


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

Branch: refs/heads/ignite-7708
Commit: 780fc07be0b257b578647682585c89548e6d695d
Parents: da77b98
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Wed Apr 11 11:51:45 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Wed Apr 11 11:51:45 2018 +0300

----------------------------------------------------------------------
 .../ApiParity/IgniteConfigurationParityTest.cs                    | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/780fc07b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
index d68083f..bf34fc0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/IgniteConfigurationParityTest.cs
@@ -63,7 +63,8 @@ namespace Apache.Ignite.Core.Tests.ApiParity
             "ClassLoader",
             "CacheStoreSessionListenerFactories",
             "PlatformConfiguration",
-            "ExecutorConfiguration"
+            "ExecutorConfiguration",
+            "CommunicationFailureResolver"
         };
 
         /** Properties that are missing on .NET side. */


[04/50] [abbrv] ignite git commit: IGNITE-7871 Implemented additional synchronization phase for correct partition counters update

Posted by ag...@apache.org.
IGNITE-7871 Implemented additional synchronization phase for correct partition counters update


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

Branch: refs/heads/ignite-7708
Commit: da77b9818a70495b7afdf6899ebd9180dadd7f68
Parents: f4de6df
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Wed Apr 11 11:23:46 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Apr 11 11:23:46 2018 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/GridTopic.java   |   5 +-
 .../communication/GridIoMessageFactory.java     |   6 +
 .../discovery/GridDiscoveryManager.java         |  10 +
 .../MetaPageUpdatePartitionDataRecord.java      |   2 +-
 .../processors/cache/CacheMetricsImpl.java      |   2 +-
 .../processors/cache/GridCacheMvccManager.java  |  38 +
 .../GridCachePartitionExchangeManager.java      |  17 +
 .../cache/GridCacheSharedContext.java           |   9 +-
 .../processors/cache/GridCacheUtils.java        |   2 +-
 .../cache/IgniteCacheOffheapManager.java        |   8 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |  10 +-
 .../dht/GridClientPartitionTopology.java        |   5 +
 .../distributed/dht/GridDhtLocalPartition.java  |   9 +-
 .../dht/GridDhtPartitionTopology.java           |   6 +
 .../dht/GridDhtPartitionTopologyImpl.java       |  26 +-
 .../dht/GridDhtPartitionsStateValidator.java    | 255 +++++++
 .../cache/distributed/dht/GridDhtTxLocal.java   |   5 +
 .../GridDhtPartitionsExchangeFuture.java        |  96 ++-
 .../GridDhtPartitionsSingleMessage.java         |  68 +-
 .../dht/preloader/InitNewCoordinatorFuture.java |   2 +-
 .../preloader/latch/ExchangeLatchManager.java   | 695 +++++++++++++++++++
 .../distributed/dht/preloader/latch/Latch.java  |  52 ++
 .../dht/preloader/latch/LatchAckMessage.java    | 165 +++++
 .../cache/distributed/near/GridNearTxLocal.java |  10 +
 .../persistence/GridCacheOffheapManager.java    |  10 +-
 .../cache/transactions/IgniteTxAdapter.java     |   2 +-
 .../cache/transactions/IgniteTxManager.java     |  36 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |   2 +-
 .../processors/cache/IgniteCacheGroupsTest.java |   1 +
 ...ExchangeLatchManagerCoordinatorFailTest.java | 244 +++++++
 .../GridCachePartitionsStateValidationTest.java | 316 +++++++++
 ...idCachePartitionsStateValidatorSelfTest.java | 158 +++++
 .../TxOptimisticOnPartitionExchangeTest.java    | 322 +++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +
 .../testsuites/IgniteCacheTestSuite6.java       |   6 +
 35 files changed, 2568 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 1227e8c..0b2d41a 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
@@ -124,7 +124,10 @@ public enum GridTopic {
     TOPIC_METRICS,
 
     /** */
-    TOPIC_AUTH;
+    TOPIC_AUTH,
+
+    /** */
+    TOPIC_EXCHANGE;
 
     /** Enum values. */
     private static final GridTopic[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 5616fd0..581c32e 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
@@ -53,6 +53,7 @@ import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
 import org.apache.ignite.internal.processors.cache.WalStateAckMessage;
 import org.apache.ignite.internal.processors.cache.binary.MetadataRequestMessage;
 import org.apache.ignite.internal.processors.cache.binary.MetadataResponseMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.LatchAckMessage;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTtlUpdateRequest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryRequest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheTxRecoveryResponse;
@@ -921,6 +922,11 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
+            case 135:
+                msg = new LatchAckMessage();
+
+                break;
+
             // [-3..119] [124..129] [-23..-27] [-36..-55]- this
             // [120..123] - DR
             // [-4..-22, -30..-35] - SQL

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a1d84e5..400bb5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -793,6 +793,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     ((IgniteKernal)ctx.grid()).onDisconnected();
 
+                    if (!locJoin.isDone())
+                        locJoin.onDone(new IgniteCheckedException("Node disconnected"));
+
                     locJoin = new GridFutureAdapter<>();
 
                     registeredCaches.clear();
@@ -2142,6 +2145,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @return Local join future.
+     */
+    public GridFutureAdapter<DiscoveryLocalJoinData> localJoinFuture() {
+        return locJoin;
+    }
+
+    /**
      * @param msg Custom message.
      * @throws IgniteCheckedException If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
index bafbf47..e5bd343 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/pagemem/wal/record/delta/MetaPageUpdatePartitionDataRecord.java
@@ -32,7 +32,7 @@ public class MetaPageUpdatePartitionDataRecord extends PageDeltaRecord {
     /** */
     private long globalRmvId;
 
-    /** */
+    /** TODO: Partition size may be long */
     private int partSize;
 
     /** */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
index 6fae8fe..b402ff2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheMetricsImpl.java
@@ -792,7 +792,7 @@ public class CacheMetricsImpl implements CacheMetrics {
                     if (cctx.cache() == null)
                         continue;
 
-                    int cacheSize = part.dataStore().cacheSize(cctx.cacheId());
+                    long cacheSize = part.dataStore().cacheSize(cctx.cacheId());
 
                     offHeapEntriesCnt += cacheSize;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index a9fa3c7..fade833 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -44,6 +44,8 @@ import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheMappedVersion;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedCacheEntry;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTxFinishFuture;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxFinishFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
@@ -314,6 +316,42 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Creates a future that will wait for finishing all remote transactions (primary -> backup)
+     * with topology version less or equal to {@code topVer}.
+     *
+     * @param topVer Topology version.
+     * @return Compound future of all {@link GridDhtTxFinishFuture} futures.
+     */
+    public IgniteInternalFuture<?> finishRemoteTxs(AffinityTopologyVersion topVer) {
+        GridCompoundFuture<?, ?> res = new CacheObjectsReleaseFuture<>("RemoteTx", topVer);
+
+        for (GridCacheFuture<?> fut : futs.values()) {
+            if (fut instanceof GridDhtTxFinishFuture) {
+                GridDhtTxFinishFuture finishTxFuture = (GridDhtTxFinishFuture) fut;
+
+                if (cctx.tm().needWaitTransaction(finishTxFuture.tx(), topVer))
+                    res.add(ignoreErrors(finishTxFuture));
+            }
+        }
+
+        res.markInitialized();
+
+        return res;
+    }
+
+    /**
+     * Future wrapper which ignores any underlying future errors.
+     *
+     * @param f Underlying future.
+     * @return Future wrapper which ignore any underlying future errors.
+     */
+    private IgniteInternalFuture ignoreErrors(IgniteInternalFuture<?> f) {
+        GridFutureAdapter<?> wrapper = new GridFutureAdapter();
+        f.listen(future -> wrapper.onDone());
+        return wrapper;
+    }
+
+    /**
      * @param leftNodeId Left node ID.
      * @param topVer Topology version.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 1a0e65f..20a3ccb 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
@@ -64,6 +64,7 @@ import org.apache.ignite.internal.managers.discovery.DiscoveryLocalJoinData;
 import org.apache.ignite.internal.managers.eventstorage.DiscoveryEventListener;
 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.preloader.latch.ExchangeLatchManager;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
@@ -216,6 +217,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** For tests only. */
     private volatile AffinityTopologyVersion exchMergeTestWaitVer;
 
+    /** Distributed latch manager. */
+    private ExchangeLatchManager latchMgr;
+
     /** Discovery listener. */
     private final DiscoveryEventListener discoLsnr = new DiscoveryEventListener() {
         @Override public void onEvent(DiscoveryEvent evt, DiscoCache cache) {
@@ -309,6 +313,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         exchWorker = new ExchangeWorker();
 
+        latchMgr = new ExchangeLatchManager(cctx.kernalContext());
+
         cctx.gridEvents().addDiscoveryEventListener(discoLsnr, EVT_NODE_JOINED, EVT_NODE_LEFT, EVT_NODE_FAILED,
             EVT_DISCOVERY_CUSTOM_EVT);
 
@@ -1255,6 +1261,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                     m.addPartitionUpdateCounters(grp.groupId(),
                         newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
+
+                    m.addPartitionSizes(grp.groupId(), grp.topology().partitionSizes());
                 }
             }
         }
@@ -1277,6 +1285,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
                 m.addPartitionUpdateCounters(top.groupId(),
                     newCntrMap ? cntrsMap : CachePartitionPartialCountersMap.toCountersMap(cntrsMap));
+
+                m.addPartitionSizes(top.groupId(), top.partitionSizes());
             }
         }
 
@@ -1570,6 +1580,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     * @return Latch manager instance.
+     */
+    public ExchangeLatchManager latch() {
+        return latchMgr;
+    }
+
+    /**
      * @param exchFut Optional current exchange future.
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index c2f9229..b3b4f0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -26,7 +26,6 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicIntegerArray;
-import java.util.function.BiFunction;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
@@ -711,7 +710,7 @@ public class GridCacheSharedContext<K, V> {
 
     /**
      * @return Ttl cleanup manager.
-     * */
+     */
     public GridCacheSharedTtlCleanupManager ttl() {
         return ttlMgr;
     }
@@ -854,10 +853,14 @@ public class GridCacheSharedContext<K, V> {
         GridCompoundFuture f = new CacheObjectsReleaseFuture("Partition", topVer);
 
         f.add(mvcc().finishExplicitLocks(topVer));
-        f.add(tm().finishTxs(topVer));
         f.add(mvcc().finishAtomicUpdates(topVer));
         f.add(mvcc().finishDataStreamerUpdates(topVer));
 
+        IgniteInternalFuture<?> finishLocalTxsFuture = tm().finishLocalTxs(topVer);
+        // To properly track progress of finishing local tx updates we explicitly add this future to compound set.
+        f.add(finishLocalTxsFuture);
+        f.add(tm().finishAllTxs(finishLocalTxsFuture, topVer));
+
         f.markInitialized();
 
         return f;

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 a5169d2..d672420 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
@@ -1732,7 +1732,7 @@ public class GridCacheUtils {
                             ver,
                             expiryPlc == null ? 0 : expiryPlc.forCreate(),
                             expiryPlc == null ? 0 : toExpireTime(expiryPlc.forCreate()),
-                            false,
+                            true,
                             topVer,
                             GridDrType.DR_BACKUP,
                             true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
index 3d83f87..a12c033 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManager.java
@@ -22,11 +22,11 @@ import javax.cache.Cache;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.IgniteDhtDemandedPartitionsMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.persistence.CacheDataRow;
 import org.apache.ignite.internal.processors.cache.persistence.RootPage;
 import org.apache.ignite.internal.processors.cache.persistence.RowStore;
 import org.apache.ignite.internal.processors.cache.persistence.tree.reuse.ReuseList;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
 import org.apache.ignite.internal.processors.query.GridQueryRowCacheCleaner;
 import org.apache.ignite.internal.util.GridAtomicLong;
@@ -344,7 +344,7 @@ public interface IgniteCacheOffheapManager {
      * @param part Partition.
      * @return Number of entries.
      */
-    public int totalPartitionEntriesCount(int part);
+    public long totalPartitionEntriesCount(int part);
 
     /**
      *
@@ -381,7 +381,7 @@ public interface IgniteCacheOffheapManager {
          * @param cacheId Cache ID.
          * @return Size.
          */
-        int cacheSize(int cacheId);
+        long cacheSize(int cacheId);
 
         /**
          * @return Cache sizes if store belongs to group containing multiple caches.
@@ -391,7 +391,7 @@ public interface IgniteCacheOffheapManager {
         /**
          * @return Total size.
          */
-        int fullSize();
+        long fullSize();
 
         /**
          * @return Update counter.

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
index b201935..f8cc86f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java
@@ -252,7 +252,7 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
     }
 
     /** {@inheritDoc} */
-    @Override public int totalPartitionEntriesCount(int p) {
+    @Override public long totalPartitionEntriesCount(int p) {
         if (grp.isLocal())
             return locCacheDataStore.fullSize();
         else {
@@ -1152,14 +1152,14 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         }
 
         /** {@inheritDoc} */
-        @Override public int cacheSize(int cacheId) {
+        @Override public long cacheSize(int cacheId) {
             if (grp.sharedGroup()) {
                 AtomicLong size = cacheSizes.get(cacheId);
 
                 return size != null ? (int)size.get() : 0;
             }
 
-            return (int)storageSize.get();
+            return storageSize.get();
         }
 
         /** {@inheritDoc} */
@@ -1176,8 +1176,8 @@ public class IgniteCacheOffheapManagerImpl implements IgniteCacheOffheapManager
         }
 
         /** {@inheritDoc} */
-        @Override public int fullSize() {
-            return (int)storageSize.get();
+        @Override public long fullSize() {
+            return storageSize.get();
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 5bbbb31..3e3bb0d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -1196,6 +1196,11 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public Map<Integer, Long> partitionSizes() {
+        return Collections.emptyMap();
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) {
         assert false : "Should not be called on non-affinity node";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 7a47f31..ea20dbf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -929,7 +929,7 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
     /**
      * @return Initial update counter.
      */
-    public Long initialUpdateCounter() {
+    public long initialUpdateCounter() {
         return store.initialUpdateCounter();
     }
 
@@ -948,6 +948,13 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
     }
 
     /**
+     * @return Total size of all caches.
+     */
+    public long fullSize() {
+        return store.fullSize();
+    }
+
+    /**
      * Removes all entries and rows from this partition.
      *
      * @return Number of rows cleared from page memory.

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 7f900cb..6f68dbb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
@@ -345,6 +346,11 @@ public interface GridDhtPartitionTopology {
     public CachePartitionPartialCountersMap localUpdateCounters(boolean skipZeros);
 
     /**
+     * @return Partition cache sizes.
+     */
+    public Map<Integer, Long> partitionSizes();
+
+    /**
      * @param part Partition to own.
      * @return {@code True} if owned.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index 538c57e..740903e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -31,6 +31,8 @@ import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReferenceArray;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -2526,6 +2528,28 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public Map<Integer, Long> partitionSizes() {
+        lock.readLock().lock();
+
+        try {
+            Map<Integer, Long> partitionSizes = new HashMap<>();
+
+            for (int p = 0; p < locParts.length(); p++) {
+                GridDhtLocalPartition part = locParts.get(p);
+                if (part == null || part.fullSize() == 0)
+                    continue;
+
+                partitionSizes.put(part.id(), part.fullSize());
+            }
+
+            return partitionSizes;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean rebalanceFinished(AffinityTopologyVersion topVer) {
         AffinityTopologyVersion curTopVer = this.readyTopVer;
 
@@ -2587,7 +2611,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (part == null)
                     continue;
 
-                int size = part.dataStore().fullSize();
+                long size = part.dataStore().fullSize();
 
                 if (size >= threshold)
                     X.println(">>>   Local partition [part=" + part.id() + ", size=" + size + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java
new file mode 100644
index 0000000..92a0584
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionsStateValidator.java
@@ -0,0 +1,255 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+import java.util.UUID;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.events.DiscoveryEvent;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.SB;
+import org.apache.ignite.lang.IgniteProductVersion;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
+
+/**
+ * Class to validate partitions update counters and cache sizes during exchange process.
+ */
+public class GridDhtPartitionsStateValidator {
+    /** Version since node is able to send cache sizes in {@link GridDhtPartitionsSingleMessage}. */
+    private static final IgniteProductVersion SIZES_VALIDATION_AVAILABLE_SINCE = IgniteProductVersion.fromString("2.5.0");
+
+    /** Cache shared context. */
+    private final GridCacheSharedContext<?, ?> cctx;
+
+    /**
+     * Constructor.
+     *
+     * @param cctx Cache shared context.
+     */
+    public GridDhtPartitionsStateValidator(GridCacheSharedContext<?, ?> cctx) {
+        this.cctx = cctx;
+    }
+
+    /**
+     * Validates partition states - update counters and cache sizes for all nodes.
+     * If update counter value or cache size for the same partitions are different on some nodes
+     * method throws exception with full information about inconsistent partitions.
+     *
+     * @param fut Current exchange future.
+     * @param top Topology to validate.
+     * @param messages Single messages received from all nodes.
+     * @throws IgniteCheckedException If validation failed. Exception message contains
+     * full information about all partitions which update counters or cache sizes are not consistent.
+     */
+    public void validatePartitionCountersAndSizes(GridDhtPartitionsExchangeFuture fut,
+                                                  GridDhtPartitionTopology top,
+                                                  Map<UUID, GridDhtPartitionsSingleMessage> messages) throws IgniteCheckedException {
+        // Ignore just joined nodes.
+        final Set<UUID> ignoringNodes = new HashSet<>();
+
+        for (DiscoveryEvent evt : fut.events().events())
+            if (evt.type() == EVT_NODE_JOINED)
+                ignoringNodes.add(evt.eventNode().id());
+
+        AffinityTopologyVersion topVer = fut.context().events().topologyVersion();
+
+        // Validate update counters.
+        Map<Integer, Map<UUID, Long>> result = validatePartitionsUpdateCounters(top, messages, ignoringNodes);
+        if (!result.isEmpty())
+            throw new IgniteCheckedException("Partitions update counters are inconsistent for " + fold(topVer, result));
+
+        // For sizes validation ignore also nodes which are not able to send cache sizes.
+        for (UUID id : messages.keySet()) {
+            ClusterNode node = cctx.discovery().node(id);
+            if (node != null && node.version().compareTo(SIZES_VALIDATION_AVAILABLE_SINCE) < 0)
+                ignoringNodes.add(id);
+        }
+
+        // Validate cache sizes.
+        result = validatePartitionsSizes(top, messages, ignoringNodes);
+        if (!result.isEmpty())
+            throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result));
+    }
+
+    /**
+     * Validate partitions update counters for given {@code top}.
+     *
+     * @param top Topology to validate.
+     * @param messages Single messages received from all nodes.
+     * @param ignoringNodes Nodes for what we ignore validation.
+     * @return Invalid partitions map with following structure: (partId, (nodeId, updateCounter)).
+     * If map is empty validation is successful.
+     */
+     Map<Integer, Map<UUID, Long>> validatePartitionsUpdateCounters(
+            GridDhtPartitionTopology top,
+            Map<UUID, GridDhtPartitionsSingleMessage> messages,
+            Set<UUID> ignoringNodes) {
+        Map<Integer, Map<UUID, Long>> invalidPartitions = new HashMap<>();
+
+        Map<Integer, T2<UUID, Long>> updateCountersAndNodesByPartitions = new HashMap<>();
+
+        // Populate counters statistics from local node partitions.
+        for (GridDhtLocalPartition part : top.currentLocalPartitions()) {
+            if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING)
+                continue;
+
+            updateCountersAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.updateCounter()));
+        }
+
+        int partitions = top.partitions();
+
+        // Then process and validate counters from other nodes.
+        for (Map.Entry<UUID, GridDhtPartitionsSingleMessage> e : messages.entrySet()) {
+            UUID nodeId = e.getKey();
+            if (ignoringNodes.contains(nodeId))
+                continue;
+
+            CachePartitionPartialCountersMap countersMap = e.getValue().partitionUpdateCounters(top.groupId(), partitions);
+
+            for (int part = 0; part < partitions; part++) {
+                if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING)
+                    continue;
+
+                int partIdx = countersMap.partitionIndex(part);
+                long currentCounter = partIdx >= 0 ? countersMap.updateCounterAt(partIdx) : 0;
+
+                process(invalidPartitions, updateCountersAndNodesByPartitions, part, nodeId, currentCounter);
+            }
+        }
+
+        return invalidPartitions;
+    }
+
+    /**
+     * Validate partitions cache sizes for given {@code top}.
+     *
+     * @param top Topology to validate.
+     * @param messages Single messages received from all nodes.
+     * @param ignoringNodes Nodes for what we ignore validation.
+     * @return Invalid partitions map with following structure: (partId, (nodeId, cacheSize)).
+     * If map is empty validation is successful.
+     */
+     Map<Integer, Map<UUID, Long>> validatePartitionsSizes(
+            GridDhtPartitionTopology top,
+            Map<UUID, GridDhtPartitionsSingleMessage> messages,
+            Set<UUID> ignoringNodes) {
+        Map<Integer, Map<UUID, Long>> invalidPartitions = new HashMap<>();
+
+        Map<Integer, T2<UUID, Long>> sizesAndNodesByPartitions = new HashMap<>();
+
+        // Populate sizes statistics from local node partitions.
+        for (GridDhtLocalPartition part : top.currentLocalPartitions()) {
+            if (top.partitionState(cctx.localNodeId(), part.id()) != GridDhtPartitionState.OWNING)
+                continue;
+
+            sizesAndNodesByPartitions.put(part.id(), new T2<>(cctx.localNodeId(), part.fullSize()));
+        }
+
+        int partitions = top.partitions();
+
+        // Then process and validate sizes from other nodes.
+        for (Map.Entry<UUID, GridDhtPartitionsSingleMessage> e : messages.entrySet()) {
+            UUID nodeId = e.getKey();
+            if (ignoringNodes.contains(nodeId))
+                continue;
+
+            Map<Integer, Long> sizesMap = e.getValue().partitionSizes(top.groupId());
+
+            for (int part = 0; part < partitions; part++) {
+                if (top.partitionState(nodeId, part) != GridDhtPartitionState.OWNING)
+                    continue;
+
+                long currentSize = sizesMap.containsKey(part) ? sizesMap.get(part) : 0L;
+
+                process(invalidPartitions, sizesAndNodesByPartitions, part, nodeId, currentSize);
+            }
+        }
+
+        return invalidPartitions;
+    }
+
+    /**
+     * Processes given {@code counter} for partition {@code part} reported by {@code node}.
+     * Populates {@code invalidPartitions} map if existing counter and current {@code counter} are different.
+     *
+     * @param invalidPartitions Invalid partitions map.
+     * @param countersAndNodes Current map of counters and nodes by partitions.
+     * @param part Processing partition.
+     * @param node Node id.
+     * @param counter Counter value reported by {@code node}.
+     */
+    private void process(Map<Integer, Map<UUID, Long>> invalidPartitions,
+                         Map<Integer, T2<UUID, Long>> countersAndNodes,
+                         int part,
+                         UUID node,
+                         long counter) {
+        T2<UUID, Long> existingData = countersAndNodes.get(part);
+
+        if (existingData == null)
+            countersAndNodes.put(part, new T2<>(node, counter));
+
+        if (existingData != null && counter != existingData.get2()) {
+            if (!invalidPartitions.containsKey(part)) {
+                Map<UUID, Long> map = new HashMap<>();
+                map.put(existingData.get1(), existingData.get2());
+                invalidPartitions.put(part, map);
+            }
+
+            invalidPartitions.get(part).put(node, counter);
+        }
+    }
+
+    /**
+     * Folds given map of invalid partition states to string representation in the following format:
+     * Part [id]: [consistentId=value*]
+     *
+     * Value can be both update counter or cache size.
+     *
+     * @param topVer Last topology version.
+     * @param invalidPartitions Invalid partitions map.
+     * @return String representation of invalid partitions.
+     */
+    private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>> invalidPartitions) {
+        SB sb = new SB();
+
+        NavigableMap<Integer, Map<UUID, Long>> sortedPartitions = new TreeMap<>(invalidPartitions);
+
+        for (Map.Entry<Integer, Map<UUID, Long>> p : sortedPartitions.entrySet()) {
+            sb.a("Part ").a(p.getKey()).a(": [");
+            for (Map.Entry<UUID, Long> e : p.getValue().entrySet()) {
+                Object consistentId = cctx.discovery().node(topVer, e.getKey()).consistentId();
+                sb.a(consistentId).a("=").a(e.getValue()).a(" ");
+            }
+            sb.a("] ");
+        }
+
+        return sb.toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 28cc018..0609f04 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
@@ -447,6 +447,11 @@ public class GridDhtTxLocal extends GridDhtTxLocalAdapter implements GridCacheMa
 
             err = e;
         }
+        catch (Throwable t) {
+            fut.onDone(t);
+
+            throw t;
+        }
 
         if (primarySync)
             sendFinishReply(err);

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/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 cbb4985..dd4a571 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
@@ -41,6 +41,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.events.DiscoveryEvent;
@@ -75,10 +76,12 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.LocalJoinCachesContext;
 import org.apache.ignite.internal.processors.cache.StateChangeRequest;
 import org.apache.ignite.internal.processors.cache.WalStateAbstractMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.Latch;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionsStateValidator;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFutureAdapter;
 import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -290,6 +293,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     @GridToStringExclude
     private GridDhtPartitionsExchangeFuture mergedWith;
 
+    /** Validator for partition states. */
+    @GridToStringExclude
+    private final GridDhtPartitionsStateValidator validator;
+
     /**
      * @param cctx Cache context.
      * @param busyLock Busy lock.
@@ -314,6 +321,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         this.exchId = exchId;
         this.exchActions = exchActions;
         this.affChangeMsg = affChangeMsg;
+        this.validator = new GridDhtPartitionsStateValidator(cctx);
 
         log = cctx.logger(getClass());
         exchLog = cctx.logger(EXCHANGE_LOG);
@@ -1099,7 +1107,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
         // To correctly rebalance when persistence is enabled, it is necessary to reserve history within exchange.
         partHistReserved = cctx.database().reserveHistoryForExchange();
 
-        waitPartitionRelease();
+        // On first phase we wait for finishing all local tx updates, atomic updates and lock releases.
+        waitPartitionRelease(1);
+
+        // Second phase is needed to wait for finishing all tx updates from primary to backup nodes remaining after first phase.
+        waitPartitionRelease(2);
 
         boolean topChanged = firstDiscoEvt.type() != EVT_DISCOVERY_CUSTOM_EVT || affChangeMsg != null;
 
@@ -1202,9 +1214,17 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
      * For the exact list of the objects being awaited for see
      * {@link GridCacheSharedContext#partitionReleaseFuture(AffinityTopologyVersion)} javadoc.
      *
+     * @param phase Phase of partition release.
+     *
      * @throws IgniteCheckedException If failed.
      */
-    private void waitPartitionRelease() throws IgniteCheckedException {
+    private void waitPartitionRelease(int phase) throws IgniteCheckedException {
+        Latch releaseLatch = null;
+
+        // Wait for other nodes only on first phase.
+        if (phase == 1)
+            releaseLatch = cctx.exchange().latch().getOrCreate("exchange", initialVersion());
+
         IgniteInternalFuture<?> partReleaseFut = cctx.partitionReleaseFuture(initialVersion());
 
         // Assign to class variable so it will be included into toString() method.
@@ -1238,6 +1258,11 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     nextDumpTime = U.currentTimeMillis() + nextDumpTimeout(dumpCnt++, futTimeout);
                 }
             }
+            catch (IgniteCheckedException e) {
+                U.warn(log,"Unable to await partitions release future", e);
+
+                throw e;
+            }
         }
 
         long waitEnd = U.currentTimeMillis();
@@ -1290,6 +1315,35 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
             }
         }
+
+        if (releaseLatch == null)
+            return;
+
+        releaseLatch.countDown();
+
+        if (!localJoinExchange()) {
+            try {
+                while (true) {
+                    try {
+                        releaseLatch.await(futTimeout, TimeUnit.MILLISECONDS);
+
+                        if (log.isInfoEnabled())
+                            log.info("Finished waiting for partitions release latch: " + releaseLatch);
+
+                        break;
+                    }
+                    catch (IgniteFutureTimeoutCheckedException ignored) {
+                        U.warn(log, "Unable to await partitions release latch within timeout: " + releaseLatch);
+
+                        // Try to resend ack.
+                        releaseLatch.countDown();
+                    }
+                }
+            }
+            catch (IgniteCheckedException e) {
+                U.warn(log, "Stop waiting for partitions release latch: " + e.getMessage());
+            }
+        }
     }
 
     /**
@@ -2499,6 +2553,8 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 }
             }
 
+            validatePartitionsState();
+
             if (firstDiscoEvt.type() == EVT_DISCOVERY_CUSTOM_EVT) {
                 assert firstDiscoEvt instanceof DiscoveryCustomEvent;
 
@@ -2683,6 +2739,42 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
+     * Validates that partition update counters and cache sizes for all caches are consistent.
+     */
+    private void validatePartitionsState() {
+        for (Map.Entry<Integer, CacheGroupDescriptor> e : cctx.affinity().cacheGroups().entrySet()) {
+            CacheGroupDescriptor grpDesc = e.getValue();
+            if (grpDesc.config().getCacheMode() == CacheMode.LOCAL)
+                continue;
+
+            int grpId = e.getKey();
+
+            CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpId);
+
+            GridDhtPartitionTopology top = grpCtx != null ?
+                    grpCtx.topology() :
+                    cctx.exchange().clientTopology(grpId, events().discoveryCache());
+
+            // Do not validate read or write through caches or caches with disabled rebalance.
+            if (grpCtx == null
+                    || grpCtx.config().isReadThrough()
+                    || grpCtx.config().isWriteThrough()
+                    || grpCtx.config().getCacheStoreFactory() != null
+                    || grpCtx.config().getRebalanceDelay() != -1
+                    || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE)
+                continue;
+
+            try {
+                validator.validatePartitionCountersAndSizes(this, top, msgs);
+            }
+            catch (IgniteCheckedException ex) {
+                log.warning("Partition states validation was failed for cache " + grpDesc.cacheOrGroupName(), ex);
+                // TODO: Handle such errors https://issues.apache.org/jira/browse/IGNITE-7833
+            }
+        }
+    }
+
+    /**
      *
      */
     private void assignPartitionsStates() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
index 215152d..6ebafac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsSingleMessage.java
@@ -17,9 +17,9 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 
-import java.util.Collection;
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
@@ -67,6 +67,14 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     /** Serialized partitions counters. */
     private byte[] partCntrsBytes;
 
+    /** Partitions sizes. */
+    @GridToStringInclude
+    @GridDirectTransient
+    private Map<Integer, Map<Integer, Long>> partSizes;
+
+    /** Serialized partitions counters. */
+    private byte[] partSizesBytes;
+
     /** Partitions history reservation counters. */
     @GridToStringInclude
     @GridDirectTransient
@@ -220,6 +228,35 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
     }
 
     /**
+     * Adds partition sizes map for specified {@code grpId} to the current message.
+     *
+     * @param grpId Group id.
+     * @param partSizesMap Partition sizes map.
+     */
+    public void addPartitionSizes(int grpId, Map<Integer, Long> partSizesMap) {
+        if (partSizesMap.isEmpty())
+            return;
+
+        if (partSizes == null)
+            partSizes = new HashMap<>();
+
+        partSizes.put(grpId, partSizesMap);
+    }
+
+    /**
+     * Returns partition sizes map for specified {@code grpId}.
+     *
+     * @param grpId Group id.
+     * @return Partition sizes map (partId, partSize).
+     */
+    public Map<Integer, Long> partitionSizes(int grpId) {
+        if (partSizes == null)
+            return Collections.emptyMap();
+
+        return partSizes.getOrDefault(grpId, Collections.emptyMap());
+    }
+
+    /**
      * @param grpId Cache group ID.
      * @param cntrMap Partition history counters.
      */
@@ -287,12 +324,14 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
         boolean marshal = (parts != null && partsBytes == null) ||
             (partCntrs != null && partCntrsBytes == null) ||
             (partHistCntrs != null && partHistCntrsBytes == null) ||
+            (partSizes != null && partSizesBytes == null) ||
             (err != null && errBytes == null);
 
         if (marshal) {
             byte[] partsBytes0 = null;
             byte[] partCntrsBytes0 = null;
             byte[] partHistCntrsBytes0 = null;
+            byte[] partSizesBytes0 = null;
             byte[] errBytes0 = null;
 
             if (parts != null && partsBytes == null)
@@ -304,6 +343,9 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             if (partHistCntrs != null && partHistCntrsBytes == null)
                 partHistCntrsBytes0 = U.marshal(ctx, partHistCntrs);
 
+            if (partSizes != null && partSizesBytes == null)
+                partSizesBytes0 = U.marshal(ctx, partSizes);
+
             if (err != null && errBytes == null)
                 errBytes0 = U.marshal(ctx, err);
 
@@ -314,11 +356,13 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                     byte[] partsBytesZip = U.zip(partsBytes0);
                     byte[] partCntrsBytesZip = U.zip(partCntrsBytes0);
                     byte[] partHistCntrsBytesZip = U.zip(partHistCntrsBytes0);
+                    byte[] partSizesBytesZip = U.zip(partSizesBytes0);
                     byte[] exBytesZip = U.zip(errBytes0);
 
                     partsBytes0 = partsBytesZip;
                     partCntrsBytes0 = partCntrsBytesZip;
                     partHistCntrsBytes0 = partHistCntrsBytesZip;
+                    partSizesBytes0 = partSizesBytesZip;
                     errBytes0 = exBytesZip;
 
                     compressed(true);
@@ -331,6 +375,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
             partsBytes = partsBytes0;
             partCntrsBytes = partCntrsBytes0;
             partHistCntrsBytes = partHistCntrsBytes0;
+            partSizesBytes = partSizesBytes0;
             errBytes = errBytes0;
         }
     }
@@ -360,6 +405,13 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
                 partHistCntrs = U.unmarshal(ctx, partHistCntrsBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
         }
 
+        if (partSizesBytes != null && partSizes == null) {
+            if (compressed())
+                partSizes = U.unmarshalZip(ctx.marshaller(), partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+            else
+                partSizes = U.unmarshal(ctx, partSizesBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
+        }
+
         if (errBytes != null && err == null) {
             if (compressed())
                 err = U.unmarshalZip(ctx.marshaller(), errBytes, U.resolveClassLoader(ldr, ctx.gridConfig()));
@@ -451,6 +503,11 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 writer.incrementState();
 
+            case 13:
+                if (!writer.writeByteArray("partsSizesBytes", partSizesBytes))
+                    return false;
+
+                writer.incrementState();
         }
 
         return true;
@@ -531,6 +588,13 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
                 reader.incrementState();
 
+            case 13:
+                partSizesBytes = reader.readByteArray("partsSizesBytes");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
         }
 
         return reader.afterMessageRead(GridDhtPartitionsSingleMessage.class);
@@ -543,7 +607,7 @@ public class GridDhtPartitionsSingleMessage extends GridDhtPartitionsAbstractMes
 
     /** {@inheritDoc} */
     @Override public byte fieldsCount() {
-        return 13;
+        return 14;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java
index 596fa8c..42a9ba6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/InitNewCoordinatorFuture.java
@@ -235,7 +235,7 @@ public class InitNewCoordinatorFuture extends GridCompoundFuture {
             if (awaited.remove(node.id())) {
                 GridDhtPartitionsFullMessage fullMsg0 = msg.finishMessage();
 
-                if (fullMsg0 != null) {
+                if (fullMsg0 != null && fullMsg0.resultTopologyVersion() != null) {
                     assert fullMsg == null || fullMsg.resultTopologyVersion().equals(fullMsg0.resultTopologyVersion());
 
                     fullMsg  = fullMsg0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
new file mode 100644
index 0000000..c205cb1
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/ExchangeLatchManager.java
@@ -0,0 +1,695 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantLock;
+import java.util.stream.Collectors;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.GridTopic;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
+import org.apache.ignite.internal.managers.discovery.GridDiscoveryManager;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.tostring.GridToStringExclude;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.T2;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteProductVersion;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
+import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
+
+/**
+ * Class is responsible to create and manage instances of distributed latches {@link Latch}.
+ */
+public class ExchangeLatchManager {
+    /** Version since latch management is available. */
+    private static final IgniteProductVersion VERSION_SINCE = IgniteProductVersion.fromString("2.5.0");
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Context. */
+    private final GridKernalContext ctx;
+
+    /** Discovery manager. */
+    private final GridDiscoveryManager discovery;
+
+    /** IO manager. */
+    private final GridIoManager io;
+
+    /** Current coordinator. */
+    private volatile ClusterNode coordinator;
+
+    /** Pending acks collection. */
+    private final ConcurrentMap<T2<String, AffinityTopologyVersion>, Set<UUID>> pendingAcks = new ConcurrentHashMap<>();
+
+    /** Server latches collection. */
+    private final ConcurrentMap<T2<String, AffinityTopologyVersion>, ServerLatch> serverLatches = new ConcurrentHashMap<>();
+
+    /** Client latches collection. */
+    private final ConcurrentMap<T2<String, AffinityTopologyVersion>, ClientLatch> clientLatches = new ConcurrentHashMap<>();
+
+    /** Lock. */
+    private final ReentrantLock lock = new ReentrantLock();
+
+    /**
+     * Constructor.
+     *
+     * @param ctx Kernal context.
+     */
+    public ExchangeLatchManager(GridKernalContext ctx) {
+        this.ctx = ctx;
+        this.log = ctx.log(getClass());
+        this.discovery = ctx.discovery();
+        this.io = ctx.io();
+
+        if (!ctx.clientNode()) {
+            ctx.io().addMessageListener(GridTopic.TOPIC_EXCHANGE, (nodeId, msg, plc) -> {
+                if (msg instanceof LatchAckMessage) {
+                    processAck(nodeId, (LatchAckMessage) msg);
+                }
+            });
+
+            // First coordinator initialization.
+            ctx.discovery().localJoinFuture().listen(f -> {
+                this.coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE);
+            });
+
+            ctx.event().addDiscoveryEventListener((e, cache) -> {
+                assert e != null;
+                assert e.type() == EVT_NODE_LEFT || e.type() == EVT_NODE_FAILED : this;
+
+                // Do not process from discovery thread.
+                ctx.closure().runLocalSafe(() -> processNodeLeft(e.eventNode()));
+            }, EVT_NODE_LEFT, EVT_NODE_FAILED);
+        }
+    }
+
+    /**
+     * Creates server latch with given {@code id} and {@code topVer}.
+     * Adds corresponding pending acks to it.
+     *
+     * @param id Latch id.
+     * @param topVer Latch topology version.
+     * @param participants Participant nodes.
+     * @return Server latch instance.
+     */
+    private Latch createServerLatch(String id, AffinityTopologyVersion topVer, Collection<ClusterNode> participants) {
+        final T2<String, AffinityTopologyVersion> latchId = new T2<>(id, topVer);
+
+        if (serverLatches.containsKey(latchId))
+            return serverLatches.get(latchId);
+
+        ServerLatch latch = new ServerLatch(id, topVer, participants);
+
+        serverLatches.put(latchId, latch);
+
+        if (log.isDebugEnabled())
+            log.debug("Server latch is created [latch=" + latchId + ", participantsSize=" + participants.size() + "]");
+
+        if (pendingAcks.containsKey(latchId)) {
+            Set<UUID> acks = pendingAcks.get(latchId);
+
+            for (UUID node : acks)
+                if (latch.hasParticipant(node) && !latch.hasAck(node))
+                    latch.ack(node);
+
+            pendingAcks.remove(latchId);
+        }
+
+        if (latch.isCompleted())
+            serverLatches.remove(latchId);
+
+        return latch;
+    }
+
+    /**
+     * Creates client latch.
+     * If there is final ack corresponds to given {@code id} and {@code topVer}, latch will be completed immediately.
+     *
+     * @param id Latch id.
+     * @param topVer Latch topology version.
+     * @param coordinator Coordinator node.
+     * @param participants Participant nodes.
+     * @return Client latch instance.
+     */
+    private Latch createClientLatch(String id, AffinityTopologyVersion topVer, ClusterNode coordinator, Collection<ClusterNode> participants) {
+        final T2<String, AffinityTopologyVersion> latchId = new T2<>(id, topVer);
+
+        if (clientLatches.containsKey(latchId))
+            return clientLatches.get(latchId);
+
+        ClientLatch latch = new ClientLatch(id, topVer, coordinator, participants);
+
+        if (log.isDebugEnabled())
+            log.debug("Client latch is created [latch=" + latchId
+                    + ", crd=" + coordinator
+                    + ", participantsSize=" + participants.size() + "]");
+
+        // There is final ack for created latch.
+        if (pendingAcks.containsKey(latchId)) {
+            latch.complete();
+            pendingAcks.remove(latchId);
+        }
+        else
+            clientLatches.put(latchId, latch);
+
+        return latch;
+    }
+
+    /**
+     * Creates new latch with specified {@code id} and {@code topVer} or returns existing latch.
+     *
+     * Participants of latch are calculated from given {@code topVer} as alive server nodes.
+     * If local node is coordinator {@code ServerLatch} instance will be created, otherwise {@code ClientLatch} instance.
+     *
+     * @param id Latch id.
+     * @param topVer Latch topology version.
+     * @return Latch instance.
+     */
+    public Latch getOrCreate(String id, AffinityTopologyVersion topVer) {
+        lock.lock();
+
+        try {
+            ClusterNode coordinator = getLatchCoordinator(topVer);
+
+            if (coordinator == null) {
+                ClientLatch latch = new ClientLatch(id, AffinityTopologyVersion.NONE, null, Collections.emptyList());
+                latch.complete();
+
+                return latch;
+            }
+
+            Collection<ClusterNode> participants = getLatchParticipants(topVer);
+
+            return coordinator.isLocal()
+                ? createServerLatch(id, topVer, participants)
+                : createClientLatch(id, topVer, coordinator, participants);
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * @param topVer Latch topology version.
+     * @return Collection of alive server nodes with latch functionality.
+     */
+    private Collection<ClusterNode> getLatchParticipants(AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> aliveNodes = topVer == AffinityTopologyVersion.NONE
+                ? discovery.aliveServerNodes()
+                : discovery.discoCache(topVer).aliveServerNodes();
+
+        return aliveNodes
+                .stream()
+                .filter(node -> node.version().compareTo(VERSION_SINCE) >= 0)
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * @param topVer Latch topology version.
+     * @return Oldest alive server node with latch functionality.
+     */
+    @Nullable private ClusterNode getLatchCoordinator(AffinityTopologyVersion topVer) {
+        Collection<ClusterNode> aliveNodes = topVer == AffinityTopologyVersion.NONE
+                ? discovery.aliveServerNodes()
+                : discovery.discoCache(topVer).aliveServerNodes();
+
+        return aliveNodes
+                .stream()
+                .filter(node -> node.version().compareTo(VERSION_SINCE) >= 0)
+                .findFirst()
+                .orElse(null);
+    }
+
+    /**
+     * Processes ack message from given {@code from} node.
+     *
+     * Completes client latch in case of final ack message.
+     *
+     * If no latch is associated with message, ack is placed to {@link #pendingAcks} set.
+     *
+     * @param from Node sent ack.
+     * @param message Ack message.
+     */
+    private void processAck(UUID from, LatchAckMessage message) {
+        lock.lock();
+
+        try {
+            ClusterNode coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE);
+
+            if (coordinator == null)
+                return;
+
+            T2<String, AffinityTopologyVersion> latchId = new T2<>(message.latchId(), message.topVer());
+
+            if (message.isFinal()) {
+                if (log.isDebugEnabled())
+                    log.debug("Process final ack [latch=" + latchId + ", from=" + from + "]");
+
+                if (clientLatches.containsKey(latchId)) {
+                    ClientLatch latch = clientLatches.remove(latchId);
+                    latch.complete();
+                }
+                else if (!coordinator.isLocal()) {
+                    pendingAcks.computeIfAbsent(latchId, (id) -> new GridConcurrentHashSet<>());
+                    pendingAcks.get(latchId).add(from);
+                }
+            } else {
+                if (log.isDebugEnabled())
+                    log.debug("Process ack [latch=" + latchId + ", from=" + from + "]");
+
+                if (serverLatches.containsKey(latchId)) {
+                    ServerLatch latch = serverLatches.get(latchId);
+
+                    if (latch.hasParticipant(from) && !latch.hasAck(from)) {
+                        latch.ack(from);
+
+                        if (latch.isCompleted())
+                            serverLatches.remove(latchId);
+                    }
+                }
+                else {
+                    pendingAcks.computeIfAbsent(latchId, (id) -> new GridConcurrentHashSet<>());
+                    pendingAcks.get(latchId).add(from);
+                }
+            }
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Changes coordinator to current local node.
+     * Restores all server latches from pending acks and own client latches.
+     */
+    private void becomeNewCoordinator() {
+        if (log.isInfoEnabled())
+            log.info("Become new coordinator " + coordinator.id());
+
+        List<T2<String, AffinityTopologyVersion>> latchesToRestore = new ArrayList<>();
+        latchesToRestore.addAll(pendingAcks.keySet());
+        latchesToRestore.addAll(clientLatches.keySet());
+
+        for (T2<String, AffinityTopologyVersion> latchId : latchesToRestore) {
+            String id = latchId.get1();
+            AffinityTopologyVersion topVer = latchId.get2();
+            Collection<ClusterNode> participants = getLatchParticipants(topVer);
+
+            if (!participants.isEmpty())
+                createServerLatch(id, topVer, participants);
+        }
+    }
+
+    /**
+     * Handles node left discovery event.
+     *
+     * Summary:
+     * Removes pending acks corresponds to the left node.
+     * Adds fake acknowledgements to server latches where such node was participant.
+     * Changes client latches coordinator to oldest available server node where such node was coordinator.
+     * Detects coordinator change.
+     *
+     * @param left Left node.
+     */
+    private void processNodeLeft(ClusterNode left) {
+        assert this.coordinator != null : "Coordinator is not initialized";
+
+        lock.lock();
+
+        try {
+            if (log.isDebugEnabled())
+                log.debug("Process node left " + left.id());
+
+            ClusterNode coordinator = getLatchCoordinator(AffinityTopologyVersion.NONE);
+
+            if (coordinator == null)
+                return;
+
+            // Clear pending acks.
+            for (Map.Entry<T2<String, AffinityTopologyVersion>, Set<UUID>> ackEntry : pendingAcks.entrySet())
+                if (ackEntry.getValue().contains(left.id()))
+                    pendingAcks.get(ackEntry.getKey()).remove(left.id());
+
+            // Change coordinator for client latches.
+            for (Map.Entry<T2<String, AffinityTopologyVersion>, ClientLatch> latchEntry : clientLatches.entrySet()) {
+                ClientLatch latch = latchEntry.getValue();
+                if (latch.hasCoordinator(left.id())) {
+                    // Change coordinator for latch and re-send ack if necessary.
+                    if (latch.hasParticipant(coordinator.id()))
+                        latch.newCoordinator(coordinator);
+                    else {
+                        /* If new coordinator is not able to take control on the latch,
+                           it means that all other latch participants are left from topology
+                           and there is no reason to track such latch. */
+                        AffinityTopologyVersion topVer = latchEntry.getKey().get2();
+
+                        assert getLatchParticipants(topVer).isEmpty();
+
+                        latch.complete(new IgniteCheckedException("All latch participants are left from topology."));
+                        clientLatches.remove(latchEntry.getKey());
+                    }
+                }
+            }
+
+            // Add acknowledgements from left node.
+            for (Map.Entry<T2<String, AffinityTopologyVersion>, ServerLatch> latchEntry : serverLatches.entrySet()) {
+                ServerLatch latch = latchEntry.getValue();
+
+                if (latch.hasParticipant(left.id()) && !latch.hasAck(left.id())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Process node left [latch=" + latchEntry.getKey() + ", left=" + left.id() + "]");
+
+                    latch.ack(left.id());
+
+                    if (latch.isCompleted())
+                        serverLatches.remove(latchEntry.getKey());
+                }
+            }
+
+            // Coordinator is changed.
+            if (coordinator.isLocal() && this.coordinator.id() != coordinator.id()) {
+                this.coordinator = coordinator;
+
+                becomeNewCoordinator();
+            }
+        }
+        finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Latch creating on coordinator node.
+     * Latch collects acks from participants: non-coordinator nodes and current local node.
+     * Latch completes when all acks from all participants are received.
+     *
+     * After latch completion final ack is sent to all participants.
+     */
+    class ServerLatch extends CompletableLatch {
+        /** Number of latch permits. This is needed to track number of countDown invocations. */
+        private final AtomicInteger permits;
+
+        /** Set of received acks. */
+        private final Set<UUID> acks = new GridConcurrentHashSet<>();
+
+        /**
+         * Constructor.
+         *
+         * @param id Latch id.
+         * @param topVer Latch topology version.
+         * @param participants Participant nodes.
+         */
+        ServerLatch(String id, AffinityTopologyVersion topVer, Collection<ClusterNode> participants) {
+            super(id, topVer, participants);
+            this.permits = new AtomicInteger(participants.size());
+
+            // Send final acks when latch is completed.
+            this.complete.listen(f -> {
+                for (ClusterNode node : participants) {
+                    try {
+                        if (discovery.alive(node)) {
+                            io.sendToGridTopic(node, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, true), GridIoPolicy.SYSTEM_POOL);
+
+                            if (log.isDebugEnabled())
+                                log.debug("Final ack is ackSent [latch=" + latchId() + ", to=" + node.id() + "]");
+                        }
+                    } catch (IgniteCheckedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to send final ack [latch=" + latchId() + ", to=" + node.id() + "]");
+                    }
+                }
+            });
+        }
+
+        /**
+         * Checks if latch has ack from given node.
+         *
+         * @param from Node.
+         * @return {@code true} if latch has ack from given node.
+         */
+        private boolean hasAck(UUID from) {
+            return acks.contains(from);
+        }
+
+        /**
+         * Receives ack from given node.
+         * Count downs latch if ack was not already processed.
+         *
+         * @param from Node.
+         */
+        private void ack(UUID from) {
+            if (log.isDebugEnabled())
+                log.debug("Ack is accepted [latch=" + latchId() + ", from=" + from + "]");
+
+            countDown0(from);
+        }
+
+        /**
+         * Count down latch from ack of given node.
+         * Completes latch if all acks are received.
+         *
+         * @param node Node.
+         */
+        private void countDown0(UUID node) {
+            if (isCompleted() || acks.contains(node))
+                return;
+
+            acks.add(node);
+
+            int remaining = permits.decrementAndGet();
+
+            if (log.isDebugEnabled())
+                log.debug("Count down + [latch=" + latchId() + ", remaining=" + remaining + "]");
+
+            if (remaining == 0)
+                complete();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void countDown() {
+            countDown0(ctx.localNodeId());
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            Set<UUID> pendingAcks = participants.stream().filter(ack -> !acks.contains(ack)).collect(Collectors.toSet());
+
+            return S.toString(ServerLatch.class, this,
+                    "pendingAcks", pendingAcks,
+                    "super", super.toString());
+        }
+    }
+
+    /**
+     * Latch creating on non-coordinator node.
+     * Latch completes when final ack from coordinator is received.
+     */
+    class ClientLatch extends CompletableLatch {
+        /** Latch coordinator node. Can be changed if coordinator is left from topology. */
+        private volatile ClusterNode coordinator;
+
+        /** Flag indicates that ack is sent to coordinator. */
+        private boolean ackSent;
+
+        /**
+         * Constructor.
+         *
+         * @param id Latch id.
+         * @param topVer Latch topology version.
+         * @param coordinator Coordinator node.
+         * @param participants Participant nodes.
+         */
+        ClientLatch(String id, AffinityTopologyVersion topVer, ClusterNode coordinator, Collection<ClusterNode> participants) {
+            super(id, topVer, participants);
+
+            this.coordinator = coordinator;
+        }
+
+        /**
+         * Checks if latch coordinator is given {@code node}.
+         *
+         * @param node Node.
+         * @return {@code true} if latch coordinator is given node.
+         */
+        private boolean hasCoordinator(UUID node) {
+            return coordinator.id().equals(node);
+        }
+
+        /**
+         * Changes coordinator of latch and resends ack to new coordinator if needed.
+         *
+         * @param coordinator New coordinator.
+         */
+        private void newCoordinator(ClusterNode coordinator) {
+            if (log.isDebugEnabled())
+                log.debug("Coordinator is changed [latch=" + latchId() + ", crd=" + coordinator.id() + "]");
+
+            synchronized (this) {
+                this.coordinator = coordinator;
+
+                // Resend ack to new coordinator.
+                if (ackSent)
+                    sendAck();
+            }
+        }
+
+        /**
+         * Sends ack to coordinator node.
+         * There is ack deduplication on coordinator. So it's fine to send same ack twice.
+         */
+        private void sendAck() {
+            try {
+                ackSent = true;
+
+                io.sendToGridTopic(coordinator, GridTopic.TOPIC_EXCHANGE, new LatchAckMessage(id, topVer, false), GridIoPolicy.SYSTEM_POOL);
+
+                if (log.isDebugEnabled())
+                    log.debug("Ack is ackSent + [latch=" + latchId() + ", to=" + coordinator.id() + "]");
+            } catch (IgniteCheckedException e) {
+                // Coordinator is unreachable. On coodinator node left discovery event ack will be resent.
+                if (log.isDebugEnabled())
+                    log.debug("Unable to send ack [latch=" + latchId() + ", to=" + coordinator.id() + "]: " + e.getMessage());
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void countDown() {
+            if (isCompleted())
+                return;
+
+            // Synchronize in case of changed coordinator.
+            synchronized (this) {
+                sendAck();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ClientLatch.class, this,
+                    "super", super.toString());
+        }
+    }
+
+    /**
+     * Base latch functionality with implemented complete / await logic.
+     */
+    private abstract static class CompletableLatch implements Latch {
+        /** Latch id. */
+        @GridToStringInclude
+        protected final String id;
+
+        /** Latch topology version. */
+        @GridToStringInclude
+        protected final AffinityTopologyVersion topVer;
+
+        /** Latch node participants. Only participant nodes are able to change state of latch. */
+        @GridToStringExclude
+        protected final Set<UUID> participants;
+
+        /** Future indicates that latch is completed. */
+        @GridToStringExclude
+        protected final GridFutureAdapter<?> complete = new GridFutureAdapter<>();
+
+        /**
+         * Constructor.
+         *
+         * @param id Latch id.
+         * @param topVer Latch topology version.
+         * @param participants Participant nodes.
+         */
+        CompletableLatch(String id, AffinityTopologyVersion topVer, Collection<ClusterNode> participants) {
+            this.id = id;
+            this.topVer = topVer;
+            this.participants = participants.stream().map(ClusterNode::id).collect(Collectors.toSet());
+        }
+
+        /** {@inheritDoc} */
+        @Override public void await() throws IgniteCheckedException {
+            complete.get();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void await(long timeout, TimeUnit timeUnit) throws IgniteCheckedException {
+            complete.get(timeout, timeUnit);
+        }
+
+        /**
+         * Checks if latch participants contain given {@code node}.
+         *
+         * @param node Node.
+         * @return {@code true} if latch participants contain given node.
+         */
+        boolean hasParticipant(UUID node) {
+            return participants.contains(node);
+        }
+
+        /**
+         * @return {@code true} if latch is completed.
+         */
+        boolean isCompleted() {
+            return complete.isDone();
+        }
+
+        /**
+         * Completes current latch.
+         */
+        void complete() {
+            complete.onDone();
+        }
+
+        /**
+         * Completes current latch with given {@code error}.
+         *
+         * @param error Error.
+         */
+        void complete(Throwable error) {
+            complete.onDone(error);
+        }
+
+        /**
+         * @return Full latch id.
+         */
+        String latchId() {
+            return id + "-" + topVer;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(CompletableLatch.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/da77b981/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java
new file mode 100644
index 0000000..9704c2e
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/latch/Latch.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Simple distributed count down latch interface.
+ * Latch supports count down and await logic.
+ * Latch functionality is not relied on caches and has own state management {@link ExchangeLatchManager}.
+ */
+public interface Latch {
+    /**
+     * Decrements count on current latch.
+     * Release all latch waiters on all nodes if count reaches zero.
+     *
+     * This is idempotent operation. Invoking this method twice or more on the same node doesn't have any effect.
+     */
+    void countDown();
+
+    /**
+     * Awaits current latch completion.
+     *
+     * @throws IgniteCheckedException If await is failed.
+     */
+    void await() throws IgniteCheckedException;
+
+    /**
+     * Awaits current latch completion with specified timeout.
+     *
+     * @param timeout Timeout value.
+     * @param timeUnit Timeout time unit.
+     * @throws IgniteCheckedException If await is failed.
+     */
+    void await(long timeout, TimeUnit timeUnit) throws IgniteCheckedException;
+}


[26/50] [abbrv] ignite git commit: IGNITE-8141 Acknowledge SWAPPINESS reduction using a range (<= 10). - Fixes #3727.

Posted by ag...@apache.org.
IGNITE-8141 Acknowledge SWAPPINESS reduction using a range (<= 10). - Fixes #3727.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 47ca3e0da7fd7468d7ea96ff512c3d1f3ed5bf81
Parents: b93595c
Author: Reed Sandberg <re...@drawbridge.com>
Authored: Thu Apr 12 20:18:10 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Thu Apr 12 20:18:10 2018 +0300

----------------------------------------------------------------------
 .../suggestions/OsConfigurationSuggestions.java       | 14 ++++++++++++--
 1 file changed, 12 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/47ca3e0d/modules/core/src/main/java/org/apache/ignite/internal/suggestions/OsConfigurationSuggestions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/suggestions/OsConfigurationSuggestions.java b/modules/core/src/main/java/org/apache/ignite/internal/suggestions/OsConfigurationSuggestions.java
index e5b4c12..695b423 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/suggestions/OsConfigurationSuggestions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/suggestions/OsConfigurationSuggestions.java
@@ -76,8 +76,18 @@ public class OsConfigurationSuggestions {
                     (dwcParamFlag && decParamFlag ? "s" : ""),
                     expected));
 
-            if ((value = readVmParam(SWAPPINESS)) != null && !value.equals(expected = "10"))
-                suggestions.add(String.format("Reduce pages swapping ratio (set vm.%s=%s)", SWAPPINESS, expected));
+            if ((value = readVmParam(SWAPPINESS)) != null) {
+                try {
+                    double maxSwappiness = 10.0;
+
+                    if (Float.parseFloat(value) > maxSwappiness)
+                        suggestions.add(String.format("Reduce pages swapping ratio (set vm.%s=%f or less)", SWAPPINESS,
+                                                      maxSwappiness));
+                }
+                catch (NumberFormatException ignored) {
+                    // OS param not parsable as a number
+                }
+            }
 
             if ((value = readVmParam(ZONE_RECLAIM_MODE)) != null && !value.equals(expected = "0"))
                 suggestions.add(String.format("Disable NUMA memory reclaim (set vm.%s=%s)", ZONE_RECLAIM_MODE,


[32/50] [abbrv] ignite git commit: IGNITE-7829: Adopt kNN regression example to the new Partitioned Dataset

Posted by ag...@apache.org.
IGNITE-7829: Adopt kNN regression example to the new Partitioned Dataset

this closes #3798


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

Branch: refs/heads/ignite-7708
Commit: 8550d61b6b39625579eb7f69f4d1218b78f7cc5b
Parents: 9be3357
Author: zaleslaw <za...@gmail.com>
Authored: Fri Apr 13 12:49:56 2018 +0300
Committer: YuriBabak <y....@gmail.com>
Committed: Fri Apr 13 12:49:56 2018 +0300

----------------------------------------------------------------------
 .../ml/knn/KNNClassificationExample.java        |   4 +-
 .../examples/ml/knn/KNNRegressionExample.java   | 310 +++++++++++++++++++
 .../java/org/apache/ignite/ml/knn/KNNUtils.java |  10 +-
 .../classification/KNNClassificationModel.java  |   9 +-
 .../ml/knn/partitions/KNNPartitionContext.java  |  28 --
 .../ignite/ml/knn/partitions/package-info.java  |  22 --
 .../ml/knn/regression/KNNRegressionModel.java   |   7 +-
 .../partition/LabelPartitionContext.java        |  28 --
 .../LabelPartitionDataBuilderOnHeap.java        |   1 -
 .../svm/SVMLinearBinaryClassificationModel.java |   3 +
 .../SVMLinearBinaryClassificationTrainer.java   |   9 +-
 .../SVMLinearMultiClassClassificationModel.java |   3 +
 ...VMLinearMultiClassClassificationTrainer.java |   8 +-
 .../ignite/ml/svm/SVMPartitionContext.java      |  28 --
 .../org/apache/ignite/ml/knn/BaseKNNTest.java   |  89 ------
 .../ignite/ml/knn/KNNClassificationTest.java    | 110 +++----
 .../apache/ignite/ml/knn/KNNRegressionTest.java | 104 +++----
 .../ignite/ml/knn/LabeledDatasetHelper.java     |  87 ++++++
 .../ignite/ml/knn/LabeledDatasetTest.java       |   2 +-
 19 files changed, 536 insertions(+), 326 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
index 39a8431..15375a1 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
@@ -80,7 +80,7 @@ public class KNNClassificationExample {
                         double prediction = knnMdl.apply(new DenseLocalOnHeapVector(inputs));
 
                         totalAmount++;
-                        if(groundTruth != prediction)
+                        if (groundTruth != prediction)
                             amountOfErrors++;
 
                         System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
@@ -89,7 +89,7 @@ public class KNNClassificationExample {
                     System.out.println(">>> ---------------------------------");
 
                     System.out.println("\n>>> Absolute amount of errors " + amountOfErrors);
-                    System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double)totalAmount));
+                    System.out.println("\n>>> Accuracy " + (1 - amountOfErrors / (double) totalAmount));
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java
new file mode 100644
index 0000000..76a07cd
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNRegressionExample.java
@@ -0,0 +1,310 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.ml.knn;
+
+import java.util.Arrays;
+import java.util.UUID;
+import javax.cache.Cache;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
+import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer;
+import org.apache.ignite.ml.knn.classification.KNNStrategy;
+import org.apache.ignite.ml.knn.regression.KNNRegressionModel;
+import org.apache.ignite.ml.knn.regression.KNNRegressionTrainer;
+import org.apache.ignite.ml.math.distances.ManhattanDistance;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.thread.IgniteThread;
+
+/**
+ * Run kNN regression trainer over distributed dataset.
+ *
+ * @see KNNClassificationTrainer
+ */
+public class KNNRegressionExample {
+    /** Run example. */
+    public static void main(String[] args) throws InterruptedException {
+        System.out.println();
+        System.out.println(">>> kNN regression over cached dataset usage example started.");
+        // Start ignite grid.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Ignite grid started.");
+
+            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
+                KNNRegressionExample.class.getSimpleName(), () -> {
+                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
+
+                KNNRegressionTrainer trainer = new KNNRegressionTrainer();
+
+                KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
+                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+                    (k, v) -> v[0]
+                ).withK(5)
+                    .withDistanceMeasure(new ManhattanDistance())
+                    .withStrategy(KNNStrategy.WEIGHTED);
+
+                int totalAmount = 0;
+                // Calculate mean squared error (MSE)
+                double mse = 0.0;
+                // Calculate mean absolute error (MAE)
+                double mae = 0.0;
+
+                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
+                    for (Cache.Entry<Integer, double[]> observation : observations) {
+                        double[] val = observation.getValue();
+                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
+                        double groundTruth = val[0];
+
+                        double prediction = knnMdl.apply(new DenseLocalOnHeapVector(inputs));
+
+                        mse += Math.pow(prediction - groundTruth, 2.0);
+                        mae += Math.abs(prediction - groundTruth);
+
+                        totalAmount++;
+                    }
+
+                    mse = mse / totalAmount;
+                    System.out.println("\n>>> Mean squared error (MSE) " + mse);
+
+                    mae = mae / totalAmount;
+                    System.out.println("\n>>> Mean absolute error (MAE) " + mae);
+                }
+            });
+
+            igniteThread.start();
+            igniteThread.join();
+        }
+    }
+
+    /**
+     * Fills cache with data and returns it.
+     *
+     * @param ignite Ignite instance.
+     * @return Filled Ignite Cache.
+     */
+    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
+        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
+        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
+
+        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
+
+        for (int i = 0; i < data.length; i++)
+            cache.put(i, data[i]);
+
+        return cache;
+    }
+
+    /** The Iris dataset. */
+    private static final double[][] data = {
+        {199, 125, 256, 6000, 256, 16, 128},
+        {253, 29, 8000, 32000, 32, 8, 32},
+        {132, 29, 8000, 16000, 32, 8, 16},
+        {290, 26, 8000, 32000, 64, 8, 32},
+        {381, 23, 16000, 32000, 64, 16, 32},
+        {749, 23, 16000, 64000, 64, 16, 32},
+        {1238, 23, 32000, 64000, 128, 32, 64},
+        {23, 400, 1000, 3000, 0, 1, 2},
+        {24, 400, 512, 3500, 4, 1, 6},
+        {70, 60, 2000, 8000, 65, 1, 8},
+        {117, 50, 4000, 16000, 65, 1, 8},
+        {15, 350, 64, 64, 0, 1, 4},
+        {64, 200, 512, 16000, 0, 4, 32},
+        {23, 167, 524, 2000, 8, 4, 15},
+        {29, 143, 512, 5000, 0, 7, 32},
+        {22, 143, 1000, 2000, 0, 5, 16},
+        {124, 110, 5000, 5000, 142, 8, 64},
+        {35, 143, 1500, 6300, 0, 5, 32},
+        {39, 143, 3100, 6200, 0, 5, 20},
+        {40, 143, 2300, 6200, 0, 6, 64},
+        {45, 110, 3100, 6200, 0, 6, 64},
+        {28, 320, 128, 6000, 0, 1, 12},
+        {21, 320, 512, 2000, 4, 1, 3},
+        {28, 320, 256, 6000, 0, 1, 6},
+        {22, 320, 256, 3000, 4, 1, 3},
+        {28, 320, 512, 5000, 4, 1, 5},
+        {27, 320, 256, 5000, 4, 1, 6},
+        {102, 25, 1310, 2620, 131, 12, 24},
+        {74, 50, 2620, 10480, 30, 12, 24},
+        {138, 56, 5240, 20970, 30, 12, 24},
+        {136, 64, 5240, 20970, 30, 12, 24},
+        {23, 50, 500, 2000, 8, 1, 4},
+        {29, 50, 1000, 4000, 8, 1, 5},
+        {44, 50, 2000, 8000, 8, 1, 5},
+        {30, 50, 1000, 4000, 8, 3, 5},
+        {41, 50, 1000, 8000, 8, 3, 5},
+        {74, 50, 2000, 16000, 8, 3, 5},
+        {54, 133, 1000, 12000, 9, 3, 12},
+        {41, 133, 1000, 8000, 9, 3, 12},
+        {18, 810, 512, 512, 8, 1, 1},
+        {28, 810, 1000, 5000, 0, 1, 1},
+        {36, 320, 512, 8000, 4, 1, 5},
+        {38, 200, 512, 8000, 8, 1, 8},
+        {34, 700, 384, 8000, 0, 1, 1},
+        {19, 700, 256, 2000, 0, 1, 1},
+        {72, 140, 1000, 16000, 16, 1, 3},
+        {36, 200, 1000, 8000, 0, 1, 2},
+        {30, 110, 1000, 4000, 16, 1, 2},
+        {56, 110, 1000, 12000, 16, 1, 2},
+        {42, 220, 1000, 8000, 16, 1, 2},
+        {34, 800, 256, 8000, 0, 1, 4},
+        {19, 125, 512, 1000, 0, 8, 20},
+        {75, 75, 2000, 8000, 64, 1, 38},
+        {113, 75, 2000, 16000, 64, 1, 38},
+        {157, 75, 2000, 16000, 128, 1, 38},
+        {18, 90, 256, 1000, 0, 3, 10},
+        {20, 105, 256, 2000, 0, 3, 10},
+        {28, 105, 1000, 4000, 0, 3, 24},
+        {33, 105, 2000, 4000, 8, 3, 19},
+        {47, 75, 2000, 8000, 8, 3, 24},
+        {54, 75, 3000, 8000, 8, 3, 48},
+        {20, 175, 256, 2000, 0, 3, 24},
+        {23, 300, 768, 3000, 0, 6, 24},
+        {25, 300, 768, 3000, 6, 6, 24},
+        {52, 300, 768, 12000, 6, 6, 24},
+        {27, 300, 768, 4500, 0, 1, 24},
+        {50, 300, 384, 12000, 6, 1, 24},
+        {18, 300, 192, 768, 6, 6, 24},
+        {53, 180, 768, 12000, 6, 1, 31},
+        {23, 330, 1000, 3000, 0, 2, 4},
+        {30, 300, 1000, 4000, 8, 3, 64},
+        {73, 300, 1000, 16000, 8, 2, 112},
+        {20, 330, 1000, 2000, 0, 1, 2},
+        {25, 330, 1000, 4000, 0, 3, 6},
+        {28, 140, 2000, 4000, 0, 3, 6},
+        {29, 140, 2000, 4000, 0, 4, 8},
+        {32, 140, 2000, 4000, 8, 1, 20},
+        {175, 140, 2000, 32000, 32, 1, 20},
+        {57, 140, 2000, 8000, 32, 1, 54},
+        {181, 140, 2000, 32000, 32, 1, 54},
+        {32, 140, 2000, 4000, 8, 1, 20},
+        {82, 57, 4000, 16000, 1, 6, 12},
+        {171, 57, 4000, 24000, 64, 12, 16},
+        {361, 26, 16000, 32000, 64, 16, 24},
+        {350, 26, 16000, 32000, 64, 8, 24},
+        {220, 26, 8000, 32000, 0, 8, 24},
+        {113, 26, 8000, 16000, 0, 8, 16},
+        {15, 480, 96, 512, 0, 1, 1},
+        {21, 203, 1000, 2000, 0, 1, 5},
+        {35, 115, 512, 6000, 16, 1, 6},
+        {18, 1100, 512, 1500, 0, 1, 1},
+        {20, 1100, 768, 2000, 0, 1, 1},
+        {20, 600, 768, 2000, 0, 1, 1},
+        {28, 400, 2000, 4000, 0, 1, 1},
+        {45, 400, 4000, 8000, 0, 1, 1},
+        {18, 900, 1000, 1000, 0, 1, 2},
+        {17, 900, 512, 1000, 0, 1, 2},
+        {26, 900, 1000, 4000, 4, 1, 2},
+        {28, 900, 1000, 4000, 8, 1, 2},
+        {28, 900, 2000, 4000, 0, 3, 6},
+        {31, 225, 2000, 4000, 8, 3, 6},
+        {42, 180, 2000, 8000, 8, 1, 6},
+        {76, 185, 2000, 16000, 16, 1, 6},
+        {76, 180, 2000, 16000, 16, 1, 6},
+        {26, 225, 1000, 4000, 2, 3, 6},
+        {59, 25, 2000, 12000, 8, 1, 4},
+        {65, 25, 2000, 12000, 16, 3, 5},
+        {101, 17, 4000, 16000, 8, 6, 12},
+        {116, 17, 4000, 16000, 32, 6, 12},
+        {18, 1500, 768, 1000, 0, 0, 0},
+        {20, 1500, 768, 2000, 0, 0, 0},
+        {20, 800, 768, 2000, 0, 0, 0},
+        {30, 50, 2000, 4000, 0, 3, 6},
+        {44, 50, 2000, 8000, 8, 3, 6},
+        {82, 50, 2000, 16000, 24, 1, 6},
+        {128, 50, 8000, 16000, 48, 1, 10},
+        {37, 100, 1000, 8000, 0, 2, 6},
+        {46, 100, 1000, 8000, 24, 2, 6},
+        {46, 100, 1000, 8000, 24, 3, 6},
+        {80, 50, 2000, 16000, 12, 3, 16},
+        {88, 50, 2000, 16000, 24, 6, 16},
+        {33, 150, 512, 4000, 0, 8, 128},
+        {46, 115, 2000, 8000, 16, 1, 3},
+        {29, 115, 2000, 4000, 2, 1, 5},
+        {53, 92, 2000, 8000, 32, 1, 6},
+        {41, 92, 2000, 8000, 4, 1, 6},
+        {86, 75, 4000, 16000, 16, 1, 6},
+        {95, 60, 4000, 16000, 32, 1, 6},
+        {107, 60, 2000, 16000, 64, 5, 8},
+        {117, 60, 4000, 16000, 64, 5, 8},
+        {119, 50, 4000, 16000, 64, 5, 10},
+        {120, 72, 4000, 16000, 64, 8, 16},
+        {48, 72, 2000, 8000, 16, 6, 8},
+        {126, 40, 8000, 16000, 32, 8, 16},
+        {266, 40, 8000, 32000, 64, 8, 24},
+        {270, 35, 8000, 32000, 64, 8, 24},
+        {426, 38, 16000, 32000, 128, 16, 32},
+        {151, 48, 4000, 24000, 32, 8, 24},
+        {267, 38, 8000, 32000, 64, 8, 24},
+        {603, 30, 16000, 32000, 256, 16, 24},
+        {19, 112, 1000, 1000, 0, 1, 4},
+        {21, 84, 1000, 2000, 0, 1, 6},
+        {26, 56, 1000, 4000, 0, 1, 6},
+        {35, 56, 2000, 6000, 0, 1, 8},
+        {41, 56, 2000, 8000, 0, 1, 8},
+        {47, 56, 4000, 8000, 0, 1, 8},
+        {62, 56, 4000, 12000, 0, 1, 8},
+        {78, 56, 4000, 16000, 0, 1, 8},
+        {80, 38, 4000, 8000, 32, 16, 32},
+        {142, 38, 8000, 16000, 64, 4, 8},
+        {281, 38, 8000, 24000, 160, 4, 8},
+        {190, 38, 4000, 16000, 128, 16, 32},
+        {21, 200, 1000, 2000, 0, 1, 2},
+        {25, 200, 1000, 4000, 0, 1, 4},
+        {67, 200, 2000, 8000, 64, 1, 5},
+        {24, 250, 512, 4000, 0, 1, 7},
+        {24, 250, 512, 4000, 0, 4, 7},
+        {64, 250, 1000, 16000, 1, 1, 8},
+        {25, 160, 512, 4000, 2, 1, 5},
+        {20, 160, 512, 2000, 2, 3, 8},
+        {29, 160, 1000, 4000, 8, 1, 14},
+        {43, 160, 1000, 8000, 16, 1, 14},
+        {53, 160, 2000, 8000, 32, 1, 13},
+        {19, 240, 512, 1000, 8, 1, 3},
+        {22, 240, 512, 2000, 8, 1, 5},
+        {31, 105, 2000, 4000, 8, 3, 8},
+        {41, 105, 2000, 6000, 16, 6, 16},
+        {47, 105, 2000, 8000, 16, 4, 14},
+        {99, 52, 4000, 16000, 32, 4, 12},
+        {67, 70, 4000, 12000, 8, 6, 8},
+        {81, 59, 4000, 12000, 32, 6, 12},
+        {149, 59, 8000, 16000, 64, 12, 24},
+        {183, 26, 8000, 24000, 32, 8, 16},
+        {275, 26, 8000, 32000, 64, 12, 16},
+        {382, 26, 8000, 32000, 128, 24, 32},
+        {56, 116, 2000, 8000, 32, 5, 28},
+        {182, 50, 2000, 32000, 24, 6, 26},
+        {227, 50, 2000, 32000, 48, 26, 52},
+        {341, 50, 2000, 32000, 112, 52, 104},
+        {360, 50, 4000, 32000, 112, 52, 104},
+        {919, 30, 8000, 64000, 96, 12, 176},
+        {978, 30, 8000, 64000, 128, 12, 176},
+        {24, 180, 262, 4000, 0, 1, 3},
+        {37, 124, 1000, 8000, 0, 1, 8},
+        {50, 98, 1000, 8000, 32, 2, 8},
+        {41, 125, 2000, 8000, 0, 2, 14},
+        {47, 480, 512, 8000, 32, 0, 0},
+        {25, 480, 1000, 4000, 0, 0, 0}
+    };
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
index 88fa70f..716eb52 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/KNNUtils.java
@@ -20,7 +20,7 @@ package org.apache.ignite.ml.knn;
 import org.apache.ignite.ml.dataset.Dataset;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
-import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 import org.apache.ignite.ml.structures.LabeledDataset;
 import org.apache.ignite.ml.structures.LabeledVector;
@@ -39,18 +39,18 @@ public class KNNUtils {
      * @param lbExtractor Label extractor.
      * @return Dataset.
      */
-    @Nullable public static <K, V> Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> buildDataset(DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
-        PartitionDataBuilder<K, V, KNNPartitionContext, LabeledDataset<Double, LabeledVector>> partDataBuilder
+    @Nullable public static <K, V> Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> buildDataset(DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
+        PartitionDataBuilder<K, V, EmptyContext, LabeledDataset<Double, LabeledVector>> partDataBuilder
             = new LabeledDatasetPartitionDataBuilderOnHeap<>(
             featureExtractor,
             lbExtractor
         );
 
-        Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset = null;
+        Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset = null;
 
         if (datasetBuilder != null) {
             dataset = datasetBuilder.build(
-                (upstream, upstreamSize) -> new KNNPartitionContext(),
+                (upstream, upstreamSize) -> new EmptyContext(),
                 partDataBuilder
             );
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
index 373f822..693b81d 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
@@ -32,7 +32,7 @@ import org.apache.ignite.ml.Exportable;
 import org.apache.ignite.ml.Exporter;
 import org.apache.ignite.ml.Model;
 import org.apache.ignite.ml.dataset.Dataset;
-import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.distances.DistanceMeasure;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
@@ -44,6 +44,9 @@ import org.jetbrains.annotations.NotNull;
  * kNN algorithm model to solve multi-class classification task.
  */
 public class KNNClassificationModel<K, V> implements Model<Vector, Double>, Exportable<KNNModelFormat> {
+    /** */
+    private static final long serialVersionUID = -127386523291350345L;
+
     /** Amount of nearest neighbors. */
     protected int k = 5;
 
@@ -54,13 +57,13 @@ public class KNNClassificationModel<K, V> implements Model<Vector, Double>, Expo
     protected KNNStrategy stgy = KNNStrategy.SIMPLE;
 
     /** Dataset. */
-    private Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset;
+    private Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset;
 
     /**
      * Builds the model via prepared dataset.
      * @param dataset Specially prepared object to run algorithm over it.
      */
-    public KNNClassificationModel(Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset) {
+    public KNNClassificationModel(Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset) {
         this.dataset = dataset;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java
deleted file mode 100644
index 0081612..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/partitions/KNNPartitionContext.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.knn.partitions;
-
-import java.io.Serializable;
-
-/**
- * Partition context of the kNN classification algorithm.
- */
-public class KNNPartitionContext implements Serializable {
-    /** */
-    private static final long serialVersionUID = -7212307112344430126L;
-}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
index cabc143..f5def43 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/regression/KNNRegressionModel.java
@@ -17,8 +17,8 @@
 package org.apache.ignite.ml.knn.regression;
 
 import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
 import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
-import org.apache.ignite.ml.knn.partitions.KNNPartitionContext;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException;
 import org.apache.ignite.ml.structures.LabeledDataset;
@@ -38,11 +38,14 @@ import java.util.List;
  * </ul>
  */
 public class KNNRegressionModel<K,V> extends KNNClassificationModel<K,V> {
+    /** */
+    private static final long serialVersionUID = -721836321291120543L;
+
     /**
      * Builds the model via prepared dataset.
      * @param dataset Specially prepared object to run algorithm over it.
      */
-    public KNNRegressionModel(Dataset<KNNPartitionContext, LabeledDataset<Double, LabeledVector>> dataset) {
+    public KNNRegressionModel(Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset) {
         super(dataset);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java
deleted file mode 100644
index 1069ff8..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionContext.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.structures.partition;
-
-import java.io.Serializable;
-
-/**
- * Base partition context.
- */
-public class LabelPartitionContext implements Serializable {
-    /** */
-    private static final long serialVersionUID = -7412302212344430126L;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java
index 14c053e..4fba028 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/partition/LabelPartitionDataBuilderOnHeap.java
@@ -22,7 +22,6 @@ import java.util.Iterator;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
 import org.apache.ignite.ml.dataset.UpstreamEntry;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.structures.LabeledDataset;
 
 /**
  * Partition data builder that builds {@link LabelPartitionDataOnHeap}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java
index dace8c6..f806fb8 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationModel.java
@@ -28,6 +28,9 @@ import org.apache.ignite.ml.math.Vector;
  * Base class for SVM linear classification model.
  */
 public class SVMLinearBinaryClassificationModel implements Model<Vector, Double>, Exportable<SVMLinearBinaryClassificationModel>, Serializable {
+    /** */
+    private static final long serialVersionUID = -996984622291440226L;
+
     /** Output label format. -1 and +1 for false value and raw distances from the separating hyperplane otherwise. */
     private boolean isKeepingRawLabels = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java
index 7f11e20..d56848c 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearBinaryClassificationTrainer.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.ml.svm;
 
 import java.util.concurrent.ThreadLocalRandom;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
 import org.apache.ignite.ml.structures.partition.LabeledDatasetPartitionDataBuilderOnHeap;
 import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
 import org.apache.ignite.ml.dataset.Dataset;
@@ -59,15 +60,15 @@ public class SVMLinearBinaryClassificationTrainer implements SingleLabelDatasetT
 
         assert datasetBuilder != null;
 
-        PartitionDataBuilder<K, V, SVMPartitionContext, LabeledDataset<Double, LabeledVector>> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>(
+        PartitionDataBuilder<K, V, EmptyContext, LabeledDataset<Double, LabeledVector>> partDataBuilder = new LabeledDatasetPartitionDataBuilderOnHeap<>(
             featureExtractor,
             lbExtractor
         );
 
         Vector weights;
 
-        try(Dataset<SVMPartitionContext, LabeledDataset<Double, LabeledVector>> dataset = datasetBuilder.build(
-            (upstream, upstreamSize) -> new SVMPartitionContext(),
+        try(Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset = datasetBuilder.build(
+            (upstream, upstreamSize) -> new EmptyContext(),
             partDataBuilder
         )) {
             final int cols = dataset.compute(data -> data.colSize(), (a, b) -> a == null ? b : a);
@@ -90,7 +91,7 @@ public class SVMLinearBinaryClassificationTrainer implements SingleLabelDatasetT
     }
 
     /** */
-    private Vector calculateUpdates(Vector weights, Dataset<SVMPartitionContext, LabeledDataset<Double, LabeledVector>> dataset) {
+    private Vector calculateUpdates(Vector weights, Dataset<EmptyContext, LabeledDataset<Double, LabeledVector>> dataset) {
         return dataset.compute(data -> {
             Vector copiedWeights = weights.copy();
             Vector deltaWeights = initializeWeightsWithZeros(weights.size());

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java
index 5879ef0..bbec791 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationModel.java
@@ -29,6 +29,9 @@ import org.apache.ignite.ml.math.Vector;
 
 /** Base class for multi-classification model for set of SVM classifiers. */
 public class SVMLinearMultiClassClassificationModel implements Model<Vector, Double>, Exportable<SVMLinearMultiClassClassificationModel>, Serializable {
+    /** */
+    private static final long serialVersionUID = -667986511191350227L;
+
     /** List of models associated with each class. */
     private Map<Double, SVMLinearBinaryClassificationModel> models;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java
index 88c342d..4e081c6 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMLinearMultiClassClassificationTrainer.java
@@ -24,12 +24,12 @@ import java.util.List;
 import java.util.Set;
 import java.util.stream.Collectors;
 import java.util.stream.Stream;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
 import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
 import org.apache.ignite.ml.dataset.Dataset;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.structures.partition.LabelPartitionContext;
 import org.apache.ignite.ml.structures.partition.LabelPartitionDataBuilderOnHeap;
 import org.apache.ignite.ml.structures.partition.LabelPartitionDataOnHeap;
 
@@ -89,12 +89,12 @@ public class SVMLinearMultiClassClassificationTrainer
     private <K, V> List<Double> extractClassLabels(DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, Double> lbExtractor) {
         assert datasetBuilder != null;
 
-        PartitionDataBuilder<K, V, LabelPartitionContext, LabelPartitionDataOnHeap> partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor);
+        PartitionDataBuilder<K, V, EmptyContext, LabelPartitionDataOnHeap> partDataBuilder = new LabelPartitionDataBuilderOnHeap<>(lbExtractor);
 
         List<Double> res = new ArrayList<>();
 
-        try (Dataset<LabelPartitionContext, LabelPartitionDataOnHeap> dataset = datasetBuilder.build(
-            (upstream, upstreamSize) -> new LabelPartitionContext(),
+        try (Dataset<EmptyContext, LabelPartitionDataOnHeap> dataset = datasetBuilder.build(
+            (upstream, upstreamSize) -> new EmptyContext(),
             partDataBuilder
         )) {
             final Set<Double> clsLabels = dataset.compute(data -> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java
deleted file mode 100644
index 0aee0fb..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/svm/SVMPartitionContext.java
+++ /dev/null
@@ -1,28 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.svm;
-
-import java.io.Serializable;
-
-/**
- * Partition context of the SVM classification algorithm.
- */
-public class SVMPartitionContext implements Serializable {
-    /** */
-    private static final long serialVersionUID = -7212307112344430126L;
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java
deleted file mode 100644
index aeac2cf..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/BaseKNNTest.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.knn;
-
-import java.io.IOException;
-import java.net.URISyntaxException;
-import java.nio.file.Path;
-import java.nio.file.Paths;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.ml.structures.LabeledDataset;
-import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Base class for decision trees test.
- */
-public class BaseKNNTest extends GridCommonAbstractTest {
-    /** Count of nodes. */
-    private static final int NODE_COUNT = 4;
-
-    /** Separator. */
-    private static final String SEPARATOR = "\t";
-
-    /** Grid instance. */
-    protected Ignite ignite;
-
-    /**
-     * Default constructor.
-     */
-    public BaseKNNTest() {
-        super(false);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * Loads labeled dataset from file with .txt extension.
-     *
-     * @param rsrcPath path to dataset.
-     * @return null if path is incorrect.
-     */
-    LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) {
-        try {
-            Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI());
-            try {
-                return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData);
-            }
-            catch (IOException e) {
-                e.printStackTrace();
-            }
-        }
-        catch (URISyntaxException e) {
-            e.printStackTrace();
-            return null;
-        }
-        return null;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
index b27fcba..0877fc0 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
@@ -17,31 +17,35 @@
 
 package org.apache.ignite.ml.knn;
 
-import org.apache.ignite.internal.util.IgniteUtils;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Assert;
+import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
 import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
+import org.junit.Test;
 
 /** Tests behaviour of KNNClassificationTest. */
-public class KNNClassificationTest extends BaseKNNTest {
+public class KNNClassificationTest {
+    /** Precision in test checks. */
+    private static final double PRECISION = 1e-2;
+
     /** */
-    public void testBinaryClassificationTest() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
+    @Test
+    public void binaryClassificationTest() {
 
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {1.0, 1.0, 1.0});
-        data.put(1, new double[] {1.0, 2.0, 1.0});
-        data.put(2, new double[] {2.0, 1.0, 1.0});
-        data.put(3, new double[] {-1.0, -1.0, 2.0});
-        data.put(4, new double[] {-1.0, -2.0, 2.0});
-        data.put(5, new double[] {-2.0, -1.0, 2.0});
+        data.put(0, new double[]{1.0, 1.0, 1.0});
+        data.put(1, new double[]{1.0, 2.0, 1.0});
+        data.put(2, new double[]{2.0, 1.0, 1.0});
+        data.put(3, new double[]{-1.0, -1.0, 2.0});
+        data.put(4, new double[]{-1.0, -2.0, 2.0});
+        data.put(5, new double[]{-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
@@ -54,23 +58,23 @@ public class KNNClassificationTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0});
-        assertEquals(knnMdl.apply(firstVector), 1.0);
-        Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0});
-        assertEquals(knnMdl.apply(secondVector), 2.0);
+        Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0});
+        Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION);
+        Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0});
+        Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION);
     }
 
     /** */
-    public void testBinaryClassificationWithSmallestKTest() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
+    @Test
+    public void binaryClassificationWithSmallestKTest() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {1.0, 1.0, 1.0});
-        data.put(1, new double[] {1.0, 2.0, 1.0});
-        data.put(2, new double[] {2.0, 1.0, 1.0});
-        data.put(3, new double[] {-1.0, -1.0, 2.0});
-        data.put(4, new double[] {-1.0, -2.0, 2.0});
-        data.put(5, new double[] {-2.0, -1.0, 2.0});
+
+        data.put(0, new double[]{1.0, 1.0, 1.0});
+        data.put(1, new double[]{1.0, 2.0, 1.0});
+        data.put(2, new double[]{2.0, 1.0, 1.0});
+        data.put(3, new double[]{-1.0, -1.0, 2.0});
+        data.put(4, new double[]{-1.0, -2.0, 2.0});
+        data.put(5, new double[]{-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
@@ -83,23 +87,23 @@ public class KNNClassificationTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0});
-        assertEquals(knnMdl.apply(firstVector), 1.0);
-        Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0});
-        assertEquals(knnMdl.apply(secondVector), 2.0);
+        Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0});
+        Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION);
+        Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0});
+        Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION);
     }
 
     /** */
-    public void testBinaryClassificationFarPointsWithSimpleStrategy() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
+    @Test
+    public void binaryClassificationFarPointsWithSimpleStrategy() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {10.0, 10.0, 1.0});
-        data.put(1, new double[] {10.0, 20.0, 1.0});
-        data.put(2, new double[] {-1, -1, 1.0});
-        data.put(3, new double[] {-2, -2, 2.0});
-        data.put(4, new double[] {-1.0, -2.0, 2.0});
-        data.put(5, new double[] {-2.0, -1.0, 2.0});
+
+        data.put(0, new double[]{10.0, 10.0, 1.0});
+        data.put(1, new double[]{10.0, 20.0, 1.0});
+        data.put(2, new double[]{-1, -1, 1.0});
+        data.put(3, new double[]{-2, -2, 2.0});
+        data.put(4, new double[]{-1.0, -2.0, 2.0});
+        data.put(5, new double[]{-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
@@ -112,21 +116,21 @@ public class KNNClassificationTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01});
-        assertEquals(knnMdl.apply(vector), 2.0);
+        Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01});
+        Assert.assertEquals(knnMdl.apply(vector), 2.0, PRECISION);
     }
 
     /** */
-    public void testBinaryClassificationFarPointsWithWeightedStrategy() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
+    @Test
+    public void binaryClassificationFarPointsWithWeightedStrategy() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {10.0, 10.0, 1.0});
-        data.put(1, new double[] {10.0, 20.0, 1.0});
-        data.put(2, new double[] {-1, -1, 1.0});
-        data.put(3, new double[] {-2, -2, 2.0});
-        data.put(4, new double[] {-1.0, -2.0, 2.0});
-        data.put(5, new double[] {-2.0, -1.0, 2.0});
+
+        data.put(0, new double[]{10.0, 10.0, 1.0});
+        data.put(1, new double[]{10.0, 20.0, 1.0});
+        data.put(2, new double[]{-1, -1, 1.0});
+        data.put(3, new double[]{-2, -2, 2.0});
+        data.put(4, new double[]{-1.0, -2.0, 2.0});
+        data.put(5, new double[]{-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
@@ -139,7 +143,7 @@ public class KNNClassificationTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.WEIGHTED);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01});
-        assertEquals(knnMdl.apply(vector), 1.0);
+        Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01});
+        Assert.assertEquals(knnMdl.apply(vector), 1.0, PRECISION);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
index 66dbca9..ce9cae5 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.ml.knn;
 
-import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
 import org.apache.ignite.ml.knn.regression.KNNRegressionModel;
@@ -30,28 +29,23 @@ import org.junit.Assert;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
+import org.junit.Test;
 
 /**
  * Tests for {@link KNNRegressionTrainer}.
  */
-public class KNNRegressionTest extends BaseKNNTest {
+public class KNNRegressionTest {
     /** */
-    private double[] y;
-
-    /** */
-    private double[][] x;
-
-    /** */
-    public void testSimpleRegressionWithOneNeighbour() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
+    @Test
+    public void simpleRegressionWithOneNeighbour() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {11.0, 0, 0, 0, 0, 0});
-        data.put(1, new double[] {12.0, 2.0, 0, 0, 0, 0});
-        data.put(2, new double[] {13.0, 0, 3.0, 0, 0, 0});
-        data.put(3, new double[] {14.0, 0, 0, 4.0, 0, 0});
-        data.put(4, new double[] {15.0, 0, 0, 0, 5.0, 0});
-        data.put(5, new double[] {16.0, 0, 0, 0, 0, 6.0});
+
+        data.put(0, new double[]{11.0, 0, 0, 0, 0, 0});
+        data.put(1, new double[]{12.0, 2.0, 0, 0, 0, 0});
+        data.put(2, new double[]{13.0, 0, 3.0, 0, 0, 0});
+        data.put(3, new double[]{14.0, 0, 0, 4.0, 0, 0});
+        data.put(4, new double[]{15.0, 0, 0, 0, 5.0, 0});
+        data.put(5, new double[]{16.0, 0, 0, 0, 0, 6.0});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
@@ -63,32 +57,31 @@ public class KNNRegressionTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0});
+        Vector vector = new DenseLocalOnHeapVector(new double[]{0, 0, 0, 5.0, 0.0});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(15, knnMdl.apply(vector), 1E-12);
     }
 
     /** */
-    public void testLongly() {
-
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
+    @Test
+    public void longly() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
-        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
-        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
-        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
-        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
-        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
-        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
-        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
-        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
-        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
-        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
-        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
-        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
-        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
-        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+
+        data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
@@ -100,31 +93,30 @@ public class KNNRegressionTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
+        Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
     }
 
     /** */
     public void testLonglyWithWeightedStrategy() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
-        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
-        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
-        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
-        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
-        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
-        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
-        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
-        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
-        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
-        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
-        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
-        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
-        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
-        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+
+        data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
@@ -136,7 +128,7 @@ public class KNNRegressionTest extends BaseKNNTest {
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
+        Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
new file mode 100644
index 0000000..a25b303
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
@@ -0,0 +1,87 @@
+/*
+ * 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.ml.knn;
+
+import java.io.IOException;
+import java.net.URISyntaxException;
+import java.nio.file.Path;
+import java.nio.file.Paths;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.ml.structures.LabeledDataset;
+import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Base class for decision trees test.
+ */
+public class LabeledDatasetHelper extends GridCommonAbstractTest {
+    /** Count of nodes. */
+    private static final int NODE_COUNT = 4;
+
+    /** Separator. */
+    private static final String SEPARATOR = "\t";
+
+    /** Grid instance. */
+    protected Ignite ignite;
+
+    /**
+     * Default constructor.
+     */
+    public LabeledDatasetHelper() {
+        super(false);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected void beforeTest() throws Exception {
+        ignite = grid(NODE_COUNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        for (int i = 1; i <= NODE_COUNT; i++)
+            startGrid(i);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * Loads labeled dataset from file with .txt extension.
+     *
+     * @param rsrcPath path to dataset.
+     * @return null if path is incorrect.
+     */
+    LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) {
+        try {
+            Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI());
+            try {
+                return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData);
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        } catch (URISyntaxException e) {
+            e.printStackTrace();
+            return null;
+        }
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/8550d61b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
index cdd5dc4..77d40a6 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
@@ -34,7 +34,7 @@ import org.apache.ignite.ml.structures.LabeledVector;
 import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
 
 /** Tests behaviour of KNNClassificationTest. */
-public class LabeledDatasetTest extends BaseKNNTest implements ExternalizableTest<LabeledDataset> {
+public class LabeledDatasetTest extends LabeledDatasetHelper implements ExternalizableTest<LabeledDataset> {
     /** */
     private static final String KNN_IRIS_TXT = "datasets/knn/iris.txt";
 


[20/50] [abbrv] ignite git commit: IGNITE-8176: Integrate gradient descent linear regression with partition based dataset

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
new file mode 100644
index 0000000..fa8fac4
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
@@ -0,0 +1,94 @@
+/*
+ * 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.ml.regressions.linear;
+
+import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
+import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Tests for {@link LinearRegressionSGDTrainer}.
+ */
+@RunWith(Parameterized.class)
+public class LinearRegressionSGDTrainerTest {
+    /** Parameters. */
+    @Parameterized.Parameters(name = "Data divided on {0} partitions")
+    public static Iterable<Integer[]> data() {
+        return Arrays.asList(
+            new Integer[] {1},
+            new Integer[] {2},
+            new Integer[] {3},
+            new Integer[] {5},
+            new Integer[] {7},
+            new Integer[] {100}
+        );
+    }
+
+    /** Number of partitions. */
+    @Parameterized.Parameter
+    public int parts;
+
+    /**
+     * Tests {@code fit()} method on a simple small dataset.
+     */
+    @Test
+    public void testSmallDataFit() {
+        Map<Integer, double[]> data = new HashMap<>();
+        data.put(0, new double[] {-1.0915526, 1.81983527, -0.91409478, 0.70890712, -24.55724107});
+        data.put(1, new double[] {-0.61072904, 0.37545517, 0.21705352, 0.09516495, -26.57226867});
+        data.put(2, new double[] {0.05485406, 0.88219898, -0.80584547, 0.94668307, 61.80919728});
+        data.put(3, new double[] {-0.24835094, -0.34000053, -1.69984651, -1.45902635, -161.65525991});
+        data.put(4, new double[] {0.63675392, 0.31675535, 0.38837437, -1.1221971, -14.46432611});
+        data.put(5, new double[] {0.14194017, 2.18158997, -0.28397346, -0.62090588, -3.2122197});
+        data.put(6, new double[] {-0.53487507, 1.4454797, 0.21570443, -0.54161422, -46.5469012});
+        data.put(7, new double[] {-1.58812173, -0.73216803, -2.15670676, -1.03195988, -247.23559889});
+        data.put(8, new double[] {0.20702671, 0.92864654, 0.32721202, -0.09047503, 31.61484949});
+        data.put(9, new double[] {-0.37890345, -0.04846179, -0.84122753, -1.14667474, -124.92598583});
+
+        LinearRegressionSGDTrainer<?> trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>(
+            new RPropUpdateCalculator(),
+            RPropParameterUpdate::sumLocal,
+            RPropParameterUpdate::avg
+        ), 100000,  10, 100, 123L);
+
+        LinearRegressionModel mdl = trainer.fit(
+            data,
+            parts,
+            (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
+            (k, v) -> v[4]
+        );
+
+        assertArrayEquals(
+            new double[] {72.26948107, 15.95144674, 24.07403921, 66.73038781},
+            mdl.getWeights().getStorage().data(),
+            1e-1
+        );
+
+        assertEquals(2.8421709430404007e-14, mdl.getIntercept(), 1e-1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java
deleted file mode 100644
index bea164d..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-
-/**
- * Tests for {@link LinearRegressionSGDTrainer} on {@link DenseLocalOnHeapMatrix}.
- */
-public class LocalLinearRegressionSGDTrainerTest extends GenericLinearRegressionTrainerTest {
-    /** */
-    public LocalLinearRegressionSGDTrainerTest() {
-        super(
-            new LinearRegressionSGDTrainer(100_000, 1e-12),
-            DenseLocalOnHeapMatrix::new,
-            DenseLocalOnHeapVector::new,
-            1e-2);
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java
index 26ba2fb..0befd9b 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMBinaryTrainerTest.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.ml.svm;
 
+import org.apache.ignite.ml.TestUtils;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.junit.Test;
+
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.junit.Test;
 
 /**
  * Tests for {@link SVMLinearBinaryClassificationTrainer}.
@@ -62,7 +62,8 @@ public class SVMBinaryTrainerTest {
         SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer();
 
         SVMLinearBinaryClassificationModel mdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 10),
+            data,
+            10,
             (k, v) -> Arrays.copyOfRange(v, 1, v.length),
             (k, v) -> v[0]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java
index ad95eb4..31ab4d7 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/svm/SVMMultiClassTrainerTest.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.ml.svm;
 
+import org.apache.ignite.ml.TestUtils;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.junit.Test;
+
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.ThreadLocalRandom;
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.junit.Test;
 
 /**
  * Tests for {@link SVMLinearBinaryClassificationTrainer}.
@@ -65,7 +65,8 @@ public class SVMMultiClassTrainerTest {
             .withAmountOfIterations(20);
 
         SVMLinearMultiClassClassificationModel mdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 10),
+            data,
+            10,
             (k, v) -> Arrays.copyOfRange(v, 1, v.length),
             (k, v) -> v[0]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java
index 94bca3f..d5b0b86 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerIntegrationTest.java
@@ -17,16 +17,16 @@
 
 package org.apache.ignite.ml.tree;
 
-import java.util.Arrays;
-import java.util.Random;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+import java.util.Random;
+
 /**
  * Tests for {@link DecisionTreeClassificationTrainer} that require to start the whole Ignite infrastructure.
  */
@@ -77,7 +77,8 @@ public class DecisionTreeClassificationTrainerIntegrationTest extends GridCommon
         DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0);
 
         DecisionTreeNode tree = trainer.fit(
-            new CacheBasedDatasetBuilder<>(ignite, data),
+            ignite,
+            data,
             (k, v) -> Arrays.copyOf(v, v.length - 1),
             (k, v) -> v[v.length - 1]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java
index 2599bfe..12ef698 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeClassificationTrainerTest.java
@@ -17,17 +17,12 @@
 
 package org.apache.ignite.ml.tree;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.*;
+
 import static junit.framework.TestCase.assertEquals;
 import static junit.framework.TestCase.assertTrue;
 
@@ -68,7 +63,8 @@ public class DecisionTreeClassificationTrainerTest {
         DecisionTreeClassificationTrainer trainer = new DecisionTreeClassificationTrainer(1, 0);
 
         DecisionTreeNode tree = trainer.fit(
-            new LocalDatasetBuilder<>(data, parts),
+            data,
+            parts,
             (k, v) -> Arrays.copyOf(v, v.length - 1),
             (k, v) -> v[v.length - 1]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java
index 754ff20..c2a4638 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerIntegrationTest.java
@@ -17,16 +17,16 @@
 
 package org.apache.ignite.ml.tree;
 
-import java.util.Arrays;
-import java.util.Random;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.util.Arrays;
+import java.util.Random;
+
 /**
  * Tests for {@link DecisionTreeRegressionTrainer} that require to start the whole Ignite infrastructure.
  */
@@ -77,7 +77,8 @@ public class DecisionTreeRegressionTrainerIntegrationTest extends GridCommonAbst
         DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0);
 
         DecisionTreeNode tree = trainer.fit(
-            new CacheBasedDatasetBuilder<>(ignite, data),
+            ignite,
+            data,
             (k, v) -> Arrays.copyOf(v, v.length - 1),
             (k, v) -> v[v.length - 1]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java
index 3bdbf60..bcfb53f 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/DecisionTreeRegressionTrainerTest.java
@@ -17,17 +17,12 @@
 
 package org.apache.ignite.ml.tree;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.*;
+
 import static junit.framework.TestCase.assertEquals;
 import static junit.framework.TestCase.assertTrue;
 
@@ -68,7 +63,8 @@ public class DecisionTreeRegressionTrainerTest {
         DecisionTreeRegressionTrainer trainer = new DecisionTreeRegressionTrainer(1, 0);
 
         DecisionTreeNode tree = trainer.fit(
-            new LocalDatasetBuilder<>(data, parts),
+            data,
+            parts,
             (k, v) -> Arrays.copyOf(v, v.length - 1),
             (k, v) -> v[v.length - 1]
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java
index b259ec9..35f805e 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTIntegrationTest.java
@@ -17,13 +17,11 @@
 
 package org.apache.ignite.ml.tree.performance;
 
-import java.io.IOException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil;
 import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer;
 import org.apache.ignite.ml.tree.DecisionTreeNode;
@@ -31,6 +29,8 @@ import org.apache.ignite.ml.tree.impurity.util.SimpleStepFunctionCompressor;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.io.IOException;
+
 /**
  * Tests {@link DecisionTreeClassificationTrainer} on the MNIST dataset that require to start the whole Ignite
  * infrastructure. For manual run.
@@ -81,7 +81,8 @@ public class DecisionTreeMNISTIntegrationTest extends GridCommonAbstractTest {
             new SimpleStepFunctionCompressor<>());
 
         DecisionTreeNode mdl = trainer.fit(
-            new CacheBasedDatasetBuilder<>(ignite, trainingSet),
+            ignite,
+            trainingSet,
             (k, v) -> v.getPixels(),
             (k, v) -> (double) v.getLabel()
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java
index 6dbd44c..b40c7ac 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/tree/performance/DecisionTreeMNISTTest.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.ml.tree.performance;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.nn.performance.MnistMLPTestUtil;
 import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer;
 import org.apache.ignite.ml.tree.DecisionTreeNode;
@@ -28,6 +24,10 @@ import org.apache.ignite.ml.tree.impurity.util.SimpleStepFunctionCompressor;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
 import static junit.framework.TestCase.assertTrue;
 
 /**
@@ -50,7 +50,8 @@ public class DecisionTreeMNISTTest {
             new SimpleStepFunctionCompressor<>());
 
         DecisionTreeNode mdl = trainer.fit(
-            new LocalDatasetBuilder<>(trainingSet, 10),
+            trainingSet,
+            10,
             (k, v) -> v.getPixels(),
             (k, v) -> (double) v.getLabel()
         );


[18/50] [abbrv] ignite git commit: IGNITE-7996 Merge with master.

Posted by ag...@apache.org.
IGNITE-7996 Merge with master.


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

Branch: refs/heads/ignite-7708
Commit: e333f306d0f32d4c02057fff4238081f25775cf1
Parents: d1be9b8 d02e87b
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Apr 12 11:07:02 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Apr 12 11:10:27 2018 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/app.js         |   2 -
 .../components/cache-edit-form/template.tpl.pug |  22 +-
 .../cache-edit-form/templates/affinity.pug      |  86 +++++
 .../cache-edit-form/templates/concurrency.pug   |  64 ++++
 .../cache-edit-form/templates/general.pug       | 113 +++++++
 .../cache-edit-form/templates/memory.pug        | 158 ++++++++++
 .../templates/near-cache-client.pug             |  50 +++
 .../templates/near-cache-server.pug             |  51 +++
 .../cache-edit-form/templates/node-filter.pug   |  53 ++++
 .../cache-edit-form/templates/query.pug         | 114 +++++++
 .../cache-edit-form/templates/rebalance.pug     |  66 ++++
 .../cache-edit-form/templates/statistics.pug    |  34 ++
 .../cache-edit-form/templates/store.pug         | 310 +++++++++++++++++++
 .../cluster-edit-form/template.tpl.pug          |  62 ++--
 .../cluster-edit-form/templates/atomic.pug      |  75 +++++
 .../cluster-edit-form/templates/attributes.pug  |  40 +++
 .../cluster-edit-form/templates/binary.pug      |  80 +++++
 .../templates/cache-key-cfg.pug                 |  63 ++++
 .../cluster-edit-form/templates/checkpoint.pug  |  82 +++++
 .../templates/checkpoint/fs.pug                 |  36 +++
 .../templates/checkpoint/jdbc.pug               |  47 +++
 .../templates/checkpoint/s3.pug                 | 204 ++++++++++++
 .../templates/client-connector.pug              |  76 +++++
 .../cluster-edit-form/templates/collision.pug   |  58 ++++
 .../templates/collision/custom.pug              |  23 ++
 .../templates/collision/fifo-queue.pug          |  26 ++
 .../templates/collision/job-stealing.pug        |  51 +++
 .../templates/collision/priority-queue.pug      |  41 +++
 .../templates/communication.pug                 | 134 ++++++++
 .../cluster-edit-form/templates/connector.pug   | 100 ++++++
 .../templates/data-storage.pug                  | 301 ++++++++++++++++++
 .../cluster-edit-form/templates/deployment.pug  | 192 ++++++++++++
 .../cluster-edit-form/templates/discovery.pug   |  97 ++++++
 .../cluster-edit-form/templates/events.pug      |  66 ++++
 .../cluster-edit-form/templates/failover.pug    |  89 ++++++
 .../cluster-edit-form/templates/general.pug     |  89 ++++++
 .../templates/general/discovery/cloud.pug       |  78 +++++
 .../templates/general/discovery/google.pug      |  38 +++
 .../templates/general/discovery/jdbc.pug        |  35 +++
 .../templates/general/discovery/kubernetes.pug  |  38 +++
 .../templates/general/discovery/multicast.pug   |  63 ++++
 .../templates/general/discovery/s3.pug          |  38 +++
 .../templates/general/discovery/shared.pug      |  24 ++
 .../templates/general/discovery/vm.pug          |  55 ++++
 .../templates/general/discovery/zookeeper.pug   |  84 +++++
 .../retrypolicy/bounded-exponential-backoff.pug |  26 ++
 .../discovery/zookeeper/retrypolicy/custom.pug  |  25 ++
 .../retrypolicy/exponential-backoff.pug         |  26 ++
 .../discovery/zookeeper/retrypolicy/forever.pug |  23 ++
 .../discovery/zookeeper/retrypolicy/n-times.pug |  24 ++
 .../zookeeper/retrypolicy/one-time.pug          |  23 ++
 .../zookeeper/retrypolicy/until-elapsed.pug     |  24 ++
 .../cluster-edit-form/templates/hadoop.pug      |  87 ++++++
 .../cluster-edit-form/templates/igfs.pug        |  34 ++
 .../templates/load-balancing.pug                | 115 +++++++
 .../cluster-edit-form/templates/logger.pug      |  60 ++++
 .../templates/logger/custom.pug                 |  24 ++
 .../templates/logger/log4j.pug                  |  49 +++
 .../templates/logger/log4j2.pug                 |  38 +++
 .../cluster-edit-form/templates/marshaller.pug  |  75 +++++
 .../cluster-edit-form/templates/memory.pug      | 195 ++++++++++++
 .../cluster-edit-form/templates/metrics.pug     |  46 +++
 .../cluster-edit-form/templates/misc.pug        |  58 ++++
 .../cluster-edit-form/templates/odbc.pug        |  70 +++++
 .../cluster-edit-form/templates/persistence.pug |  82 +++++
 .../cluster-edit-form/templates/service.pug     |  89 ++++++
 .../templates/sql-connector.pug                 |  58 ++++
 .../cluster-edit-form/templates/ssl.pug         |  89 ++++++
 .../cluster-edit-form/templates/swap.pug        |  74 +++++
 .../cluster-edit-form/templates/thread.pug      | 144 +++++++++
 .../cluster-edit-form/templates/time.pug        |  44 +++
 .../templates/transactions.pug                  |  65 ++++
 .../components/igfs-edit-form/template.tpl.pug  |  12 +-
 .../igfs-edit-form/templates/dual.pug           |  42 +++
 .../igfs-edit-form/templates/fragmentizer.pug   |  37 +++
 .../igfs-edit-form/templates/general.pug        |  72 +++++
 .../components/igfs-edit-form/templates/ipc.pug |  55 ++++
 .../igfs-edit-form/templates/misc.pug           | 110 +++++++
 .../igfs-edit-form/templates/secondary.pug      |  55 ++++
 .../components/model-edit-form/template.tpl.pug |   6 +-
 .../model-edit-form/templates/general.pug       |  57 ++++
 .../model-edit-form/templates/query.pug         | 255 +++++++++++++++
 .../model-edit-form/templates/store.pug         | 123 ++++++++
 .../page-configure-basic/template.pug           |  18 +-
 .../components/preview-panel/directive.js       | 246 +++++++++++++++
 .../components/preview-panel/index.js           |  23 ++
 .../app/components/page-configure/index.js      |  25 +-
 .../services/ConfigurationResource.js           |  49 +++
 .../page-configure/services/SummaryZipper.js    |  44 +++
 .../page-configure/services/summary.worker.js   | 147 +++++++++
 .../app/components/page-configure/states.js     | 270 ++++++++++++++++
 .../app/modules/states/configuration.state.js   | 297 ------------------
 .../configuration/Configuration.resource.js     |  42 ---
 .../states/configuration/caches/affinity.pug    |  86 -----
 .../states/configuration/caches/concurrency.pug |  64 ----
 .../states/configuration/caches/general.pug     | 113 -------
 .../states/configuration/caches/memory.pug      | 158 ----------
 .../configuration/caches/near-cache-client.pug  |  50 ---
 .../configuration/caches/near-cache-server.pug  |  51 ---
 .../states/configuration/caches/node-filter.pug |  53 ----
 .../states/configuration/caches/query.pug       | 114 -------
 .../states/configuration/caches/rebalance.pug   |  66 ----
 .../states/configuration/caches/statistics.pug  |  34 --
 .../states/configuration/caches/store.pug       | 310 -------------------
 .../states/configuration/clusters/atomic.pug    |  75 -----
 .../configuration/clusters/attributes.pug       |  40 ---
 .../states/configuration/clusters/binary.pug    |  80 -----
 .../configuration/clusters/cache-key-cfg.pug    |  63 ----
 .../configuration/clusters/checkpoint.pug       |  82 -----
 .../configuration/clusters/checkpoint/fs.pug    |  36 ---
 .../configuration/clusters/checkpoint/jdbc.pug  |  47 ---
 .../configuration/clusters/checkpoint/s3.pug    | 204 ------------
 .../configuration/clusters/client-connector.pug |  76 -----
 .../states/configuration/clusters/collision.pug |  58 ----
 .../configuration/clusters/collision/custom.pug |  23 --
 .../clusters/collision/fifo-queue.pug           |  26 --
 .../clusters/collision/job-stealing.pug         |  51 ---
 .../clusters/collision/priority-queue.pug       |  41 ---
 .../configuration/clusters/communication.pug    | 134 --------
 .../states/configuration/clusters/connector.pug | 100 ------
 .../configuration/clusters/data-storage.pug     | 301 ------------------
 .../configuration/clusters/deployment.pug       | 192 ------------
 .../states/configuration/clusters/discovery.pug |  97 ------
 .../states/configuration/clusters/events.pug    |  66 ----
 .../states/configuration/clusters/failover.pug  |  89 ------
 .../states/configuration/clusters/general.pug   |  89 ------
 .../clusters/general/discovery/cloud.pug        |  78 -----
 .../clusters/general/discovery/google.pug       |  38 ---
 .../clusters/general/discovery/jdbc.pug         |  35 ---
 .../clusters/general/discovery/kubernetes.pug   |  38 ---
 .../clusters/general/discovery/multicast.pug    |  63 ----
 .../clusters/general/discovery/s3.pug           |  38 ---
 .../clusters/general/discovery/shared.pug       |  24 --
 .../clusters/general/discovery/vm.pug           |  55 ----
 .../clusters/general/discovery/zookeeper.pug    |  84 -----
 .../retrypolicy/bounded-exponential-backoff.pug |  26 --
 .../discovery/zookeeper/retrypolicy/custom.pug  |  25 --
 .../retrypolicy/exponential-backoff.pug         |  26 --
 .../discovery/zookeeper/retrypolicy/forever.pug |  23 --
 .../discovery/zookeeper/retrypolicy/n-times.pug |  24 --
 .../zookeeper/retrypolicy/one-time.pug          |  23 --
 .../zookeeper/retrypolicy/until-elapsed.pug     |  24 --
 .../states/configuration/clusters/hadoop.pug    |  87 ------
 .../states/configuration/clusters/igfs.pug      |  34 --
 .../configuration/clusters/load-balancing.pug   | 115 -------
 .../states/configuration/clusters/logger.pug    |  60 ----
 .../configuration/clusters/logger/custom.pug    |  24 --
 .../configuration/clusters/logger/log4j.pug     |  49 ---
 .../configuration/clusters/logger/log4j2.pug    |  38 ---
 .../configuration/clusters/marshaller.pug       |  75 -----
 .../states/configuration/clusters/memory.pug    | 195 ------------
 .../states/configuration/clusters/metrics.pug   |  46 ---
 .../states/configuration/clusters/misc.pug      |  58 ----
 .../states/configuration/clusters/odbc.pug      |  70 -----
 .../configuration/clusters/persistence.pug      |  82 -----
 .../states/configuration/clusters/service.pug   |  89 ------
 .../configuration/clusters/sql-connector.pug    |  58 ----
 .../states/configuration/clusters/ssl.pug       |  89 ------
 .../states/configuration/clusters/swap.pug      |  74 -----
 .../states/configuration/clusters/thread.pug    | 144 ---------
 .../states/configuration/clusters/time.pug      |  44 ---
 .../configuration/clusters/transactions.pug     |  65 ----
 .../states/configuration/domains/general.pug    |  57 ----
 .../states/configuration/domains/query.pug      | 255 ---------------
 .../states/configuration/domains/store.pug      | 123 --------
 .../modules/states/configuration/igfs/dual.pug  |  42 ---
 .../states/configuration/igfs/fragmentizer.pug  |  37 ---
 .../states/configuration/igfs/general.pug       |  72 -----
 .../modules/states/configuration/igfs/ipc.pug   |  55 ----
 .../modules/states/configuration/igfs/misc.pug  | 110 -------
 .../states/configuration/igfs/secondary.pug     |  55 ----
 .../configuration/preview-panel.directive.js    | 239 --------------
 .../summary/summary-zipper.service.js           |  39 ---
 .../configuration/summary/summary.worker.js     | 147 ---------
 174 files changed, 6858 insertions(+), 6822 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/e333f306/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug
----------------------------------------------------------------------
diff --cc modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug
index 0000000,bd8971a..8b43521
mode 000000,100644..100644
--- a/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug
+++ b/modules/web-console/frontend/app/components/page-configure-advanced/components/cluster-edit-form/templates/communication.pug
@@@ -1,0 -1,134 +1,134 @@@
+ //-
+     Licensed to the Apache Software Foundation (ASF) under one or more
+     contributor license agreements.  See the NOTICE file distributed with
+     this work for additional information regarding copyright ownership.
+     The ASF licenses this file to You under the Apache License, Version 2.0
+     (the "License"); you may not use this file except in compliance with
+     the License.  You may obtain a copy of the License at
+ 
+          http://www.apache.org/licenses/LICENSE-2.0
+ 
+     Unless required by applicable law or agreed to in writing, software
+     distributed under the License is distributed on an "AS IS" BASIS,
+     WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+     See the License for the specific language governing permissions and
+     limitations under the License.
+ 
+ include /app/helpers/jade/mixins
+ 
+ -var form = 'communication'
+ -var model = '$ctrl.clonedCluster'
+ -var communication = model + '.communication'
+ 
+ panel-collapsible(ng-form=form on-open=`ui.loadPanel('${form}')`)
+     panel-title Communication
+     panel-description
+         | Configuration of communication with other nodes by TCP/IP.
+         | Provide basic plumbing to send and receive grid messages and is utilized for all distributed grid operations. 
+         | #[a.link-success(href="https://apacheignite.readme.io/docs/network-config" target="_blank") More info]
+     panel-content.pca-form-row(ng-if=`ui.isPanelLoaded('${form}')`)
+         .pca-form-column-6.pc-form-grid-row
+             .pc-form-grid-col-30
+                 +number('Timeout:', `${model}.networkTimeout`, '"commNetworkTimeout"', 'true', '5000', '1', 'Maximum timeout in milliseconds for network requests')
+             .pc-form-grid-col-30
+                 +number('Send retry delay:', `${model}.networkSendRetryDelay`, '"networkSendRetryDelay"', 'true', '1000', '1', 'Interval in milliseconds between message send retries')
+             .pc-form-grid-col-30
+                 +number('Send retry count:', `${model}.networkSendRetryCount`, '"networkSendRetryCount"', 'true', '3', '1', 'Message send retries count')
+             .pc-form-grid-col-30(ng-if='$ctrl.available(["1.0.0", "2.3.0"])')
+                 +number('Discovery startup delay:', `${model}.discoveryStartupDelay`, '"discoveryStartupDelay"', 'true', '60000', '1', 'This value is used to expire messages from waiting list whenever node discovery discrepancies happen')
+             .pc-form-grid-col-60
+                 +java-class('Communication listener:', `${communication}.listener`, '"comListener"', 'true', 'false', 'Listener of communication events')
+             .pc-form-grid-col-30
+                 +text-ip-address('Local IP address:', `${communication}.localAddress`, '"comLocalAddress"', 'true', '0.0.0.0',
+                     'Local host address for socket binding<br/>\
+                     If not specified use all available addres on local host')
+             .pc-form-grid-col-30
+                 +number-min-max('Local port:', `${communication}.localPort`, '"comLocalPort"', 'true', '47100', '1024', '65535', 'Local port for socket binding')
+             .pc-form-grid-col-30
+                 +number('Local port range:', `${communication}.localPortRange`, '"comLocalPortRange"', 'true', '100', '1', 'Local port range for local host ports')
+             .pc-form-grid-col-30
+                 +sane-ignite-form-field-number({
+                     label: 'Shared memory port:',
+                     model: `${communication}.sharedMemoryPort`,
+                     name: '"sharedMemoryPort"',
+                     placeholder: '{{ ::$ctrl.Clusters.sharedMemoryPort.default }}',
+                     min: '{{ ::$ctrl.Clusters.sharedMemoryPort.min }}',
+                     max: '{{ ::$ctrl.Clusters.sharedMemoryPort.max }}',
+                     tip: `Local port to accept shared memory connections<br/>If set to <b>-1</b> shared memory communication will be disabled`
+                 })(
+                     pc-not-in-collection='::$ctrl.Clusters.sharedMemoryPort.invalidValues'
+                 )
+                     +form-field-feedback('"sharedMemoryPort"', 'notInCollection', 'Shared memory port should be more than "{{ ::$ctrl.Clusters.sharedMemoryPort.invalidValues[0] }}" or equal to "{{ ::$ctrl.Clusters.sharedMemoryPort.min }}"')
+             .pc-form-grid-col-30
+                 +number('Idle connection timeout:', `${communication}.idleConnectionTimeout`, '"idleConnectionTimeout"', 'true', '30000', '1',
+                     'Maximum idle connection timeout upon which a connection to client will be closed')
+             .pc-form-grid-col-30
+                 +number('Connect timeout:', `${communication}.connectTimeout`, '"connectTimeout"', 'true', '5000', '0', 'Connect timeout used when establishing connection with remote nodes')
+             .pc-form-grid-col-30
+                 +number('Max. connect timeout:', `${communication}.maxConnectTimeout`, '"maxConnectTimeout"', 'true', '600000', '0', 'Maximum connect timeout')
+             .pc-form-grid-col-30
+                 +number('Reconnect count:', `${communication}.reconnectCount`, '"comReconnectCount"', 'true', '10', '1',
+                     'Maximum number of reconnect attempts used when establishing connection with remote nodes')
+             .pc-form-grid-col-30
+                 +number('Socket send buffer:', `${communication}.socketSendBuffer`, '"socketSendBuffer"', 'true', '32768', '0', 'Send buffer size for sockets created or accepted by this SPI')
+             .pc-form-grid-col-30
+                 +number('Socket receive buffer:', `${communication}.socketReceiveBuffer`, '"socketReceiveBuffer"', 'true', '32768', '0', 'Receive buffer size for sockets created or accepted by this SPI')
+             .pc-form-grid-col-30
+                 +number('Slow client queue limit:', `${communication}.slowClientQueueLimit`, '"slowClientQueueLimit"', 'true', '0', '0', 'Slow client queue limit')
+             .pc-form-grid-col-30
+                 +sane-ignite-form-field-number({
+                     label: 'Ack send threshold:',
+                     model: `${communication}.ackSendThreshold`,
+                     name: '"ackSendThreshold"',
+                     placeholder: '{{ ::$ctrl.Clusters.ackSendThreshold.default }}',
+                     min: '{{ ::$ctrl.Clusters.ackSendThreshold.min }}',
+                     tip: 'Number of received messages per connection to node after which acknowledgment message is sent'
+                 })
+             .pc-form-grid-col-30
+                 +sane-ignite-form-field-number({
+                     label: 'Message queue limit:',
+                     model: `${communication}.messageQueueLimit`,
+                     name: '"messageQueueLimit"',
+                     placeholder: '{{ ::$ctrl.Clusters.messageQueueLimit.default }}',
+                     min: '{{ ::$ctrl.Clusters.messageQueueLimit.min }}',
+                     tip: 'Message queue limit for incoming and outgoing messages'
+                 })
+             .pc-form-grid-col-30
++                //- allowInvalid: true prevents from infinite digest loop when old value was 0 and becomes less than allowed minimum
+                 +sane-ignite-form-field-number({
+                     label: 'Unacknowledged messages:',
+                     model: `${communication}.unacknowledgedMessagesBufferSize`,
+                     name: '"unacknowledgedMessagesBufferSize"',
+                     placeholder: '{{ ::$ctrl.Clusters.unacknowledgedMessagesBufferSize.default }}',
+                     min: `{{ $ctrl.Clusters.unacknowledgedMessagesBufferSize.min(
+                         ${communication}.unacknowledgedMessagesBufferSize,
+                         ${communication}.messageQueueLimit,
+                         ${communication}.ackSendThreshold
+                     ) }}`,
+                     tip: `Maximum number of stored unacknowledged messages per connection to node<br/>
+                     If specified non zero value it should be
+                     <ul>
+                         <li>At least ack send threshold * {{ ::$ctrl.Clusters.unacknowledgedMessagesBufferSize.validRatio }}</li>
+                         <li>At least message queue limit * {{ ::$ctrl.Clusters.unacknowledgedMessagesBufferSize.validRatio }}</li>
+                     </ul>`
+                 })(
 -                    //- allowInvalid: true prevents from infinite digest loop when old value was 0 and becomes less than allowed minimum
+                     ng-model-options=`{
+                         allowInvalid: true
+                     }`
+                 )
+             .pc-form-grid-col-30
+                 +number('Socket write timeout:', `${communication}.socketWriteTimeout`, '"socketWriteTimeout"', 'true', '2000', '0', 'Socket write timeout')
+             .pc-form-grid-col-30
+                 +number('Selectors count:', `${communication}.selectorsCount`, '"selectorsCount"', 'true', 'min(4, availableProcessors)', '1', 'Count of selectors te be used in TCP server')
+             .pc-form-grid-col-60
+                 +java-class('Address resolver:', `${communication}.addressResolver`, '"comAddressResolver"', 'true', 'false', 'Provides resolution between external and internal addresses')
+             .pc-form-grid-col-60
+                 +checkbox('Direct buffer', `${communication}.directBuffer`, '"directBuffer"',
+                 'If value is true, then SPI will use ByteBuffer.allocateDirect(int) call<br/>\
+                 Otherwise, SPI will use ByteBuffer.allocate(int) call')
+             .pc-form-grid-col-60
+                 +checkbox('Direct send buffer', `${communication}.directSendBuffer`, '"directSendBuffer"', 'Flag defining whether direct send buffer should be used')
+             .pc-form-grid-col-60
+                 +checkbox('TCP_NODELAY option', `${communication}.tcpNoDelay`, '"tcpNoDelay"', 'Value for TCP_NODELAY socket option')
+         .pca-form-column-6
+             +preview-xml-java(model, 'clusterCommunication')

http://git-wip-us.apache.org/repos/asf/ignite/blob/e333f306/modules/web-console/frontend/app/components/page-configure/services/summary.worker.js
----------------------------------------------------------------------
diff --cc modules/web-console/frontend/app/components/page-configure/services/summary.worker.js
index 0000000,c80d698..b3b0bce
mode 000000,100644..100644
--- a/modules/web-console/frontend/app/components/page-configure/services/summary.worker.js
+++ b/modules/web-console/frontend/app/components/page-configure/services/summary.worker.js
@@@ -1,0 -1,147 +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.
+  */
+ 
+ import JSZip from 'jszip';
+ 
+ import IgniteMavenGenerator from 'app/modules/configuration/generator/Maven.service';
+ import IgniteDockerGenerator from 'app/modules/configuration/generator/Docker.service';
+ import IgniteReadmeGenerator from 'app/modules/configuration/generator/Readme.service';
+ import IgnitePropertiesGenerator from 'app/modules/configuration/generator/Properties.service';
+ import IgniteConfigurationGenerator from 'app/modules/configuration/generator/ConfigurationGenerator';
+ 
+ import IgniteJavaTransformer from 'app/modules/configuration/generator/JavaTransformer.service';
+ import IgniteSpringTransformer from 'app/modules/configuration/generator/SpringTransformer.service';
+ 
+ import {nonEmpty, nonNil} from 'app/utils/lodashMixins';
+ import get from 'lodash/get';
+ import filter from 'lodash/filter';
+ import isEmpty from 'lodash/isEmpty';
+ 
+ const maven = new IgniteMavenGenerator();
+ const docker = new IgniteDockerGenerator();
+ const readme = new IgniteReadmeGenerator();
+ const properties = new IgnitePropertiesGenerator();
+ 
+ const java = IgniteJavaTransformer;
+ const spring = IgniteSpringTransformer;
+ 
+ const generator = IgniteConfigurationGenerator;
+ 
+ const escapeFileName = (name) => name.replace(/[\\\/*\"\[\],\.:;|=<>?]/g, '-').replace(/ /g, '_');
+ 
+ const kubernetesConfig = (cluster) => {
+     if (!cluster.discovery.Kubernetes)
+         cluster.discovery.Kubernetes = { serviceName: 'ignite' };
+ 
+     return `apiVersion: v1\n\
+ kind: Service\n\
+ metadata:\n\
+   # Name of Ignite Service used by Kubernetes IP finder for IP addresses lookup.\n\
+   name: ${ cluster.discovery.Kubernetes.serviceName || 'ignite' }\n\
+ spec:\n\
+   clusterIP: None # custom value.\n\
+   ports:\n\
+     - port: 9042 # custom value.\n\
+   selector:\n\
+     # Must be equal to one of the labels set in Ignite pods'\n\
+     # deployement configuration.\n\
+     app: ${ cluster.discovery.Kubernetes.serviceName || 'ignite' }`;
+ };
+ 
+ // eslint-disable-next-line no-undef
+ onmessage = function(e) {
+     const {cluster, data, demo, targetVer} = e.data;
+ 
+     const zip = new JSZip();
+ 
+     if (!data.docker)
+         data.docker = docker.generate(cluster, targetVer);
+ 
+     zip.file('Dockerfile', data.docker);
+     zip.file('.dockerignore', docker.ignoreFile());
+ 
+     const cfg = generator.igniteConfiguration(cluster, targetVer, false);
+     const clientCfg = generator.igniteConfiguration(cluster, targetVer, true);
+     const clientNearCaches = filter(cluster.caches, (cache) =>
+         cache.cacheMode === 'PARTITIONED' && 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`;
+ 
+     if (cluster.discovery.kind === 'Kubernetes')
+         zip.file(`${metaPath}/ignite-service.yaml`, kubernetesConfig(cluster));
+ 
+     zip.file(`${metaPath}/${serverXml}`, spring.igniteConfiguration(cfg, targetVer).asString());
+     zip.file(`${metaPath}/${clientXml}`, spring.igniteConfiguration(clientCfg, targetVer, clientNearCaches).asString());
+ 
+     const cfgPath = `${srcPath}/config`;
+ 
+     zip.file(`${cfgPath}/ServerConfigurationFactory.java`, java.igniteConfiguration(cfg, targetVer, 'config', 'ServerConfigurationFactory').asString());
+     zip.file(`${cfgPath}/ClientConfigurationFactory.java`, java.igniteConfiguration(clientCfg, targetVer, '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));
++    const cachesToLoad = filter(cluster.caches, (cache) => nonNil(_.get(cache, 'cacheStoreFactory.kind')));
+ 
+     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, targetVer));
+ 
+     zip.file('README.txt', readme.generate());
+     zip.file('jdbc-drivers/README.txt', readme.generateJDBC());
+ 
+     if (isEmpty(data.pojos))
+         data.pojos = java.pojos(cluster.caches, true);
+ 
+     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/e333f306/modules/web-console/frontend/app/components/page-configure/states.js
----------------------------------------------------------------------
diff --cc modules/web-console/frontend/app/components/page-configure/states.js
index 0000000,f8bb4dc..a75e851
mode 000000,100644..100644
--- a/modules/web-console/frontend/app/components/page-configure/states.js
+++ b/modules/web-console/frontend/app/components/page-configure/states.js
@@@ -1,0 -1,273 +1,270 @@@
+ /*
+  * 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 angular from 'angular';
 -
+ import base2 from 'views/base2.pug';
+ import pageConfigureAdvancedClusterComponent from '../page-configure-advanced/components/page-configure-advanced-cluster/component';
+ import pageConfigureAdvancedModelsComponent from '../page-configure-advanced/components/page-configure-advanced-models/component';
+ import pageConfigureAdvancedCachesComponent from '../page-configure-advanced/components/page-configure-advanced-caches/component';
+ import pageConfigureAdvancedIGFSComponent from '../page-configure-advanced/components/page-configure-advanced-igfs/component';
+ 
 -import get from 'lodash/get';
+ import {Observable} from 'rxjs/Observable';
+ 
+ const idRegex = `new|[a-z0-9]+`;
+ 
+ const shortCachesResolve = ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => {
+     if ($transition$.params().clusterID === 'new') return Promise.resolve();
+     return Observable.fromPromise($transition$.injector().getAsync('_cluster'))
+     .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1))
+     .switchMap((cluster) => {
+         return etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id});
+     })
+     .toPromise();
+ }];
+ 
+ function registerStates($stateProvider) {
+     // Setup the states.
+     $stateProvider
+     .state('base.configuration', {
+         abstract: true,
+         permission: 'configuration',
+         url: '/configuration',
+         onEnter: ['ConfigureState', (ConfigureState) => ConfigureState.dispatchAction({type: 'PRELOAD_STATE', state: {}})],
+         views: {
+             '@': {
+                 template: base2
+             }
+         },
+         resolve: {
+             _shortClusters: ['ConfigEffects', ({etp}) => {
+                 return etp('LOAD_USER_CLUSTERS');
+             }]
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         }
+     })
+     .state('base.configuration.overview', {
+         url: '/overview',
+         component: 'pageConfigureOverview',
+         permission: 'configuration',
+         tfMetaTags: {
+             title: 'Configuration'
+         }
+     })
+     .state('base.configuration.edit', {
+         url: `/{clusterID:${idRegex}}`,
+         permission: 'configuration',
+         component: 'pageConfigure',
+         resolve: {
+             _cluster: ['ConfigEffects', '$transition$', ({etp}, $transition$) => {
+                 return $transition$.injector().getAsync('_shortClusters').then(() => {
+                     return etp('LOAD_AND_EDIT_CLUSTER', {clusterID: $transition$.params().clusterID});
+                 });
+             }]
+         },
+         data: {
+             errorState: 'base.configuration.overview'
+         },
+         redirectTo: ($transition$) => {
+             const [ConfigureState, ConfigSelectors] = ['ConfigureState', 'ConfigSelectors'].map((t) => $transition$.injector().get(t));
+             const waitFor = ['_cluster', '_shortClusters'].map((t) => $transition$.injector().getAsync(t));
+             return Observable.fromPromise(Promise.all(waitFor)).switchMap(() => {
+                 return Observable.combineLatest(
+                     ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1),
+                     ConfigureState.state$.let(ConfigSelectors.selectShortClusters()).take(1)
+                 );
+             })
+             .map(([cluster = {caches: []}, clusters]) => {
+                 return (clusters.value.size > 10 || cluster.caches.length > 5)
+                     ? 'base.configuration.edit.advanced'
+                     : 'base.configuration.edit.basic';
+             })
+             .toPromise();
+         },
+         failState: 'signin',
+         tfMetaTags: {
+             title: 'Configuration'
+         }
+     })
+     .state('base.configuration.edit.basic', {
+         url: '/basic',
+         component: 'pageConfigureBasic',
+         permission: 'configuration',
+         resolve: {
+             _shortCaches: shortCachesResolve
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Basic Configuration'
+         }
+     })
+     .state('base.configuration.edit.advanced', {
+         url: '/advanced',
+         component: 'pageConfigureAdvanced',
+         permission: 'configuration',
+         redirectTo: 'base.configuration.edit.advanced.cluster'
+     })
+     .state('base.configuration.edit.advanced.cluster', {
+         url: '/cluster',
+         component: pageConfigureAdvancedClusterComponent.name,
+         permission: 'configuration',
+         resolve: {
+             _shortCaches: shortCachesResolve
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Configure Cluster'
+         }
+     })
+     .state('base.configuration.edit.advanced.caches', {
+         url: '/caches',
+         permission: 'configuration',
+         component: pageConfigureAdvancedCachesComponent.name,
+         resolve: {
+             _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => {
+                 if ($transition$.params().clusterID === 'new') return Promise.resolve();
+                 return Observable.fromPromise($transition$.injector().getAsync('_cluster'))
+                 .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1))
+                 .map((cluster) => {
+                     return Promise.all([
+                         etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}),
+                         etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id}),
+                         etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id})
+                     ]);
+                 })
+                 .toPromise();
+             }]
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Configure Caches'
+         }
+     })
+     .state('base.configuration.edit.advanced.caches.cache', {
+         url: `/{cacheID:${idRegex}}`,
+         permission: 'configuration',
+         resolve: {
+             _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => {
+                 const {clusterID, cacheID} = $transition$.params();
+                 if (cacheID === 'new') return Promise.resolve();
+                 return etp('LOAD_CACHE', {cacheID});
+             }]
+         },
+         data: {
+             errorState: 'base.configuration.edit.advanced.caches'
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Configure Caches'
+         }
+     })
+     .state('base.configuration.edit.advanced.models', {
+         url: '/models',
+         component: pageConfigureAdvancedModelsComponent.name,
+         permission: 'configuration',
+         resolve: {
+             _shortCachesAndModels: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => {
+                 if ($transition$.params().clusterID === 'new') return Promise.resolve();
+                 return Observable.fromPromise($transition$.injector().getAsync('_cluster'))
+                 .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1))
+                 .map((cluster) => {
+                     return Promise.all([
+                         etp('LOAD_SHORT_CACHES', {ids: cluster.caches, clusterID: cluster._id}),
+                         etp('LOAD_SHORT_MODELS', {ids: cluster.models, clusterID: cluster._id})
+                     ]);
+                 })
+                 .toPromise();
+             }]
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Configure SQL Schemes'
+         }
+     })
+     .state('base.configuration.edit.advanced.models.model', {
+         url: `/{modelID:${idRegex}}`,
+         resolve: {
+             _cache: ['ConfigEffects', '$transition$', ({etp}, $transition$) => {
+                 const {clusterID, modelID} = $transition$.params();
+                 if (modelID === 'new') return Promise.resolve();
+                 return etp('LOAD_MODEL', {modelID});
+             }]
+         },
+         data: {
+             errorState: 'base.configuration.edit.advanced.models'
+         },
+         permission: 'configuration',
+         resolvePolicy: {
+             async: 'NOWAIT'
+         }
+     })
+     .state('base.configuration.edit.advanced.igfs', {
+         url: '/igfs',
+         component: pageConfigureAdvancedIGFSComponent.name,
+         permission: 'configuration',
+         resolve: {
+             _shortIGFSs: ['ConfigSelectors', 'ConfigureState', 'ConfigEffects', '$transition$', (ConfigSelectors, ConfigureState, {etp}, $transition$) => {
+                 if ($transition$.params().clusterID === 'new') return Promise.resolve();
+                 return Observable.fromPromise($transition$.injector().getAsync('_cluster'))
+                 .switchMap(() => ConfigureState.state$.let(ConfigSelectors.selectCluster($transition$.params().clusterID)).take(1))
+                 .map((cluster) => {
+                     return Promise.all([
+                         etp('LOAD_SHORT_IGFSS', {ids: cluster.igfss, clusterID: cluster._id})
+                     ]);
+                 })
+                 .toPromise();
+             }]
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         },
+         tfMetaTags: {
+             title: 'Configure IGFS'
+         }
+     })
+     .state('base.configuration.edit.advanced.igfs.igfs', {
+         url: `/{igfsID:${idRegex}}`,
+         permission: 'configuration',
+         resolve: {
+             _igfs: ['ConfigEffects', '$transition$', ({etp}, $transition$) => {
+                 const {clusterID, igfsID} = $transition$.params();
+                 if (igfsID === 'new') return Promise.resolve();
+                 return etp('LOAD_IGFS', {igfsID});
+             }]
+         },
+         data: {
+             errorState: 'base.configuration.edit.advanced.igfs'
+         },
+         resolvePolicy: {
+             async: 'NOWAIT'
+         }
+     });
+ }
+ 
+ registerStates.$inject = ['$stateProvider'];
+ 
+ export {registerStates};


[11/50] [abbrv] ignite git commit: IGNITE-4756 Print info about partition distribution to log

Posted by ag...@apache.org.
IGNITE-4756 Print info about partition distribution to log

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: a3eb1f5d753a38c4019440e1bf39d00bc6136455
Parents: 0e73fa2
Author: Vyacheslav Daradur <da...@gmail.com>
Authored: Wed Apr 11 14:41:29 2018 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Apr 11 14:41:29 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   7 +
 .../affinity/GridAffinityAssignmentCache.java   |  50 +++-
 .../AffinityDistributionLoggingTest.java        | 268 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite5.java       |   9 +-
 4 files changed, 327 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a3eb1f5d/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 9da123e..04eb425 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -857,6 +857,13 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_BPLUS_TREE_LOCK_RETRIES = "IGNITE_BPLUS_TREE_LOCK_RETRIES";
 
     /**
+     * The threshold of uneven distribution above which partition distribution will be logged.
+     *
+     * The default is '50', that means: warn about nodes with 50+% difference.
+     */
+    public static final String IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD = "IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD";
+
+    /**
      * Enforces singleton.
      */
     private IgniteSystemProperties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3eb1f5d/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 18edd02..b1899e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -34,13 +34,14 @@ import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.affinity.AffinityCentralizedFunction;
 import org.apache.ignite.cache.affinity.AffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.internal.GridKernalContext;
-import org.apache.ignite.internal.cluster.NodeOrderComparator;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.cluster.NodeOrderComparator;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents;
 import org.apache.ignite.internal.processors.cluster.BaselineTopology;
@@ -53,7 +54,10 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE;
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD;
+import static org.apache.ignite.IgniteSystemProperties.getFloat;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT;
 
 /**
@@ -63,6 +67,9 @@ public class GridAffinityAssignmentCache {
     /** Cleanup history size. */
     private final int MAX_HIST_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 500);
 
+    /** Partition distribution. */
+    private final float partDistribution = getFloat(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, 50f);
+
     /** Group name if specified or cache name. */
     private final String cacheOrGrpName;
 
@@ -367,6 +374,9 @@ public class GridAffinityAssignmentCache {
 
         idealAssignment = assignment;
 
+        if (ctx.cache().cacheMode(cacheOrGrpName) == PARTITIONED)
+            printDistributionIfThresholdExceeded(assignment, sorted.size());
+
         if (hasBaseline) {
             baselineTopology = discoCache.state().baselineTopology();
             assert baselineAssignment != null;
@@ -418,6 +428,44 @@ public class GridAffinityAssignmentCache {
     }
 
     /**
+     * Calculates and logs partitions distribution if threshold of uneven distribution {@link #partDistribution} is exceeded.
+     *
+     * @param assignments Assignments to calculate partitions distribution.
+     * @param nodes Affinity nodes number.
+     * @see IgniteSystemProperties#IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD
+     */
+    private void printDistributionIfThresholdExceeded(List<List<ClusterNode>> assignments, int nodes) {
+        int locPrimaryCnt = 0;
+        int locBackupCnt = 0;
+
+        for (List<ClusterNode> assignment : assignments) {
+            for (int i = 0; i < assignment.size(); i++) {
+                ClusterNode node = assignment.get(i);
+
+                if (node.isLocal()) {
+                    if (i == 0)
+                        locPrimaryCnt++;
+                    else
+                        locBackupCnt++;
+                }
+            }
+        }
+
+        float expCnt = (float)partsCnt / nodes;
+
+        float deltaPrimary = Math.abs(1 - (float)locPrimaryCnt / expCnt) * 100;
+        float deltaBackup = Math.abs(1 - (float)locBackupCnt / (expCnt * backups)) * 100;
+
+        if (deltaPrimary > partDistribution || deltaBackup > partDistribution) {
+            log.info(String.format("Local node affinity assignment distribution is not ideal " +
+                    "[cache=%s, expectedPrimary=%.2f, actualPrimary=%d, " +
+                    "expectedBackups=%.2f, actualBackups=%d, warningThreshold=%.2f%%]",
+                cacheOrGrpName, expCnt, locPrimaryCnt,
+                expCnt * backups, locBackupCnt, partDistribution));
+        }
+    }
+
+    /**
      * Copies previous affinity assignment when discovery event does not cause affinity assignment changes
      * (e.g. client node joins on leaves).
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3eb1f5d/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityDistributionLoggingTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityDistributionLoggingTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityDistributionLoggingTest.java
new file mode 100644
index 0000000..813c830
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityDistributionLoggingTest.java
@@ -0,0 +1,268 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.affinity;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheProcessor;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.testframework.GridStringLogger;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IGNITE_INSTANCE_NAME;
+
+/**
+ * Tests of partitions distribution logging.
+ *
+ * Tests based on using of affinity function which provides an even distribution of partitions between nodes.
+ *
+ * @see EvenDistributionAffinityFunction
+ */
+public class AffinityDistributionLoggingTest extends GridCommonAbstractTest {
+    /** Pattern to test. */
+    private static final String LOG_MESSAGE_PREFIX = "Local node affinity assignment distribution is not ideal ";
+
+    /** Partitions number. */
+    private int parts = 0;
+
+    /** Nodes number. */
+    private int nodes = 0;
+
+    /** Backups number. */
+    private int backups = 0;
+
+    /** For storing original value of system property. */
+    private String tempProp;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        tempProp = System.getProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        if (tempProp != null)
+            System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, tempProp);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        System.clearProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD);
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setCacheMode(CacheMode.PARTITIONED);
+        cacheCfg.setBackups(backups);
+        cacheCfg.setAffinity(new EvenDistributionAffinityFunction(parts));
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception In case of an error.
+     */
+    public void test2PartitionsIdealDistributionIsNotLogged() throws Exception {
+        System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, "0");
+
+        nodes = 2;
+        parts = 2;
+        backups = 1;
+
+        String testsLog = runAndGetExchangeLog();
+
+        assertFalse(testsLog.contains(LOG_MESSAGE_PREFIX));
+    }
+
+    /**
+     * @throws Exception In case of an error.
+     */
+    public void test120PartitionsIdeadDistributionIsNotLogged() throws Exception {
+        System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, "0.0");
+
+        nodes = 3;
+        parts = 120;
+        backups = 2;
+
+        String testsLog = runAndGetExchangeLog();
+
+        assertFalse(testsLog.contains(LOG_MESSAGE_PREFIX));
+    }
+
+    /**
+     * @throws Exception In case of an error.
+     */
+    public void test5PartitionsNotIdealDistributionIsLogged() throws Exception {
+        System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, "50.0");
+
+        nodes = 4;
+        parts = 5;
+        backups = 3;
+
+        String testsLog = runAndGetExchangeLog();
+
+        assertTrue(testsLog.contains(LOG_MESSAGE_PREFIX));
+    }
+
+    /**
+     * @throws Exception In case of an error.
+     */
+    public void test7PartitionsNotIdealDistributionSuppressedLogging() throws Exception {
+        System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, "50.0");
+
+        nodes = 3;
+        parts = 7;
+        backups = 0;
+
+        String testsLog = runAndGetExchangeLog();
+
+        assertFalse(testsLog.contains(LOG_MESSAGE_PREFIX));
+    }
+
+    /**
+     * @throws Exception In case of an error.
+     */
+    public void test5PartitionsNotIdealDistributionSuppressedLogging() throws Exception {
+        System.setProperty(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, "65");
+
+        nodes = 4;
+        parts = 5;
+        backups = 3;
+
+        String testsLog = runAndGetExchangeLog();
+
+        assertFalse(testsLog.contains(LOG_MESSAGE_PREFIX));
+    }
+
+    /**
+     * Starts a specified number of Ignite nodes and log partition node exchange during a last node's startup.
+     *
+     * @return Log of latest partition map exchange.
+     * @throws Exception In case of an error.
+     */
+    private String runAndGetExchangeLog() throws Exception {
+        assert nodes > 1;
+
+        IgniteEx ignite = (IgniteEx)startGrids(nodes - 1);
+
+        awaitPartitionMapExchange();
+
+        GridCacheProcessor proc = ignite.context().cache();
+
+        GridCacheContext cctx = proc.context().cacheContext(CU.cacheId(DEFAULT_CACHE_NAME));
+
+        final GridStringLogger log = new GridStringLogger(false, this.log);
+
+        GridAffinityAssignmentCache aff = GridTestUtils.getFieldValue(cctx.affinity(), "aff");
+
+        GridTestUtils.setFieldValue(aff, "log", log);
+
+        startGrid(nodes);
+
+        awaitPartitionMapExchange();
+
+        return log.toString();
+    }
+
+    /**
+     * Affinity function for a partitioned cache which provides even distribution partitions between nodes in cluster.
+     */
+    private static class EvenDistributionAffinityFunction implements AffinityFunction {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** Partitions number. */
+        private int parts;
+
+        /**
+         * @param parts Number of partitions for one cache.
+         */
+        private EvenDistributionAffinityFunction(int parts) {
+            this.parts = parts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int partitions() {
+            return parts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int partition(Object key) {
+            return key.hashCode() % parts;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
+            List<ClusterNode> nodes = new ArrayList<>(affCtx.currentTopologySnapshot());
+
+            nodes.sort(Comparator.comparing(o -> o.<String>attribute(ATTR_IGNITE_INSTANCE_NAME)));
+
+            List<List<ClusterNode>> res = new ArrayList<>(parts);
+
+            for (int i = 0; i < parts; i++) {
+                Set<ClusterNode> n0 = new LinkedHashSet<>();
+
+                n0.add(nodes.get(i % nodes.size()));
+
+                for (int j = 1; j <= affCtx.backups(); j++)
+                    n0.add(nodes.get((i + j) % nodes.size()));
+
+                res.add(new ArrayList<>(n0));
+            }
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void removeNode(UUID nodeId) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void reset() {
+            // No-op.
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3eb1f5d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 7c41e49..945a76c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.GridCacheAffinityBackupsSelfTest;
 import org.apache.ignite.IgniteCacheAffinitySelfTest;
 import org.apache.ignite.cache.affinity.AffinityClientNodeSelfTest;
+import org.apache.ignite.cache.affinity.AffinityDistributionLoggingTest;
 import org.apache.ignite.cache.affinity.AffinityHistoryCleanupTest;
 import org.apache.ignite.cache.affinity.local.LocalAffinityFunctionTest;
 import org.apache.ignite.internal.GridCachePartitionExchangeManagerHistSizeTest;
@@ -35,13 +36,7 @@ import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadTh
 import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughEvictionsVariationsSuite;
 import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest;
-import org.apache.ignite.internal.processors.cache.PartitionedAtomicCacheGetsDistributionTest;
-import org.apache.ignite.internal.processors.cache.PartitionedTransactionalOptimisticCacheGetsDistributionTest;
-import org.apache.ignite.internal.processors.cache.PartitionedTransactionalPessimisticCacheGetsDistributionTest;
 import org.apache.ignite.internal.processors.cache.PartitionsExchangeOnDiscoveryHistoryOverflowTest;
-import org.apache.ignite.internal.processors.cache.ReplicatedAtomicCacheGetsDistributionTest;
-import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalOptimisticCacheGetsDistributionTest;
-import org.apache.ignite.internal.processors.cache.ReplicatedTransactionalPessimisticCacheGetsDistributionTest;
 import org.apache.ignite.internal.processors.cache.distributed.Cache64kPartitionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
@@ -95,6 +90,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
         suite.addTestSuite(LocalAffinityFunctionTest.class);
         suite.addTestSuite(AffinityHistoryCleanupTest.class);
 
+        suite.addTestSuite(AffinityDistributionLoggingTest.class);
+
         suite.addTestSuite(IgniteCacheAtomicProtocolTest.class);
 
         suite.addTestSuite(PartitionsExchangeOnDiscoveryHistoryOverflowTest.class);


[50/50] [abbrv] ignite git commit: IGNITE-7708 Debug

Posted by ag...@apache.org.
IGNITE-7708 Debug


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

Branch: refs/heads/ignite-7708
Commit: eced6a11a759b822772e1de76420f957a6679e88
Parents: ad80570
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Apr 16 16:56:49 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Apr 16 16:56:49 2018 +0300

----------------------------------------------------------------------
 .../configvariations/ConfigVariationsTestSuiteBuilder.java  | 9 ++++++++-
 .../junits/IgniteConfigVariationsAbstractTest.java          | 4 ++++
 2 files changed, 12 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/eced6a11/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
index 4a60671..94e56c9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/configvariations/ConfigVariationsTestSuiteBuilder.java
@@ -191,6 +191,8 @@ public class ConfigVariationsTestSuiteBuilder {
             + ", igniteCfg=" + factory.getIgniteConfigurationDescription()
             + ", cacheCfg=" + factory.getCacheConfigurationDescription() + "]";
 
+        System.out.println("Building test suite [suffix=" + clsNameSuffix + ", stopNodes=" + stopNodes + ']');
+
         VariationsTestsConfig testCfg = new VariationsTestsConfig(factory, clsNameSuffix, stopNodes, cacheStartMode,
             gridsCnt, !skipWaitPartMapExchange);
 
@@ -223,7 +225,12 @@ public class ConfigVariationsTestSuiteBuilder {
             boolean startCache = i == 0;
             boolean stopCache = i + 1 == testedNodeCnt;
 
-            VariationsTestsConfig cfg0 = new VariationsTestsConfig(cfg.configurationFactory(), cfg.description(),
+            System.out.println("Building multi-node test suite [clsSuffix=" + cfg.description() +
+                ", testedNode=" + i + ", stopNodes=" + stopNodes + ']');
+
+            VariationsTestsConfig cfg0 = new VariationsTestsConfig(
+                cfg.configurationFactory(),
+                cfg.description() + "-node-" + i,
                 stopNodes, startCache, stopCache, cfg.cacheStartMode(), cfg.gridCount(), i, withClients,
                 !skipWaitParMapExchange);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/eced6a11/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
index eba201a..2e6031c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteConfigVariationsAbstractTest.java
@@ -132,6 +132,9 @@ public abstract class IgniteConfigVariationsAbstractTest extends GridCommonAbstr
 
             memoryUsage();
         }
+        else {
+            info("Will NOT stop nodes: " + this);
+        }
     }
 
     /**
@@ -246,6 +249,7 @@ public abstract class IgniteConfigVariationsAbstractTest extends GridCommonAbstr
             }
 
             info("Running test in data mode [dataMode=" + dataMode +
+                ", igniteInstance=" + getTestIgniteInstanceName() +
                 ", marshaller=" + getConfiguration().getMarshaller() + ']');
 
             if (i != 0)


[30/50] [abbrv] ignite git commit: IGNITE-8240 .NET: Use default scheduler when starting Tasks

Posted by ag...@apache.org.
IGNITE-8240 .NET: Use default scheduler when starting Tasks

This closes #3812


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

Branch: refs/heads/ignite-7708
Commit: 44c4656ce8cc304b0858a18438f57fd9daf9ecb0
Parents: 6df5f99
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Apr 13 12:28:19 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Apr 13 12:28:19 2018 +0300

----------------------------------------------------------------------
 .gitignore                                      |  1 +
 .../IgniteSessionStateStoreProviderTest.cs      |  7 +-
 .../Apache.Ignite.Benchmarks/BenchmarkRunner.cs |  1 -
 .../Binary/BinaryDynamicRegistrationTest.cs     |  2 +-
 .../Cache/CacheAbstractTransactionalTest.cs     |  5 +-
 .../Client/ClientConnectionTest.cs              |  3 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |  9 +--
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |  3 +-
 .../IgniteStartStopTest.cs                      |  5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |  5 +-
 .../Apache.Ignite.Core.csproj                   |  1 +
 .../Impl/Client/ClientSocket.cs                 |  6 +-
 .../Impl/Common/TaskRunner.cs                   | 70 ++++++++++++++++++++
 .../Impl/Datastream/DataStreamerBatch.cs        |  2 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |  2 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |  2 +-
 .../Impl/Transactions/TransactionImpl.cs        |  3 +-
 17 files changed, 103 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/.gitignore
----------------------------------------------------------------------
diff --git a/.gitignore b/.gitignore
index 535a8ff..47220b2 100644
--- a/.gitignore
+++ b/.gitignore
@@ -29,6 +29,7 @@ git-patch-prop-local.sh
 **/dotnet/libs/
 *.classname*
 *.exe
+.mvn/
 
 #Visual Studio files
 *.[Oo]bj

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
index 25700c6..08c44a6 100644
--- a/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.AspNet.Tests/IgniteSessionStateStoreProviderTest.cs
@@ -28,6 +28,7 @@ namespace Apache.Ignite.AspNet.Tests
     using System.Web.SessionState;
     using Apache.Ignite.Core;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Tests;
     using NUnit.Framework;
 
@@ -265,7 +266,7 @@ namespace Apache.Ignite.AspNet.Tests
             Assert.AreEqual(SessionStateActions.None, actions);
 
             // Try to get it in a different thread.
-            Task.Factory.StartNew(() =>
+            TaskRunner.Run(() =>
             {
                 object lockId1;   // do not overwrite lockId
                 res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId1, out actions);
@@ -277,7 +278,7 @@ namespace Apache.Ignite.AspNet.Tests
             }).Wait();
 
             // Try to get it in a different thread.
-            Task.Factory.StartNew(() =>
+            TaskRunner.Run(() =>
             {
                 object lockId1;   // do not overwrite lockId
                 res = provider.GetItemExclusive(HttpContext, Id, out locked, out lockAge, out lockId1, out actions);
@@ -292,7 +293,7 @@ namespace Apache.Ignite.AspNet.Tests
             provider.ReleaseItemExclusive(HttpContext, Id, lockId);
 
             // Make sure it is accessible in a different thread.
-            Task.Factory.StartNew(() =>
+            TaskRunner.Run(() =>
             {
                 res = provider.GetItem(HttpContext, Id, out locked, out lockAge, out lockId, out actions);
                 Assert.IsNotNull(res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs
index e152ffb..fb2fbd2 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Benchmarks/BenchmarkRunner.cs
@@ -22,7 +22,6 @@ namespace Apache.Ignite.Benchmarks
     using System.IO;
     using System.Text;
     using Apache.Ignite.Benchmarks.Interop;
-    using Apache.Ignite.Benchmarks.ThinClient;
 
     /// <summary>
     /// Benchmark runner.

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs
index e635bd1..272a0ca 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Binary/BinaryDynamicRegistrationTest.cs
@@ -399,7 +399,7 @@ namespace Apache.Ignite.Core.Tests.Binary
                         };
 
                         var tasks = Enumerable.Range(0, threads)
-                            .Select(x => Task.Factory.StartNew(registerType))
+                            .Select(x => TaskRunner.Run(registerType))
                             .ToArray();
 
                         Task.WaitAll(tasks);

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
index 2602a02..3d0168c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheAbstractTransactionalTest.cs
@@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Tests.Cache
     using System.Transactions;
     using Apache.Ignite.Core.Cache;
     using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
 
@@ -563,8 +564,8 @@ namespace Apache.Ignite.Core.Tests.Cache
             var aex = Assert.Throws<AggregateException>(() =>
                 Task.WaitAll(new[]
                     {
-                        Task.Factory.StartNew(() => increment(keys0)),
-                        Task.Factory.StartNew(() => increment(keys0.Reverse().ToArray()))
+                        TaskRunner.Run(() => increment(keys0)),
+                        TaskRunner.Run(() => increment(keys0.Reverse().ToArray()))
                     },
                     TimeSpan.FromSeconds(40)));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 2ea17a8..cb30f40 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -31,6 +31,7 @@ namespace Apache.Ignite.Core.Tests.Client
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Configuration;
+    using Apache.Ignite.Core.Impl.Common;
     using NUnit.Framework;
 
     /// <summary>
@@ -310,7 +311,7 @@ namespace Apache.Ignite.Core.Tests.Client
             var evt = new ManualResetEventSlim();
             var ignite = Ignition.Start(TestUtils.GetTestConfiguration());
 
-            var putGetTask = Task.Factory.StartNew(() =>
+            var putGetTask = TaskRunner.Run(() =>
             {
                 using (var client = StartClient())
                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
index a7c0534..e9bac02 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/EventsTest.cs
@@ -34,6 +34,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Events;
     using Apache.Ignite.Core.Impl;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Impl.Events;
     using Apache.Ignite.Core.Resource;
     using Apache.Ignite.Core.Tests.Compute;
@@ -385,14 +386,14 @@ namespace Apache.Ignite.Core.Tests
         /// </summary>
         private static IEnumerable<Func<IEventFilter<IEvent>, int[], Task<IEvent>>> GetWaitTasks(IEvents events)
         {
-            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types));
-            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(types.ToList()));
+            yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(types));
+            yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(types.ToList()));
 
             yield return (filter, types) => events.WaitForLocalAsync(types);
             yield return (filter, types) => events.WaitForLocalAsync(types.ToList());
 
-            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types));
-            yield return (filter, types) => Task.Factory.StartNew(() => events.WaitForLocal(filter, types.ToList()));
+            yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(filter, types));
+            yield return (filter, types) => TaskRunner.Run(() => events.WaitForLocal(filter, types.ToList()));
 
             yield return (filter, types) => events.WaitForLocalAsync(filter, types);
             yield return (filter, types) => events.WaitForLocalAsync(filter, types.ToList());

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
index f7568ef..0b06ea3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ExceptionsTest.cs
@@ -29,6 +29,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Services;
     using Apache.Ignite.Core.Transactions;
     using NUnit.Framework;
@@ -348,7 +349,7 @@ namespace Apache.Ignite.Core.Tests
                     cache = cache.WithKeepBinary<TK, int>();
 
                 // Do cache puts in parallel
-                var putTask = Task.Factory.StartNew(() =>
+                var putTask = TaskRunner.Run(() =>
                 {
                     try
                     {

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
index 792b33d..f9c1cad 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteStartStopTest.cs
@@ -23,6 +23,7 @@ namespace Apache.Ignite.Core.Tests
     using System.Threading;
     using System.Threading.Tasks;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Messaging;
     using Apache.Ignite.Core.Tests.Process;
     using NUnit.Framework;
@@ -207,7 +208,7 @@ namespace Apache.Ignite.Core.Tests
 
                 if (i % 2 == 0) // Try to stop ignite from another thread.
                 {
-                    Task.Factory.StartNew(() => grid.Dispose()).Wait();
+                    TaskRunner.Run(() => grid.Dispose()).Wait();
                 }
                 else
                 {
@@ -306,7 +307,7 @@ namespace Apache.Ignite.Core.Tests
 
             // Spam message subscriptions on a separate thread 
             // to test race conditions during processor init on remote node
-            var listenTask = Task.Factory.StartNew(() =>
+            var listenTask = TaskRunner.Run(() =>
             {
                 var filter = new MessageListener();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
index e644e31..7db4eef 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/MessagingTest.cs
@@ -27,6 +27,7 @@ namespace Apache.Ignite.Core.Tests
     using Apache.Ignite.Core.Cache.Configuration;
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Messaging;
     using Apache.Ignite.Core.Resource;
     using Apache.Ignite.Core.Tests.Cache;
@@ -252,7 +253,7 @@ namespace Apache.Ignite.Core.Tests
 
             var messaging = _grid1.GetMessaging();
 
-            var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() =>
+            var senders = TaskRunner.Run(() => TestUtils.RunMultiThreaded(() =>
             {
                 messaging.Send(NextMessage());
                 Thread.Sleep(50);
@@ -423,7 +424,7 @@ namespace Apache.Ignite.Core.Tests
 
             var messaging = _grid1.GetMessaging();
 
-            var senders = Task.Factory.StartNew(() => TestUtils.RunMultiThreaded(() =>
+            var senders = TaskRunner.Run(() => TestUtils.RunMultiThreaded(() =>
             {
                 MessagingTestHelper.ClearReceived(int.MaxValue);
                 messaging.Send(NextMessage());

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index ec84a38..93c45c3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -72,6 +72,7 @@
     <Compile Include="Client\IgniteClientException.cs" />
     <Compile Include="Client\IIgniteClient.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
+    <Compile Include="Impl\Common\TaskRunner.cs" />
     <Compile Include="Ssl\ISslContextFactory.cs" />
     <Compile Include="Configuration\Package-Info.cs" />
     <Compile Include="Configuration\ClientConnectorConfiguration.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index 27d8f0b..bce681f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -28,11 +28,11 @@ namespace Apache.Ignite.Core.Impl.Client
     using System.Net.Sockets;
     using System.Threading;
     using System.Threading.Tasks;
-    using System.Xml.Schema;
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
     using Apache.Ignite.Core.Impl.Binary.IO;
+    using Apache.Ignite.Core.Impl.Common;
 
     /// <summary>
     /// Wrapper over framework socket for Ignite thin client operations.
@@ -117,7 +117,7 @@ namespace Apache.Ignite.Core.Impl.Client
             }
 
             // Continuously and asynchronously wait for data from server.
-            Task.Factory.StartNew(WaitForMessages);
+            TaskRunner.Run(WaitForMessages);
         }
 
         /// <summary>
@@ -174,7 +174,7 @@ namespace Apache.Ignite.Core.Impl.Client
             var task = SendRequestAsync(ref reqMsg);
 
             // Decode.
-            return task.ContinueWith(responseTask => DecodeResponse(responseTask.Result, readFunc, errorFunc));
+            return task.ContWith(responseTask => DecodeResponse(responseTask.Result, readFunc, errorFunc));
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs
new file mode 100644
index 0000000..51a7c6a
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/TaskRunner.cs
@@ -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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using System;
+    using System.Threading;
+    using System.Threading.Tasks;
+
+    /// <summary>
+    /// Extensions for <see cref="Task"/> classes.
+    /// Fixes the issue with <see cref="TaskScheduler.Current"/> being used by defaut by system APIs.
+    /// </summary>
+    internal static class TaskRunner
+    {
+        /// <summary>
+        /// ContinueWith using default scheduler.
+        /// </summary>
+        public static Task<TNewResult> ContWith<TResult, TNewResult>(this Task<TResult> task,
+            Func<Task<TResult>, TNewResult> continuationFunction)
+        {
+            IgniteArgumentCheck.NotNull(task, "task");
+            
+            return task.ContinueWith(continuationFunction, TaskScheduler.Default);
+        }
+        
+        /// <summary>
+        /// ContinueWith using default scheduler.
+        /// </summary>
+        public static Task ContWith(this Task task,
+            Action<Task> continuationFunction)
+        {
+            IgniteArgumentCheck.NotNull(task, "task");
+            
+            return task.ContinueWith(continuationFunction, TaskScheduler.Default);
+        }
+
+        /// <summary>
+        /// Run new task using default scheduler.
+        /// </summary>
+        public static Task Run(Action action)
+        {
+            return Task.Factory.StartNew(action, CancellationToken.None, TaskCreationOptions.None, 
+                TaskScheduler.Default);
+        }
+        
+        /// <summary>
+        /// Run new task using default scheduler.
+        /// </summary>
+        public static Task<TResult> Run<TResult>(Func<TResult> func)
+        {
+            return Task.Factory.StartNew(func, CancellationToken.None, TaskCreationOptions.None, 
+                TaskScheduler.Default);
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
index 38a8ea8..0026701 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerBatch.cs
@@ -69,7 +69,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
             if (prev != null)
                 Thread.MemoryBarrier(); // Prevent "prev" field escape.
 
-            _fut.Task.ContinueWith(x => ParentsCompleted());
+            _fut.Task.ContWith(x => ParentsCompleted());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index 555c6e6..7aaa84a 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -897,7 +897,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
             /// </summary>
             public void RunThread()
             {
-                Task.Factory.StartNew(Run);
+                TaskRunner.Run(Run);
             }
         }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
index a81523a..04cc210 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Events/Events.cs
@@ -241,7 +241,7 @@ namespace Apache.Ignite.Core.Impl.Events
                 if (hnd != null)
                 {
                     // Dispose handle as soon as future ends.
-                    task.ContinueWith(x => Ignite.HandleRegistry.Release(hnd.Value));
+                    task.ContWith(x => Ignite.HandleRegistry.Release(hnd.Value));
                 }
 
                 return task;

http://git-wip-us.apache.org/repos/asf/ignite/blob/44c4656c/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
index 0b04a68..c800859 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionImpl.cs
@@ -21,6 +21,7 @@ namespace Apache.Ignite.Core.Impl.Transactions
     using System.Globalization;
     using System.Threading;
     using System.Threading.Tasks;
+    using Apache.Ignite.Core.Impl.Common;
     using Apache.Ignite.Core.Transactions;
 
     /// <summary>
@@ -457,7 +458,7 @@ namespace Apache.Ignite.Core.Impl.Transactions
         /// </summary>
         private Task CloseWhenComplete(Task task)
         {
-            return task.ContinueWith(x => Close());
+            return task.ContWith(x => Close());
         }
 
         /** <inheritdoc /> */


[09/50] [abbrv] ignite git commit: IGNITE-4091 Web Console: Refactored using of internal Angular API.

Posted by ag...@apache.org.
IGNITE-4091 Web Console: Refactored using of internal Angular API.


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

Branch: refs/heads/ignite-7708
Commit: 74d254564a44a95db9945652c9b579ed6b431ee9
Parents: 9752466
Author: Alexander Kalinin <ve...@yandex.ru>
Authored: Wed Apr 11 17:09:41 2018 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Apr 11 17:09:41 2018 +0700

----------------------------------------------------------------------
 modules/web-console/frontend/app/app.config.js  | 14 +++---
 .../components/modal-import-models/component.js |  4 +-
 .../app/components/page-profile/controller.js   |  4 +-
 .../frontend/app/modules/ace.module.js          | 47 ++++++++++----------
 .../services/AngularStrapSelect.decorator.js    |  5 ++-
 .../services/AngularStrapTooltip.decorator.js   |  8 ++--
 .../frontend/app/services/FormUtils.service.js  |  3 +-
 7 files changed, 45 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/app.config.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/app.config.js b/modules/web-console/frontend/app/app.config.js
index 9d8dc99..e2bc057 100644
--- a/modules/web-console/frontend/app/app.config.js
+++ b/modules/web-console/frontend/app/app.config.js
@@ -43,7 +43,7 @@ igniteConsoleCfg.config(['$animateProvider', ($animateProvider) => {
 
 // AngularStrap modal popup configuration.
 igniteConsoleCfg.config(['$modalProvider', ($modalProvider) => {
-    angular.extend($modalProvider.defaults, {
+    Object.assign($modalProvider.defaults, {
         animation: 'am-fade-and-scale',
         placement: 'center',
         html: true
@@ -52,7 +52,7 @@ igniteConsoleCfg.config(['$modalProvider', ($modalProvider) => {
 
 // AngularStrap popover configuration.
 igniteConsoleCfg.config(['$popoverProvider', ($popoverProvider) => {
-    angular.extend($popoverProvider.defaults, {
+    Object.assign($popoverProvider.defaults, {
         trigger: 'manual',
         placement: 'right',
         container: 'body',
@@ -62,7 +62,7 @@ igniteConsoleCfg.config(['$popoverProvider', ($popoverProvider) => {
 
 // AngularStrap tooltips configuration.
 igniteConsoleCfg.config(['$tooltipProvider', ($tooltipProvider) => {
-    angular.extend($tooltipProvider.defaults, {
+    Object.assign($tooltipProvider.defaults, {
         container: 'body',
         delay: {show: 150, hide: 150},
         placement: 'right',
@@ -73,7 +73,7 @@ igniteConsoleCfg.config(['$tooltipProvider', ($tooltipProvider) => {
 
 // AngularStrap select (combobox) configuration.
 igniteConsoleCfg.config(['$selectProvider', ($selectProvider) => {
-    angular.extend($selectProvider.defaults, {
+    Object.assign($selectProvider.defaults, {
         container: 'body',
         maxLength: '5',
         allText: 'Select All',
@@ -87,7 +87,7 @@ igniteConsoleCfg.config(['$selectProvider', ($selectProvider) => {
 
 // AngularStrap alerts configuration.
 igniteConsoleCfg.config(['$alertProvider', ($alertProvider) => {
-    angular.extend($alertProvider.defaults, {
+    Object.assign($alertProvider.defaults, {
         container: 'body',
         placement: 'top-right',
         duration: '5',
@@ -99,7 +99,7 @@ igniteConsoleCfg.config(['$alertProvider', ($alertProvider) => {
 
 // AngularStrap dropdowns () configuration.
 igniteConsoleCfg.config(['$dropdownProvider', ($dropdownProvider) => {
-    angular.extend($dropdownProvider.defaults, {
+    Object.assign($dropdownProvider.defaults, {
         templateUrl: dropdownTemplateUrl,
         animation: ''
     });
@@ -107,7 +107,7 @@ igniteConsoleCfg.config(['$dropdownProvider', ($dropdownProvider) => {
 
 // AngularStrap dropdowns () configuration.
 igniteConsoleCfg.config(['$datepickerProvider', ($datepickerProvider) => {
-    angular.extend($datepickerProvider.defaults, {
+    Object.assign($datepickerProvider.defaults, {
         autoclose: true,
         iconLeft: 'icon-datepicker-left',
         iconRight: 'icon-datepicker-right'

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js
index 7f852b0..813c998 100644
--- a/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js
+++ b/modules/web-console/frontend/app/components/page-configure/components/modal-import-models/component.js
@@ -84,7 +84,7 @@ const DFLT_REPLICATED_CACHE = {
 const CACHE_TEMPLATES = [DFLT_PARTITIONED_CACHE, DFLT_REPLICATED_CACHE];
 
 export class ModalImportModels {
-    /** 
+    /**
      * Cluster ID to import models into
      * @type {string}
      */
@@ -771,7 +771,7 @@ export class ModalImportModels {
 
                 // Prepare caches for generation.
                 if (table.action === IMPORT_DM_NEW_CACHE) {
-                    const newCache = angular.copy(this.loadedCaches[table.cacheOrTemplate]);
+                    const newCache = _.cloneDeep(this.loadedCaches[table.cacheOrTemplate]);
 
                     batchAction.newCache = newCache;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/components/page-profile/controller.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/components/page-profile/controller.js b/modules/web-console/frontend/app/components/page-profile/controller.js
index 05fe118..c67a603 100644
--- a/modules/web-console/frontend/app/components/page-profile/controller.js
+++ b/modules/web-console/frontend/app/components/page-profile/controller.js
@@ -15,6 +15,8 @@
  * limitations under the License.
  */
 
+import _ from 'lodash';
+
 export default class PageProfileController {
     static $inject = [
         '$rootScope', '$scope', '$http', 'IgniteLegacyUtils', 'IgniteMessages', 'IgniteFocus', 'IgniteConfirm', 'IgniteCountries', 'User'
@@ -28,7 +30,7 @@ export default class PageProfileController {
         this.ui = {};
 
         this.User.read()
-            .then((user) => this.ui.user = angular.copy(user));
+            .then((user) => this.ui.user = _.cloneDeep(user));
 
         this.ui.countries = this.Countries.getAll();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/modules/ace.module.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/modules/ace.module.js b/modules/web-console/frontend/app/modules/ace.module.js
index a28536a..6a6e70a 100644
--- a/modules/web-console/frontend/app/modules/ace.module.js
+++ b/modules/web-console/frontend/app/modules/ace.module.js
@@ -16,12 +16,13 @@
  */
 
 import angular from 'angular';
+import _ from 'lodash';
 
 angular
     .module('ignite-console.ace', [])
     .constant('igniteAceConfig', {})
     .directive('igniteAce', ['igniteAceConfig', (aceConfig) => {
-        if (angular.isUndefined(window.ace))
+        if (_.isUndefined(window.ace))
             throw new Error('ignite-ace need ace to work... (o rly?)');
 
         /**
@@ -43,7 +44,7 @@ angular
          */
         const setOptions = (acee, session, opts) => {
             // Sets the ace worker path, if running from concatenated or minified source.
-            if (angular.isDefined(opts.workerPath)) {
+            if (!_.isUndefined(opts.workerPath)) {
                 const config = window.ace.acequire('ace/config');
 
                 config.set('workerPath', opts.workerPath);
@@ -53,26 +54,26 @@ angular
             _.forEach(opts.require, (n) => window.ace.acequire(n));
 
             // Boolean options.
-            if (angular.isDefined(opts.showGutter))
+            if (!_.isUndefined(opts.showGutter))
                 acee.renderer.setShowGutter(opts.showGutter);
 
-            if (angular.isDefined(opts.useWrapMode))
+            if (!_.isUndefined(opts.useWrapMode))
                 session.setUseWrapMode(opts.useWrapMode);
 
-            if (angular.isDefined(opts.showInvisibles))
+            if (!_.isUndefined(opts.showInvisibles))
                 acee.renderer.setShowInvisibles(opts.showInvisibles);
 
-            if (angular.isDefined(opts.showIndentGuides))
+            if (!_.isUndefined(opts.showIndentGuides))
                 acee.renderer.setDisplayIndentGuides(opts.showIndentGuides);
 
-            if (angular.isDefined(opts.useSoftTabs))
+            if (!_.isUndefined(opts.useSoftTabs))
                 session.setUseSoftTabs(opts.useSoftTabs);
 
-            if (angular.isDefined(opts.showPrintMargin))
+            if (!_.isUndefined(opts.showPrintMargin))
                 acee.setShowPrintMargin(opts.showPrintMargin);
 
             // Commands.
-            if (angular.isDefined(opts.disableSearch) && opts.disableSearch) {
+            if (!_.isUndefined(opts.disableSearch) && opts.disableSearch) {
                 acee.commands.addCommands([{
                     name: 'unfind',
                     bindKey: {
@@ -85,21 +86,21 @@ angular
             }
 
             // Base options.
-            if (angular.isString(opts.theme))
+            if (_.isString(opts.theme))
                 acee.setTheme('ace/theme/' + opts.theme);
 
-            if (angular.isString(opts.mode))
+            if (_.isString(opts.mode))
                 session.setMode('ace/mode/' + opts.mode);
 
-            if (angular.isDefined(opts.firstLineNumber)) {
-                if (angular.isNumber(opts.firstLineNumber))
+            if (!_.isUndefined(opts.firstLineNumber)) {
+                if (_.isNumber(opts.firstLineNumber))
                     session.setOption('firstLineNumber', opts.firstLineNumber);
-                else if (angular.isFunction(opts.firstLineNumber))
+                else if (_.isFunction(opts.firstLineNumber))
                     session.setOption('firstLineNumber', opts.firstLineNumber());
             }
 
             // Advanced options.
-            if (angular.isDefined(opts.advanced)) {
+            if (!_.isUndefined(opts.advanced)) {
                 for (const key in opts.advanced) {
                     if (opts.advanced.hasOwnProperty(key)) {
                         // Create a javascript object with the key and value.
@@ -112,7 +113,7 @@ angular
             }
 
             // Advanced options for the renderer.
-            if (angular.isDefined(opts.rendererOptions)) {
+            if (!_.isUndefined(opts.rendererOptions)) {
                 for (const key in opts.rendererOptions) {
                     if (opts.rendererOptions.hasOwnProperty(key)) {
                         // Create a javascript object with the key and value.
@@ -126,7 +127,7 @@ angular
 
             // onLoad callbacks.
             _.forEach(opts.callbacks, (cb) => {
-                if (angular.isFunction(cb))
+                if (_.isFunction(cb))
                     cb(acee);
             });
         };
@@ -147,7 +148,7 @@ angular
                  *
                  * @type object
                  */
-                let opts = angular.extend({}, options, scope.$eval(attrs.igniteAce));
+                let opts = Object.assign({}, options, scope.$eval(attrs.igniteAce));
 
                 /**
                  * ACE editor.
@@ -191,9 +192,9 @@ angular
                             !scope.$$phase && !scope.$root.$$phase)
                             scope.$eval(() => ngModel.$setViewValue(newValue));
 
-                        if (angular.isDefined(callback)) {
+                        if (!_.isUndefined(callback)) {
                             scope.$evalAsync(() => {
-                                if (angular.isFunction(callback))
+                                if (_.isFunction(callback))
                                     callback([e, acee]);
                                 else
                                     throw new Error('ignite-ace use a function as callback');
@@ -210,10 +211,10 @@ angular
                     form && form.$removeControl(ngModel);
 
                     ngModel.$formatters.push((value) => {
-                        if (angular.isUndefined(value) || value === null)
+                        if (_.isUndefined(value) || value === null)
                             return '';
 
-                        if (angular.isObject(value) || angular.isArray(value))
+                        if (_.isObject(value) || _.isArray(value))
                             throw new Error('ignite-ace cannot use an object or an array as a model');
 
                         return value;
@@ -229,7 +230,7 @@ angular
                     if (current === previous)
                         return;
 
-                    opts = angular.extend({}, options, scope.$eval(attrs.igniteAce));
+                    opts = Object.assign({}, options, scope.$eval(attrs.igniteAce));
 
                     opts.callbacks = [opts.onLoad];
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js b/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js
index 39f7ccd..32fa167 100644
--- a/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js
+++ b/modules/web-console/frontend/app/services/AngularStrapSelect.decorator.js
@@ -16,6 +16,7 @@
  */
 
 import angular from 'angular';
+import _ from 'lodash';
 
 /**
  * Special decorator that fix problem in AngularStrap selectAll / deselectAll methods.
@@ -27,12 +28,12 @@ export default angular.module('mgcrea.ngStrap.select')
             const delegate = $delegate(element, controller, config);
 
             // Common vars.
-            const options = angular.extend({}, $delegate.defaults, config);
+            const options = Object.assign({}, $delegate.defaults, config);
 
             const scope = delegate.$scope;
 
             const valueByIndex = (index) => {
-                if (angular.isUndefined(scope.$matches[index]))
+                if (_.isUndefined(scope.$matches[index]))
                     return null;
 
                 return scope.$matches[index].value;

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js b/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js
index d01a450..fa59f32 100644
--- a/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js
+++ b/modules/web-console/frontend/app/services/AngularStrapTooltip.decorator.js
@@ -16,7 +16,7 @@
  */
 
 import angular from 'angular';
-import flow from 'lodash/flow';
+import _ from 'lodash';
 
 /**
  * Decorator that fix problem in AngularStrap $tooltip.
@@ -62,7 +62,7 @@ export default angular
 
                 scope.$emit(options.prefixEvent + '.hide.before', $tooltip);
 
-                if (angular.isDefined(options.onBeforeHide) && angular.isFunction(options.onBeforeHide))
+                if (!_.isUndefined(options.onBeforeHide) && _.isFunction(options.onBeforeHide))
                     options.onBeforeHide($tooltip);
 
                 $tooltip.$isShown = scope.$isShown = false;
@@ -82,8 +82,8 @@ export default angular
             const $tooltip = $delegate(el, config);
 
             $tooltip.$referenceElement = el;
-            $tooltip.destroy = flow($tooltip.destroy, () => $tooltip.$referenceElement = null);
-            $tooltip.$applyPlacement = flow($tooltip.$applyPlacement, () => {
+            $tooltip.destroy = _.flow($tooltip.destroy, () => $tooltip.$referenceElement = null);
+            $tooltip.$applyPlacement = _.flow($tooltip.$applyPlacement, () => {
                 if (!$tooltip.$element)
                     return;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/74d25456/modules/web-console/frontend/app/services/FormUtils.service.js
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/services/FormUtils.service.js b/modules/web-console/frontend/app/services/FormUtils.service.js
index f22d4bc..da1d737 100644
--- a/modules/web-console/frontend/app/services/FormUtils.service.js
+++ b/modules/web-console/frontend/app/services/FormUtils.service.js
@@ -14,6 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+import _ from 'lodash';
 
 export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) => {
     function ensureActivePanel(ui, pnl, focusId) {
@@ -41,7 +42,7 @@ export default ['IgniteFormUtils', ['$window', 'IgniteFocus', ($window, Focus) =
                 if (!activePanels || activePanels.length < 1)
                     ui.activePanels = [idx];
                 else if (!_.includes(activePanels, idx)) {
-                    const newActivePanels = angular.copy(activePanels);
+                    const newActivePanels = _.cloneDeep(activePanels);
 
                     newActivePanels.push(idx);
 


[49/50] [abbrv] ignite git commit: Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-7708

Posted by ag...@apache.org.
Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-7708


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

Branch: refs/heads/ignite-7708
Commit: ad80570425765fc930fd6774f123c176535d3d58
Parents: 83be65e 7586cfd
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Apr 16 15:37:13 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Apr 16 15:37:13 2018 +0300

----------------------------------------------------------------------
 .gitignore                                      |     1 +
 assembly/release-fabric-base.xml                |     1 +
 examples/README-LGPL.txt                        |     1 +
 examples/README.txt                             |     1 +
 examples/pom.xml                                |     9 +-
 examples/sql/world.sql                          |  5354 ++++++
 .../examples/client/ClientPutGetExample.java    |    69 +
 .../AlgorithmSpecificDatasetExample.java        |    12 +-
 .../ml/dataset/CacheBasedDatasetExample.java    |     3 +-
 .../ml/dataset/LocalDatasetExample.java         |     3 +-
 .../ignite/examples/ml/genetic/change/Coin.java |    89 +
 .../OptimizeMakeChangeFitnessFunction.java      |   105 +
 .../change/OptimizeMakeChangeGAExample.java     |   218 +
 .../OptimizeMakeChangeTerminateCriteria.java    |    84 +
 .../helloworld/HelloWorldFitnessFunction.java   |    63 +
 .../genetic/helloworld/HelloWorldGAExample.java |   127 +
 .../helloworld/HelloWorldTerminateCriteria.java |    89 +
 .../ignite/examples/ml/genetic/movie/Movie.java |   133 +
 .../ml/genetic/movie/MovieFitnessFunction.java  |   104 +
 .../ml/genetic/movie/MovieGAExample.java        |   274 +
 .../genetic/movie/MovieTerminateCriteria.java   |    88 +
 .../ml/knn/KNNClassificationExample.java        |   273 +
 .../examples/ml/knn/KNNRegressionExample.java   |   310 +
 .../KNNClassificationExample.java               |   154 -
 .../ml/knn/classification/package-info.java     |    22 -
 .../ml/knn/regression/KNNRegressionExample.java |   155 -
 .../ml/knn/regression/package-info.java         |    22 -
 .../examples/ml/nn/MLPGroupTrainerExample.java  |   140 -
 .../examples/ml/nn/MLPLocalTrainerExample.java  |   161 -
 .../examples/ml/nn/MLPTrainerExample.java       |   167 +
 .../ml/preprocessing/NormalizationExample.java  |    18 +-
 ...dLinearRegressionWithLSQRTrainerExample.java |   170 -
 ...tedLinearRegressionWithQRTrainerExample.java |   136 -
 ...edLinearRegressionWithSGDTrainerExample.java |   137 -
 .../LinearRegressionLSQRTrainerExample.java     |   169 +
 ...sionLSQRTrainerWithNormalizationExample.java |   180 +
 .../LinearRegressionSGDTrainerExample.java      |   176 +
 .../ml/svm/SVMBinaryClassificationExample.java  |   131 -
 .../ml/svm/SVMMultiClassificationExample.java   |   130 -
 .../binary/SVMBinaryClassificationExample.java  |   233 +
 .../examples/ml/svm/binary/package-info.java    |    22 +
 .../SVMMultiClassClassificationExample.java     |   289 +
 .../ml/svm/multiclass/package-info.java         |    22 +
 .../ignite/examples/ml/svm/package-info.java    |    22 +
 ...ecisionTreeClassificationTrainerExample.java |   148 +
 .../DecisionTreeRegressionTrainerExample.java   |   124 +
 .../ignite/examples/ml/tree/package-info.java   |    22 +
 .../examples/ml/trees/DecisionTreesExample.java |   354 -
 .../ignite/examples/ml/trees/package-info.java  |    22 -
 .../ignite/examples/sql/SqlJdbcCopyExample.java |    70 +-
 examples/src/main/resources/cityBulkLoad.csv    |     3 -
 .../src/main/resources/datasets/titanic.txt     |  1309 --
 examples/src/main/resources/personBulkLoad.csv  |     4 -
 examples/src/main/resources/sql/city.csv        |  4079 +++++
 .../spark/IgniteDataFrameWriteExample.scala     |     4 +-
 .../spark/examples/IgniteDataFrameSelfTest.java |     3 +-
 .../examples/SharedRDDExampleSelfTest.java      |     3 +-
 .../IgniteExamplesSparkSelfTestSuite.java       |     2 +-
 .../client/impl/ClientCacheFlagsCodecTest.java  |    35 +-
 .../client/suite/IgniteClientTestSuite.java     |     2 +
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |     6 +-
 .../internal/jdbc2/JdbcStreamingSelfTest.java   |     2 +
 .../JettyRestProcessorAbstractSelfTest.java     |   307 +-
 ...ettyRestProcessorAuthenticationSelfTest.java |   172 +
 .../internal/processors/rest/SimplePerson.java  |     4 +-
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |    12 +-
 .../JdbcThinAuthenticateConnectionSelfTest.java |   235 +
 .../thin/JdbcThinBulkLoadAbstractSelfTest.java  |   207 +-
 ...JdbcThinConnectionMultipleAddressesTest.java |   424 +
 .../jdbc/thin/JdbcThinConnectionSSLTest.java    |    20 +-
 .../jdbc/thin/JdbcThinConnectionSelfTest.java   |   265 +-
 .../jdbc/thin/JdbcThinErrorsSelfTest.java       |     2 +-
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |    22 +-
 .../jdbc/thin/JdbcThinStreamingSelfTest.java    |   237 +-
 .../ignite/jdbc/thin/JdbcThinTcpIoTest.java     |     8 +-
 .../thin/JdbcThinWalModeChangeSelfTest.java     |    11 +-
 .../IgniteUuidCompatibilityTest.java            |   189 +
 .../junits/IgniteCompatibilityAbstractTest.java |     4 +-
 .../junits/IgniteCompatibilityNodeRunner.java   |     5 +-
 .../util/CompatibilityTestsUtils.java           |    93 +
 .../IgniteCompatibilityBasicTestSuite.java      |     3 +
 modules/core/licenses/jBCrypt-license.txt       |    17 +
 .../org/apache/ignite/IgniteJdbcThinDriver.java |    91 +-
 .../java/org/apache/ignite/IgniteState.java     |     7 +-
 .../apache/ignite/IgniteSystemProperties.java   |    65 +-
 .../main/java/org/apache/ignite/Ignition.java   |    19 +
 .../ignite/binary/BinaryTypeConfiguration.java  |     6 +-
 .../org/apache/ignite/cache/CacheMetrics.java   |    12 +
 .../org/apache/ignite/cache/QueryEntity.java    |    41 +-
 .../cache/query/annotations/QuerySqlField.java  |    16 +-
 .../client/ClientAuthenticationException.java   |    33 +
 .../client/ClientAuthorizationException.java    |    46 +
 .../org/apache/ignite/client/ClientCache.java   |   356 +
 .../ignite/client/ClientCacheConfiguration.java |   656 +
 .../client/ClientConnectionException.java       |    45 +
 .../apache/ignite/client/ClientException.java   |    78 +
 .../org/apache/ignite/client/IgniteClient.java  |    91 +
 .../java/org/apache/ignite/client/SslMode.java  |    26 +
 .../org/apache/ignite/client/SslProtocol.java   |    28 +
 .../ignite/compute/ComputeTaskAdapter.java      |     2 +-
 .../configuration/BinaryConfiguration.java      |     6 +-
 .../configuration/CacheConfiguration.java       |    37 +
 .../configuration/ClientConfiguration.java      |   400 +
 .../CommunicationFailureContext.java            |    62 +
 .../CommunicationFailureResolver.java           |    28 +
 .../configuration/DataStorageConfiguration.java |    10 +-
 .../DefaultCommunicationFailureResolver.java    |   305 +
 .../configuration/IgniteConfiguration.java      |    74 +
 .../apache/ignite/failure/FailureContext.java   |    65 +
 .../apache/ignite/failure/FailureHandler.java   |    39 +
 .../org/apache/ignite/failure/FailureType.java  |    32 +
 .../ignite/failure/NoOpFailureHandler.java      |    30 +
 .../failure/RestartProcessFailureHandler.java   |    45 +
 .../ignite/failure/StopNodeFailureHandler.java  |    43 +
 .../failure/StopNodeOrHaltFailureHandler.java   |   102 +
 .../org/apache/ignite/failure/package-info.java |    22 +
 .../apache/ignite/internal/GridComponent.java   |     5 +-
 .../ignite/internal/GridKernalContext.java      |    43 +-
 .../ignite/internal/GridKernalContextImpl.java  |    55 +-
 .../org/apache/ignite/internal/GridTopic.java   |    11 +-
 .../apache/ignite/internal/IgniteKernal.java    |    27 +-
 .../ignite/internal/IgniteNodeAttributes.java   |     3 +
 .../org/apache/ignite/internal/IgnitionEx.java  |    85 +-
 .../ignite/internal/MarshallerContextImpl.java  |    15 +-
 .../apache/ignite/internal/NodeInvalidator.java |    53 -
 .../ignite/internal/NodeStoppingException.java  |     7 +
 .../binary/BinaryCachingMetadataHandler.java    |    11 +-
 .../internal/binary/BinaryClassDescriptor.java  |   394 +-
 .../ignite/internal/binary/BinaryContext.java   |    18 +
 .../internal/binary/BinaryFieldAccessor.java    |    61 +-
 .../internal/binary/BinaryMetadataHandler.java  |     8 +
 .../binary/BinaryNoopMetadataHandler.java       |     6 +
 .../internal/binary/BinaryObjectExImpl.java     |    69 +-
 .../internal/binary/BinaryObjectImpl.java       |    66 +
 .../ignite/internal/binary/BinaryUtils.java     |     4 +-
 .../internal/client/GridClientCacheFlag.java    |    48 +
 .../impl/connection/GridClientConnection.java   |    18 +-
 .../GridClientConnectionManagerAdapter.java     |    11 +
 .../connection/GridClientNioTcpConnection.java  |     6 +-
 .../impl/GridTcpRouterNioListenerAdapter.java   |    11 +
 .../internal/client/thin/ClientBinary.java      |   167 +
 .../client/thin/ClientBinaryMarshaller.java     |   112 +
 .../internal/client/thin/ClientCacheEntry.java  |    62 +
 .../internal/client/thin/ClientChannel.java     |    46 +
 .../client/thin/ClientChannelConfiguration.java |   242 +
 .../internal/client/thin/ClientError.java       |    71 +
 .../client/thin/ClientFieldsQueryCursor.java    |    40 +
 .../client/thin/ClientFieldsQueryPager.java     |    81 +
 .../internal/client/thin/ClientOperation.java   |    72 +
 .../client/thin/ClientProtocolError.java        |    40 +
 .../internal/client/thin/ClientQueryCursor.java |    98 +
 .../internal/client/thin/ClientQueryPager.java  |    60 +
 .../internal/client/thin/ClientServerError.java |    47 +
 .../internal/client/thin/ClientUtils.java       |   636 +
 .../internal/client/thin/FieldsQueryPager.java  |    30 +
 .../internal/client/thin/GenericQueryPager.java |   139 +
 .../internal/client/thin/ProtocolVersion.java   |   100 +
 .../ignite/internal/client/thin/QueryPager.java |    36 +
 .../internal/client/thin/ReliableChannel.java   |   209 +
 .../ignite/internal/client/thin/Result.java     |    60 +
 .../internal/client/thin/TcpClientCache.java    |   504 +
 .../internal/client/thin/TcpClientChannel.java  |   590 +
 .../internal/client/thin/TcpIgniteClient.java   |   411 +
 .../ignite/internal/commandline/Arguments.java  |    22 +-
 .../ignite/internal/commandline/Command.java    |    65 +
 .../internal/commandline/CommandHandler.java    |   369 +-
 .../ignite/internal/igfs/common/IgfsLogger.java |     8 +-
 .../jdbc/thin/ConnectionProperties.java         |    91 +-
 .../jdbc/thin/ConnectionPropertiesImpl.java     |   486 +-
 .../internal/jdbc/thin/JdbcThinConnection.java  |   140 +-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |     6 +-
 .../jdbc/thin/JdbcThinPreparedStatement.java    |     6 +-
 .../internal/jdbc/thin/JdbcThinResultSet.java   |    14 +-
 .../internal/jdbc/thin/JdbcThinSSLUtil.java     |    10 +-
 .../internal/jdbc/thin/JdbcThinStatement.java   |   125 +-
 .../internal/jdbc/thin/JdbcThinTcpIo.java       |   117 +-
 .../internal/managers/GridManagerAdapter.java   |     8 +
 .../managers/communication/GridIoManager.java   |    22 +-
 .../communication/GridIoMessageFactory.java     |    44 +-
 .../deployment/GridDeploymentLocalStore.java    |    27 +-
 .../discovery/CustomMessageWrapper.java         |     7 +-
 .../internal/managers/discovery/DiscoCache.java |     8 +
 .../discovery/DiscoveryCustomMessage.java       |    10 +-
 .../DiscoveryMessageResultsCollector.java       |   222 +
 .../DiscoveryServerOnlyCustomMessage.java       |    26 +
 .../discovery/GridDiscoveryManager.java         |   197 +-
 .../managers/discovery/IgniteClusterNode.java   |    69 +
 .../managers/discovery/IgniteDiscoverySpi.java  |    67 +
 .../IgniteDiscoverySpiInternalListener.java     |    42 +
 .../optimized/OptimizedObjectInputStream.java   |   103 +-
 .../optimized/OptimizedObjectOutputStream.java  |    94 +-
 .../internal/pagemem/PageIdAllocator.java       |     4 +-
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |    19 +-
 .../pagemem/wal/IgniteWriteAheadLogManager.java |    18 +-
 .../internal/pagemem/wal/StorageException.java  |     2 +-
 .../MetaPageUpdatePartitionDataRecord.java      |     2 +-
 .../processors/GridProcessorAdapter.java        |     1 -
 .../affinity/GridAffinityAssignmentCache.java   |   101 +-
 .../authentication/AuthorizationContext.java    |    89 +
 .../IgniteAccessControlException.java           |    48 +
 .../IgniteAuthenticationProcessor.java          |  1349 ++
 .../processors/authentication/User.java         |   147 +
 .../authentication/UserAcceptedMessage.java     |   103 +
 .../UserAuthenticateRequestMessage.java         |   171 +
 .../UserAuthenticateResponseMessage.java        |   154 +
 .../authentication/UserManagementException.java |    48 +
 .../authentication/UserManagementOperation.java |   105 +
 .../UserManagementOperationFinishedMessage.java |   155 +
 .../authentication/UserProposedMessage.java     |    96 +
 .../processors/bulkload/BulkLoadCsvFormat.java  |     1 -
 .../pipeline/CsvLineProcessorBlock.java         |     5 +-
 .../cache/CacheAffinityChangeMessage.java       |     5 +
 .../cache/CacheAffinitySharedManager.java       |    63 +-
 .../cache/CacheClusterMetricsMXBeanImpl.java    |     5 +
 .../processors/cache/CacheGroupContext.java     |    13 +
 .../cache/CacheLocalMetricsMXBeanImpl.java      |     5 +
 .../processors/cache/CacheMetricsImpl.java      |    18 +-
 .../processors/cache/CacheMetricsSnapshot.java  |     9 +
 .../cache/CacheStatisticsModeChangeMessage.java |     5 +
 .../ClientCacheChangeDiscoveryMessage.java      |     5 +
 .../ClientCacheChangeDummyDiscoveryMessage.java |     5 +
 .../cache/DynamicCacheChangeBatch.java          |     5 +
 .../cache/DynamicCacheDescriptor.java           |    17 +-
 .../cache/GatewayProtectedCacheProxy.java       |   676 +-
 .../processors/cache/GridCacheAdapter.java      |     3 +-
 .../processors/cache/GridCacheContext.java      |    68 +
 .../processors/cache/GridCacheGateway.java      |     7 +
 .../processors/cache/GridCacheIoManager.java    |     6 -
 .../processors/cache/GridCacheMapEntry.java     |   162 +-
 .../processors/cache/GridCacheMvccManager.java  |    45 +-
 .../GridCachePartitionExchangeManager.java      |   103 +-
 .../processors/cache/GridCachePreloader.java    |    16 +-
 .../cache/GridCachePreloaderAdapter.java        |    12 +-
 .../processors/cache/GridCacheProcessor.java    |   179 +-
 .../cache/GridCacheSharedContext.java           |     9 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |    44 +-
 .../processors/cache/GridCacheUtils.java        |    30 +-
 .../cache/GridDeferredAckMessageSender.java     |    10 +-
 .../cache/IgniteCacheOffheapManager.java        |    22 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    |   120 +-
 .../processors/cache/IgniteCacheProxyImpl.java  |    34 +-
 .../cache/IgniteRebalanceIterator.java          |    23 +-
 .../processors/cache/StoredCacheData.java       |     9 +-
 .../processors/cache/WalStateFinishMessage.java |     5 +
 .../processors/cache/WalStateManager.java       |     8 +-
 .../cache/WalStateProposeMessage.java           |     5 +
 .../cache/binary/BinaryMetadataTransport.java   |    24 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |     4 +
 .../binary/MetadataUpdateAcceptedMessage.java   |     5 +
 .../binary/MetadataUpdateProposedMessage.java   |     5 +
 .../GridDistributedTxRemoteAdapter.java         |     2 +-
 .../dht/CacheDistributedGetFutureAdapter.java   |    19 -
 .../dht/GridClientPartitionTopology.java        |    69 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |     4 +
 .../distributed/dht/GridDhtLocalPartition.java  |   492 +-
 .../distributed/dht/GridDhtLockFuture.java      |    19 +-
 .../dht/GridDhtPartitionTopology.java           |    12 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   266 +-
 .../dht/GridDhtPartitionsEvictor.java           |   140 +
 .../dht/GridDhtPartitionsReservation.java       |     6 +-
 .../dht/GridDhtPartitionsStateValidator.java    |   255 +
 .../cache/distributed/dht/GridDhtTxLocal.java   |     5 +
 .../dht/GridPartitionedGetFuture.java           |     2 +-
 .../dht/GridPartitionedSingleGetFuture.java     |    21 +-
 .../dht/atomic/GridDhtAtomicCache.java          |     4 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |     3 +-
 .../CachePartitionFullCountersMap.java          |    22 +
 .../CachePartitionPartialCountersMap.java       |    44 +-
 .../GridDhtPartitionDemandLegacyMessage.java    |   437 +
 .../GridDhtPartitionDemandMessage.java          |   201 +-
 .../dht/preloader/GridDhtPartitionDemander.java |   598 +-
 .../preloader/GridDhtPartitionExchangeId.java   |     3 +
 .../dht/preloader/GridDhtPartitionSupplier.java |   548 +-
 .../GridDhtPartitionSupplyMessage.java          |    22 +-
 .../GridDhtPartitionsExchangeFuture.java        |   169 +-
 .../GridDhtPartitionsSingleMessage.java         |    68 +-
 .../dht/preloader/GridDhtPreloader.java         |   198 +-
 .../IgniteDhtDemandedPartitionsMap.java         |   218 +
 .../IgniteDhtPartitionCountersMap.java          |     6 +
 .../IgniteDhtPartitionHistorySuppliersMap.java  |     6 +
 .../IgniteDhtPartitionsToReloadMap.java         |     6 +
 .../dht/preloader/IgniteHistoricalIterator.java |    39 +
 .../preloader/IgniteRebalanceIteratorImpl.java  |   201 +
 .../dht/preloader/InitNewCoordinatorFuture.java |     2 +-
 .../preloader/latch/ExchangeLatchManager.java   |   696 +
 .../distributed/dht/preloader/latch/Latch.java  |    52 +
 .../dht/preloader/latch/LatchAckMessage.java    |   165 +
 .../distributed/near/GridNearCacheEntry.java    |     3 +-
 .../distributed/near/GridNearGetFuture.java     |     2 +-
 .../cache/distributed/near/GridNearTxLocal.java |    42 +-
 .../cache/local/GridLocalLockFuture.java        |    27 +-
 .../GridCacheDatabaseSharedManager.java         |   308 +-
 .../persistence/GridCacheOffheapManager.java    |   224 +-
 .../IgniteCacheDatabaseSharedManager.java       |    17 +-
 .../cache/persistence/file/AsyncFileIO.java     |     9 +-
 .../cache/persistence/file/FileIO.java          |    20 +-
 .../cache/persistence/file/FileIODecorator.java |     9 +-
 .../persistence/file/FilePageStoreManager.java  |    22 +-
 .../persistence/file/RandomAccessFileIO.java    |    13 +-
 .../cache/persistence/file/UnzipFileIO.java     |     7 +-
 .../persistence/metastorage/MetaStorage.java    |    89 +-
 .../metastorage/ReadOnlyMetastorage.java        |     2 +-
 .../pagemem/CheckpointMetricsTracker.java       |    30 +-
 .../persistence/pagemem/EvictCandidate.java     |    77 -
 .../persistence/pagemem/FullPageIdTable.java    |   358 +-
 .../persistence/pagemem/LoadedPagesMap.java     |   137 +
 .../cache/persistence/pagemem/PageMemoryEx.java |    11 +-
 .../persistence/pagemem/PageMemoryImpl.java     |   367 +-
 .../pagemem/PagesWriteSpeedBasedThrottle.java   |    16 +-
 .../persistence/pagemem/PagesWriteThrottle.java |    42 +-
 .../persistence/pagemem/ReplaceCandidate.java   |    78 +
 .../pagemem/RobinHoodBackwardShiftHashMap.java  |   657 +
 .../TrackingPageIsCorruptedException.java       |    60 +
 .../cache/persistence/tree/BPlusTree.java       |    41 +-
 .../cache/persistence/tree/io/PageMetaIO.java   |     6 +-
 .../persistence/tree/io/TrackingPageIO.java     |   158 +-
 .../wal/FileWriteAheadLogManager.java           |   192 +-
 .../wal/FsyncModeFileWriteAheadLogManager.java  |   161 +-
 .../wal/reader/StandaloneGridKernalContext.java |    30 +-
 .../cache/query/QueryEntityTypeDescriptor.java  |    21 +
 .../CacheContinuousQueryEventBuffer.java        |     5 +-
 .../continuous/CacheContinuousQueryManager.java |    21 +-
 .../cache/store/GridCacheWriteBehindStore.java  |    13 +-
 .../cache/transactions/IgniteTxAdapter.java     |     4 +-
 .../transactions/IgniteTxLocalAdapter.java      |   166 +-
 .../transactions/IgniteTxLocalStateAdapter.java |     8 +-
 .../cache/transactions/IgniteTxManager.java     |    39 +-
 .../cluster/ChangeGlobalStateFinishMessage.java |     5 +
 .../cluster/ChangeGlobalStateMessage.java       |     5 +
 .../cluster/ClusterMetricsUpdateMessage.java    |   158 +
 .../processors/cluster/ClusterNodeMetrics.java  |    62 +
 .../processors/cluster/ClusterProcessor.java    |   249 +-
 .../cluster/GridClusterStateProcessor.java      |    67 +-
 .../continuous/AbstractContinuousMessage.java   |     5 +
 .../continuous/ContinuousRoutineInfo.java       |   100 +
 .../ContinuousRoutineStartResultMessage.java    |   206 +
 .../ContinuousRoutinesCommonDiscoveryData.java  |    45 +
 .../continuous/ContinuousRoutinesInfo.java      |   132 +
 ...tinuousRoutinesJoiningNodeDiscoveryData.java |    45 +
 .../continuous/GridContinuousBatchAdapter.java  |     5 +-
 .../continuous/GridContinuousProcessor.java     |   905 +-
 .../continuous/StartRequestDataV2.java          |   164 +
 .../StartRoutineDiscoveryMessageV2.java         |    77 +
 .../StopRoutineAckDiscoveryMessage.java         |     5 +
 .../datastreamer/DataStreamProcessor.java       |     3 +
 .../datastreamer/DataStreamerImpl.java          |    70 +-
 .../datastructures/DataStructuresProcessor.java |    32 +
 .../processors/failure/FailureProcessor.java    |   124 +
 .../processors/failure/package-info.java        |    22 +
 .../processors/igfs/IgfsMetaManager.java        |    30 +-
 .../internal/processors/job/GridJobWorker.java  |    10 +-
 .../GridMarshallerMappingProcessor.java         |    31 +-
 .../marshaller/MappingAcceptedMessage.java      |     5 +
 .../marshaller/MappingProposedMessage.java      |     5 +
 .../odbc/ClientListenerConnectionContext.java   |     5 +-
 .../odbc/ClientListenerNioListener.java         |    34 +-
 .../odbc/jdbc/JdbcBatchExecuteRequest.java      |    40 +-
 .../odbc/jdbc/JdbcBulkLoadBatchRequest.java     |     5 +-
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |    14 +
 .../processors/odbc/jdbc/JdbcColumnMetaV4.java  |    90 +
 .../odbc/jdbc/JdbcConnectionContext.java        |    47 +-
 .../odbc/jdbc/JdbcMetaColumnsResultV4.java      |    50 +
 .../odbc/jdbc/JdbcRequestHandler.java           |    53 +-
 .../processors/odbc/jdbc/JdbcResult.java        |     8 +
 .../odbc/odbc/OdbcConnectionContext.java        |    44 +-
 .../odbc/odbc/OdbcRequestHandler.java           |    15 +-
 .../platform/cache/PlatformCache.java           |     1 +
 .../client/ClientConnectionContext.java         |    99 +-
 .../platform/client/ClientRequest.java          |    29 +
 .../platform/client/ClientRequestHandler.java   |    23 +-
 .../platform/client/ClientResourceRegistry.java |    10 +-
 .../platform/client/ClientStatus.java           |     9 +
 .../cache/ClientCacheClearKeyRequest.java       |     3 +
 .../cache/ClientCacheClearKeysRequest.java      |     3 +
 .../client/cache/ClientCacheClearRequest.java   |     3 +
 .../cache/ClientCacheContainsKeyRequest.java    |     3 +
 .../cache/ClientCacheContainsKeysRequest.java   |     3 +
 ...ientCacheCreateWithConfigurationRequest.java |     6 +-
 .../cache/ClientCacheCreateWithNameRequest.java |     3 +
 .../client/cache/ClientCacheDestroyRequest.java |     3 +
 .../client/cache/ClientCacheGetAllRequest.java  |     3 +
 .../ClientCacheGetAndPutIfAbsentRequest.java    |     3 +
 .../cache/ClientCacheGetAndPutRequest.java      |     3 +
 .../cache/ClientCacheGetAndRemoveRequest.java   |     3 +
 .../cache/ClientCacheGetAndReplaceRequest.java  |     3 +
 ...acheGetOrCreateWithConfigurationRequest.java |     6 +-
 .../ClientCacheGetOrCreateWithNameRequest.java  |     3 +
 .../client/cache/ClientCacheGetRequest.java     |     3 +
 .../client/cache/ClientCacheGetSizeRequest.java |     3 +
 .../client/cache/ClientCachePutAllRequest.java  |     3 +
 .../cache/ClientCachePutIfAbsentRequest.java    |     3 +
 .../client/cache/ClientCachePutRequest.java     |     3 +
 .../cache/ClientCacheRemoveAllRequest.java      |     3 +
 .../cache/ClientCacheRemoveIfEqualsRequest.java |     3 +
 .../cache/ClientCacheRemoveKeyRequest.java      |     3 +
 .../cache/ClientCacheRemoveKeysRequest.java     |     3 +
 .../ClientCacheReplaceIfEqualsRequest.java      |     3 +
 .../client/cache/ClientCacheReplaceRequest.java |     3 +
 .../client/cache/ClientCacheRequest.java        |    32 +
 .../cache/ClientCacheScanQueryRequest.java      |     3 +
 .../cache/ClientCacheSqlFieldsQueryRequest.java |    18 +-
 .../cache/ClientCacheSqlQueryRequest.java       |     1 +
 .../dotnet/PlatformDotNetCacheStore.java        |     9 +-
 .../platform/services/PlatformServices.java     |     8 +-
 .../utils/PlatformConfigurationUtils.java       |    24 +
 .../processors/query/GridQueryProcessor.java    |    44 +-
 .../processors/query/GridQueryProperty.java     |    14 +
 .../internal/processors/query/QueryField.java   |    26 +-
 .../internal/processors/query/QueryUtils.java   |    50 +-
 .../processors/query/SqlClientContext.java      |   108 +-
 .../query/property/QueryBinaryProperty.java     |    23 +-
 .../query/property/QueryClassProperty.java      |    10 +
 .../message/SchemaFinishDiscoveryMessage.java   |     5 +
 .../message/SchemaProposeDiscoveryMessage.java  |     5 +
 .../processors/rest/GridRestCommand.java        |    15 +-
 .../processors/rest/GridRestProcessor.java      |   132 +-
 .../handlers/cache/GridCacheCommandHandler.java |   102 +-
 .../cluster/GridChangeStateCommandHandler.java  |     3 +-
 .../top/GridTopologyCommandHandler.java         |     2 +-
 .../handlers/user/UserActionCommandHandler.java |    97 +
 .../rest/handlers/user/package-info.java        |    22 +
 .../protocols/tcp/GridTcpRestNioListener.java   |     7 +-
 .../rest/request/GridRestRequest.java           |    21 +-
 .../rest/request/RestUserActionRequest.java     |    64 +
 .../security/SecurityContextHolder.java         |    53 +
 .../service/GridServiceProcessor.java           |    15 +-
 .../processors/service/GridServiceProxy.java    |    27 +-
 .../processors/task/GridTaskProcessor.java      |     2 +-
 .../processors/task/GridTaskWorker.java         |     4 +-
 .../timeout/GridTimeoutProcessor.java           |   105 +-
 .../apache/ignite/internal/sql/SqlKeyword.java  |    38 +-
 .../apache/ignite/internal/sql/SqlLexer.java    |    26 +
 .../ignite/internal/sql/SqlLexerTokenType.java  |     3 +
 .../apache/ignite/internal/sql/SqlParser.java   |    46 +-
 .../ignite/internal/sql/SqlParserUtils.java     |    50 +
 .../sql/command/SqlAlterUserCommand.java        |    79 +
 .../sql/command/SqlBulkLoadCommand.java         |     1 +
 .../sql/command/SqlCreateUserCommand.java       |    79 +
 .../sql/command/SqlDropUserCommand.java         |    59 +
 .../sql/command/SqlSetStreamingCommand.java     |   191 +
 .../suggestions/OsConfigurationSuggestions.java |    14 +-
 .../ignite/internal/util/HostAndPortRange.java  |    15 +-
 .../ignite/internal/util/IgniteUtils.java       |    77 +-
 .../apache/ignite/internal/util/MpscQueue.java  |   240 -
 .../ignite/internal/util/StripedExecutor.java   |   337 +-
 .../internal/util/future/GridFutureAdapter.java |    12 +-
 .../ignite/internal/util/lang/GridFunc.java     |    18 -
 .../ConcurrentDequeFactoryCallable.java         |    40 -
 .../ignite/internal/util/nio/GridNioServer.java |   109 +-
 .../util/nio/GridNioServerListener.java         |    14 +
 .../util/nio/GridNioServerListenerAdapter.java  |    11 +
 .../util/nio/GridSelectorNioSessionImpl.java    |     5 +-
 .../util/nio/ssl/BlockingSslHandler.java        |    10 +-
 .../apache/ignite/internal/util/typedef/X.java  |    37 +-
 .../visor/baseline/VisorBaselineTask.java       |    24 +-
 .../cache/VisorCacheLostPartitionsTask.java     |    85 +
 .../cache/VisorCacheLostPartitionsTaskArg.java  |    73 +
 .../VisorCacheLostPartitionsTaskResult.java     |    74 +
 .../VisorCacheResetLostPartitionsTask.java      |    65 +
 .../VisorCacheResetLostPartitionsTaskArg.java   |    73 +
 .../internal/visor/query/VisorQueryUtils.java   |     9 +
 .../visor/util/VisorExceptionWrapper.java       |    11 +-
 .../internal/visor/util/VisorTaskUtils.java     |     8 +-
 .../worker/WorkersControlMXBeanImpl.java        |    62 +
 .../ignite/internal/worker/WorkersRegistry.java |    80 +
 .../ignite/internal/worker/package-info.java    |    22 +
 .../ignite/mxbean/ClusterMetricsMXBean.java     |     6 +-
 .../ignite/mxbean/WorkersControlMXBean.java     |    49 +
 .../plugin/security/AuthenticationContext.java  |    40 +
 .../plugin/security/SecurityPermission.java     |    11 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    10 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |    11 +
 .../jobstealing/JobStealingCollisionSpi.java    |     4 +-
 .../tcp/TcpCommunicationMetricsListener.java    |   306 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   277 +-
 .../tcp/internal/ConnectionKey.java             |   117 +
 .../TcpCommunicationConnectionCheckFuture.java  |   519 +
 ...pCommunicationNodeConnectionCheckFuture.java |    30 +
 .../ignite/spi/discovery/DiscoveryDataBag.java  |    18 +-
 .../discovery/DiscoverySpiCustomMessage.java    |    15 +-
 ...DiscoverySpiMutableCustomMessageSupport.java |    40 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |    21 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |    82 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |    55 +-
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |     3 +-
 .../tcp/internal/DiscoveryDataPacket.java       |    16 +-
 .../tcp/internal/TcpDiscoveryNode.java          |    32 +-
 ...cpDiscoveryServerOnlyCustomEventMessage.java |    43 +
 .../startup/cmdline/CommandLineStartup.java     |     5 +-
 .../thread/IgniteStripedThreadPoolExecutor.java |     8 +-
 .../ignite/thread/IgniteThreadFactory.java      |    30 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |    12 +-
 .../ignite/thread/OomExceptionHandler.java      |    44 +
 .../apache/ignite/util/deque/FastSizeDeque.java |   372 +
 .../java/org/jsr166/ConcurrentHashMap8.java     |    63 +
 .../core/src/main/java/org/mindrot/BCrypt.java  |   783 +
 .../src/main/java/org/mindrot/package-info.java |    16 +
 .../resources/META-INF/classnames.properties    |    42 +-
 .../ignite/GridSuppressedExceptionSelfTest.java |    23 +-
 .../AffinityDistributionLoggingTest.java        |   268 +
 ...unctionExcludeNeighborsAbstractSelfTest.java |     8 +-
 .../client/ClientCacheConfigurationTest.java    |   100 +
 .../ignite/client/ClientConfigurationTest.java  |    69 +
 .../org/apache/ignite/client/Comparers.java     |   112 +
 .../java/org/apache/ignite/client/Config.java   |    61 +
 .../apache/ignite/client/FunctionalTest.java    |   413 +
 .../apache/ignite/client/IgniteBinaryTest.java  |   221 +
 .../java/org/apache/ignite/client/LoadTest.java |   133 +
 .../ignite/client/LocalIgniteCluster.java       |   193 +
 .../java/org/apache/ignite/client/Person.java   |    65 +
 .../apache/ignite/client/ReliabilityTest.java   |   186 +
 .../failure/AbstractFailureHandlerTest.java     |    74 +
 .../failure/FailureHandlerTriggeredTest.java    |   135 +
 .../ignite/failure/IoomFailureHandlerTest.java  |   144 +
 .../ignite/failure/OomFailureHandlerTest.java   |   255 +
 .../failure/StopNodeFailureHandlerTest.java     |    74 +
 .../StopNodeOrHaltFailureHandlerTest.java       |   100 +
 .../internal/ClusterGroupHostsSelfTest.java     |     3 +
 .../ignite/internal/ClusterGroupSelfTest.java   |     2 +
 .../internal/ClusterNodeMetricsUpdateTest.java  |   173 +
 .../internal/DiscoverySpiTestListener.java      |   162 +
 .../ignite/internal/GridDiscoverySelfTest.java  |    14 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |     2 +
 .../internal/GridJobStealingSelfTest.java       |     2 +
 .../internal/GridSameVmStartupSelfTest.java     |    19 +-
 .../apache/ignite/internal/GridSelfTest.java    |     2 +
 .../IgniteClientReconnectAbstractTest.java      |    53 +-
 .../IgniteClientReconnectApiExceptionTest.java  |    47 +-
 .../IgniteClientReconnectAtomicsTest.java       |    30 +-
 .../IgniteClientReconnectCacheTest.java         |    49 +-
 .../IgniteClientReconnectCollectionsTest.java   |    14 +-
 .../IgniteClientReconnectComputeTest.java       |     6 +-
 ...eClientReconnectContinuousProcessorTest.java |    13 +-
 ...IgniteClientReconnectDiscoveryStateTest.java |    22 +-
 ...niteClientReconnectFailoverAbstractTest.java |    12 +-
 .../IgniteClientReconnectServicesTest.java      |     8 +-
 .../internal/IgniteClientReconnectStopTest.java |    12 +-
 .../IgniteClientReconnectStreamerTest.java      |     4 +-
 .../ignite/internal/IgniteClientRejoinTest.java |     3 +
 .../IgniteComputeResultExceptionTest.java       |   186 +
 .../binary/BinaryObjectExceptionSelfTest.java   |    41 +
 .../internal/binary/BinaryTreeSelfTest.java     |   273 +-
 .../binary/TestCachingMetadataHandler.java      |     9 +-
 .../commandline/CommandHandlerParsingTest.java  |    28 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    16 +-
 .../OptimizedObjectStreamSelfTest.java          |    94 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |     4 +-
 .../Authentication1kUsersNodeRestartTest.java   |   107 +
 .../AuthenticationConfigurationClusterTest.java |   204 +
 .../AuthenticationOnNotActiveClusterTest.java   |   122 +
 .../AuthenticationProcessorNodeRestartTest.java |   425 +
 .../AuthenticationProcessorSelfTest.java        |   637 +
 .../cache/CacheDataRegionConfigurationTest.java |   202 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |     2 +-
 .../cache/CacheGroupsMetricsRebalanceTest.java  |    31 +-
 .../CacheMetricsForClusterGroupSelfTest.java    |    12 +-
 .../cache/CacheNearReaderUpdateTest.java        |    29 +-
 .../DataStorageConfigurationValidationTest.java |    75 +
 .../cache/GridCacheAbstractFullApiSelfTest.java |     9 +-
 .../cache/GridCacheAbstractSelfTest.java        |     2 +
 ...ridCacheReplicatedSynchronousCommitTest.java |     4 +-
 ...cheTransactionalAbstractMetricsSelfTest.java |    87 +-
 .../processors/cache/IgniteCacheGroupsTest.java |     1 +
 .../cache/IgniteCacheNearLockValueSelfTest.java |     4 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    11 +
 .../IgniteClusterActivateDeactivateTest.java    |    65 +
 .../IgniteDaemonNodeMarshallerCacheTest.java    |     3 +-
 .../IgniteOnePhaseCommitNearReadersTest.java    |     2 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    12 +-
 ...titionedAtomicCacheGetsDistributionTest.java |    49 +
 ...onalOptimisticCacheGetsDistributionTest.java |    46 +
 ...nalPessimisticCacheGetsDistributionTest.java |    33 +
 ...plicatedAtomicCacheGetsDistributionTest.java |   364 +
 ...onalOptimisticCacheGetsDistributionTest.java |    46 +
 ...nalPessimisticCacheGetsDistributionTest.java |    33 +
 .../WalModeChangeCommonAbstractSelfTest.java    |    55 +-
 .../binary/BinaryMetadataUpdatesFlowTest.java   |    12 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |     2 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |    75 +-
 .../GridCacheQueueClientDisconnectTest.java     |    10 +
 .../IgniteClientDataStructuresAbstractTest.java |    43 +-
 .../IgniteCountDownLatchAbstractSelfTest.java   |    10 +-
 ...ExchangeLatchManagerCoordinatorFailTest.java |   244 +
 .../SemaphoreFailoverNoWaitingAcquirerTest.java |   134 +
 .../CacheLateAffinityAssignmentTest.java        |   127 +-
 .../GridCacheNodeFailureAbstractTest.java       |     5 +-
 .../distributed/IgniteCache150ClientsTest.java  |     2 +
 .../distributed/IgniteCacheManyClientsTest.java |    44 +-
 .../IgniteCachePartitionLossPolicySelfTest.java |    31 +-
 .../IgniteOptimisticTxSuspendResumeTest.java    |     2 +
 .../IgniteTxRemoveTimeoutObjectsTest.java       |    34 +-
 ...achePartitionPartialCountersMapSelfTest.java |    57 +
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |     4 +
 .../dht/GridCacheDhtPreloadSelfTest.java        |     2 +
 .../GridCachePartitionsStateValidationTest.java |   316 +
 ...idCachePartitionsStateValidatorSelfTest.java |   158 +
 .../IgniteCacheTopologySplitAbstractTest.java   |    49 +-
 .../dht/TxRecoveryStoreEnabledTest.java         |    18 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |     3 +-
 .../GridCacheRebalancingSyncSelfTest.java       |     3 +-
 ...idCacheRebalancingWithAsyncClearingTest.java |   240 +
 .../IgnitePdsCacheRebalancingAbstractTest.java  |    78 +-
 .../IgnitePdsCorruptedStoreTest.java            |   347 +
 ...gnitePdsRecoveryAfterFileCorruptionTest.java |     4 +-
 .../IgnitePdsRemoveDuringRebalancingTest.java   |     3 +-
 .../checkpoint/IgniteMassLoadSandboxTest.java   |     2 +-
 ...ckpointSimulationWithRealCpDisabledTest.java |    10 +-
 .../file/IgnitePdsDiskErrorsRecoveringTest.java |   253 +-
 .../db/wal/IgniteWalFlushFailoverTest.java      |     8 +-
 ...lFlushMultiNodeFailoverAbstractSelfTest.java |     8 +-
 .../wal/IgniteWalHistoryReservationsTest.java   |    21 -
 .../db/wal/IgniteWalSerializerVersionTest.java  |     2 +-
 .../persistence/db/wal/WalCompactionTest.java   |    10 +-
 .../cache/persistence/db/wal/WalPathsTest.java  |    86 +
 .../db/wal/WalRecoveryTxLogicalRecordsTest.java |   103 +-
 .../metastorage/IgniteMetaStorageBasicTest.java |   147 +
 .../pagemem/BPlusTreePageMemoryImplTest.java    |     2 +-
 .../BPlusTreeReuseListPageMemoryImplTest.java   |     3 +-
 .../pagemem/FullPageIdTableTest.java            |   195 +-
 ...gnitePageMemReplaceDelayedWriteUnitTest.java |     5 +-
 .../pagemem/IgniteThrottlingUnitTest.java       |     1 -
 .../pagemem/IndexStoragePageMemoryImplTest.java |     3 +-
 .../persistence/pagemem/NoOpWALManager.java     |     7 +-
 .../pagemem/PageMemoryImplNoLoadTest.java       |     3 +-
 .../persistence/pagemem/PageMemoryImplTest.java |   220 +-
 .../pagemem/PagesWriteThrottleSmokeTest.java    |     4 +-
 .../RobinHoodBackwardShiftHashMapTest.java      |   437 +
 .../persistence/tree/io/TrackingPageIOTest.java |   116 +-
 .../IgniteCacheQueryCacheDestroySelfTest.java   |     4 +
 .../ClientReconnectContinuousQueryTest.java     |    19 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java |    23 +-
 ...CacheContinuousQueryClientReconnectTest.java |     3 +
 .../GridCacheWriteBehindStoreSelfTest.java      |    41 +-
 .../TxOptimisticOnPartitionExchangeTest.java    |   322 +
 .../TxPessimisticDeadlockDetectionTest.java     |     2 -
 .../CacheVersionedEntryAbstractTest.java        |    33 +-
 .../closure/GridClosureSerializationTest.java   |     2 +-
 .../continuous/GridEventConsumeSelfTest.java    |    34 +-
 .../processors/database/BPlusTreeSelfTest.java  |    29 +
 .../cache/GridCacheCommandHandlerSelfTest.java  |     9 +-
 .../service/ClosureServiceClientsNodesTest.java |    19 +-
 .../GridServiceProcessorProxySelfTest.java      |    12 +-
 .../internal/sql/SqlParserBulkLoadSelfTest.java |     6 +-
 .../sql/SqlParserSetStreamingSelfTest.java      |   134 +
 .../internal/sql/SqlParserUserSelfTest.java     |   134 +
 .../internal/util/GridTestClockTimer.java       |     9 +
 .../internal/util/StripedExecutorTest.java      |     2 +-
 .../util/future/GridCompoundFutureSelfTest.java |     8 +-
 .../future/GridFutureListenPerformanceTest.java |     4 +-
 .../util/future/GridFutureQueueTest.java        |     5 +-
 .../lang/GridFutureListenPerformanceTest.java   |     4 +-
 .../GridCircularBufferPerformanceTest.java      |     5 +-
 .../lang/utils/GridCircularBufferSelfTest.java  |     9 +-
 .../GridMarshallerMappingConsistencyTest.java   |   183 +
 .../ignite/messaging/GridMessagingSelfTest.java |   126 +-
 .../platform/PlatformCacheWriteMetricsTask.java |     5 +
 .../GridTcpCommunicationSpiAbstractTest.java    |    71 +
 .../tcp/GridTcpCommunicationSpiLanTest.java     |    13 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    30 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    24 +
 .../TcpCommunicationSpiSkipMessageSendTest.java |   414 +
 .../tcp/TcpCommunicationStatisticsTest.java     |    14 +-
 .../FilterDataForClientNodeDiscoveryTest.java   |   219 +
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |     2 -
 .../testframework/GridSpiTestContext.java       |    10 +
 .../ignite/testframework/GridTestUtils.java     |   133 +-
 .../config/GridTestProperties.java              |     9 +
 .../testframework/junits/GridAbstractTest.java  |   196 +-
 .../IgniteConfigVariationsAbstractTest.java     |     6 +
 .../junits/common/GridCommonAbstractTest.java   |    38 +
 .../junits/multijvm/IgniteNodeRunner.java       |     2 +
 .../junits/multijvm/IgniteProcessProxy.java     |    43 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    15 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |     2 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |     6 +
 .../testsuites/IgniteCacheTestSuite2.java       |     3 +
 .../testsuites/IgniteCacheTestSuite3.java       |     2 +
 .../testsuites/IgniteCacheTestSuite5.java       |     3 +
 .../testsuites/IgniteCacheTestSuite6.java       |    32 +
 .../testsuites/IgniteClientTestSuite.java       |    38 -
 .../testsuites/IgniteComputeGridTestSuite.java  |     4 +
 .../IgniteMarshallerSelfTestSuite.java          |     2 +
 .../ignite/testsuites/IgnitePdsTestSuite.java   |     4 +
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |     3 +
 .../testsuites/IgnitePdsUnitTestSuite.java      |     7 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |     3 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |     3 +
 .../ignite/util/GridCommandHandlerTest.java     |   141 +-
 modules/core/src/test/resources/client.jks      |   Bin 0 -> 3232 bytes
 modules/core/src/test/resources/server.jks      |   Bin 0 -> 3230 bytes
 modules/core/src/test/resources/trust.jks       |   Bin 0 -> 2432 bytes
 modules/direct-io/pom.xml                       |    14 -
 .../file/AlignedBuffersDirectFileIO.java        |     7 +-
 modules/gce/pom.xml                             |    29 +
 .../external/HadoopExternalTaskExecutor.java    |     4 +-
 .../HadoopExternalCommunication.java            |     4 +-
 .../query/h2/DmlStatementsProcessor.java        |     4 +-
 .../processors/query/h2/H2RowCache.java         |    21 +-
 .../processors/query/h2/H2RowCacheRegistry.java |     2 +-
 .../internal/processors/query/h2/H2Utils.java   |    28 +-
 .../processors/query/h2/IgniteH2Indexing.java   |    67 +-
 .../query/h2/database/H2TreeIndex.java          |    15 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |    58 +-
 .../processors/query/h2/dml/DmlBatchSender.java |    20 +-
 .../query/h2/opt/GridH2IndexBase.java           |    10 +
 .../query/h2/opt/GridH2QueryContext.java        |    30 +-
 .../processors/query/h2/opt/GridH2Table.java    |    19 +-
 .../processors/query/h2/sql/GridSqlColumn.java  |    17 +-
 .../query/h2/sql/GridSqlCreateTable.java        |     7 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |     3 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    29 +-
 .../query/h2/twostep/MapQueryLazyWorker.java    |    20 +-
 .../query/h2/twostep/MapQueryResult.java        |     2 +-
 .../apache/ignite/client/ClientTestSuite.java   |    40 +
 .../ignite/client/FunctionalQueryTest.java      |   169 +
 .../ignite/client/IgniteBinaryQueryTest.java    |   130 +
 .../org/apache/ignite/client/SecurityTest.java  |   226 +
 .../ClientReconnectAfterClusterRestartTest.java |    33 +-
 ...GridCacheLazyQueryPartitionsReleaseTest.java |     9 +-
 ...niteCacheLockPartitionOnAffinityRunTest.java |    46 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |     6 +-
 .../authentication/SqlUserCommandSelfTest.java  |   294 +
 ...lientQueryReplicatedNodeRestartSelfTest.java |     1 +
 ...niteCacheDistributedQueryCancelSelfTest.java |     2 +-
 ...ynamicColumnsAbstractConcurrentSelfTest.java |     6 +-
 .../DynamicIndexAbstractBasicSelfTest.java      |     5 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  |    37 +
 .../cache/index/H2DynamicTableSelfTest.java     |    37 +
 .../cache/index/H2RowCacheSelfTest.java         |    69 +-
 .../cache/index/IgniteDecimalSelfTest.java      |   265 +
 .../client/IgniteDataStreamerTest.java          |   118 +
 .../query/SqlPushDownFunctionTest.java          |    78 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |    12 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    12 +
 .../IgniteCacheWithIndexingTestSuite.java       |     3 +
 .../IgnitePdsWithIndexingCoreTestSuite.java     |     2 +
 .../GridJtaTransactionManagerSelfTest.java      |    21 +-
 .../GridPartitionedCacheJtaFactorySelfTest.java |    19 +-
 .../kubernetes/config/ignite-deployment.yaml    |     3 +-
 modules/ml/pom.xml                              |    13 +
 .../org/apache/ignite/ml/DatasetTrainer.java    |    42 -
 .../main/java/org/apache/ignite/ml/Trainer.java |    39 -
 .../ignite/ml/dataset/DatasetFactory.java       |   126 +-
 .../dataset/impl/cache/CacheBasedDataset.java   |    14 +-
 .../dataset/impl/cache/util/ComputeUtils.java   |     9 +-
 .../ml/dataset/impl/local/LocalDataset.java     |    16 +-
 .../dataset/impl/local/LocalDatasetBuilder.java |     8 +-
 .../ml/dataset/primitive/SimpleDataset.java     |     6 +-
 .../builder/data/SimpleDatasetDataBuilder.java  |    19 +-
 .../data/SimpleLabeledDatasetDataBuilder.java   |    44 +-
 .../primitive/data/SimpleDatasetData.java       |    12 +-
 .../data/SimpleLabeledDatasetData.java          |    22 +-
 .../apache/ignite/ml/estimators/Estimators.java |    50 -
 .../ignite/ml/estimators/package-info.java      |    22 -
 .../apache/ignite/ml/genetic/Chromosome.java    |   107 +
 .../apache/ignite/ml/genetic/CrossOverJob.java  |   141 +
 .../apache/ignite/ml/genetic/CrossOverTask.java |   136 +
 .../apache/ignite/ml/genetic/FitnessJob.java    |   101 +
 .../apache/ignite/ml/genetic/FitnessTask.java   |   101 +
 .../org/apache/ignite/ml/genetic/GAGrid.java    |   446 +
 .../java/org/apache/ignite/ml/genetic/Gene.java |    86 +
 .../ignite/ml/genetic/IFitnessFunction.java     |    37 +
 .../org/apache/ignite/ml/genetic/MutateJob.java |    88 +
 .../apache/ignite/ml/genetic/MutateTask.java    |   188 +
 .../ignite/ml/genetic/TruncateSelectionJob.java |    81 +
 .../ml/genetic/TruncateSelectionTask.java       |   159 +
 .../ml/genetic/cache/GeneCacheConfig.java       |    50 +
 .../ml/genetic/cache/PopulationCacheConfig.java |    51 +
 .../ignite/ml/genetic/cache/package-info.java   |    21 +
 .../ml/genetic/functions/GAGridFunction.java    |   125 +
 .../ml/genetic/functions/package-info.java      |    21 +
 .../apache/ignite/ml/genetic/package-info.java  |    21 +
 .../genetic/parameter/ChromosomeCriteria.java   |    49 +
 .../ml/genetic/parameter/GAConfiguration.java   |   333 +
 .../ml/genetic/parameter/GAGridConstants.java   |    35 +
 .../genetic/parameter/ITerminateCriteria.java   |    39 +
 .../ml/genetic/parameter/package-info.java      |    21 +
 .../ignite/ml/genetic/utils/GAGridUtils.java    |   120 +
 .../ignite/ml/genetic/utils/package-info.java   |    21 +
 .../java/org/apache/ignite/ml/knn/KNNUtils.java |    59 +
 .../classification/KNNClassificationModel.java  |   273 +
 .../KNNClassificationTrainer.java               |    41 +
 .../ml/knn/classification/KNNModelFormat.java   |    88 +
 .../ml/knn/classification/KNNStrategy.java      |    29 +
 .../ml/knn/classification/package-info.java     |    22 +
 .../apache/ignite/ml/knn/models/KNNModel.java   |   233 -
 .../ignite/ml/knn/models/KNNModelFormat.java    |    96 -
 .../ignite/ml/knn/models/KNNStrategy.java       |    27 -
 .../ignite/ml/knn/models/package-info.java      |    22 -
 .../regression/KNNMultipleLinearRegression.java |    83 -
 .../ml/knn/regression/KNNRegressionModel.java   |    90 +
 .../ml/knn/regression/KNNRegressionTrainer.java |    40 +
 .../ignite/ml/knn/regression/package-info.java  |     4 +-
 .../ml/math/distances/DistanceMeasure.java      |    12 +
 .../ml/math/distances/EuclideanDistance.java    |    10 +
 .../ml/math/distances/HammingDistance.java      |     5 +
 .../ml/math/distances/ManhattanDistance.java    |     5 +
 .../ml/math/functions/IgniteBiFunction.java     |     8 +-
 .../LinSysPartitionDataBuilderOnHeap.java       |    85 -
 .../math/isolve/LinSysPartitionDataOnHeap.java  |    75 -
 .../ml/math/isolve/lsqr/AbstractLSQR.java       |     3 +-
 .../ignite/ml/math/isolve/lsqr/LSQROnHeap.java  |    38 +-
 .../org/apache/ignite/ml/nn/Activators.java     |    20 +
 .../ignite/ml/nn/LabeledVectorsCache.java       |    63 -
 .../ml/nn/MLPGroupUpdateTrainerCacheInput.java  |   157 -
 .../org/apache/ignite/ml/nn/MLPTrainer.java     |   212 +
 .../ignite/ml/nn/MultilayerPerceptron.java      |    14 +-
 .../apache/ignite/ml/nn/UpdatesStrategy.java    |    95 +
 .../ml/nn/initializers/RandomInitializer.java   |    16 +
 .../AbstractMLPGroupUpdateTrainerInput.java     |    60 -
 .../ml/nn/trainers/distributed/MLPCache.java    |    91 -
 .../distributed/MLPGroupTrainingCacheValue.java |    48 -
 .../distributed/MLPGroupUpdateTrainer.java      |   377 -
 .../MLPGroupUpdateTrainerDataCache.java         |    77 -
 .../MLPGroupUpdateTrainerLocalContext.java      |   117 -
 .../MLPGroupUpdateTrainingContext.java          |    64 -
 .../distributed/MLPGroupUpdateTrainingData.java |   103 -
 .../MLPGroupUpdateTrainingLoopData.java         |   126 -
 .../trainers/distributed/MLPMetaoptimizer.java  |    76 -
 .../nn/trainers/distributed/package-info.java   |    22 -
 .../nn/trainers/local/MLPLocalBatchTrainer.java |    78 -
 .../ml/nn/trainers/local/package-info.java      |    22 -
 .../ignite/ml/nn/trainers/package-info.java     |    22 -
 .../ignite/ml/optimization/GradientDescent.java |   202 -
 .../ml/optimization/GradientFunction.java       |    31 -
 .../LeastSquaresGradientFunction.java           |    33 -
 .../NesterovParameterUpdate.java                |    16 +-
 .../NesterovUpdateCalculator.java               |    11 +-
 .../ParameterUpdateCalculator.java              |     3 +-
 .../updatecalculators/RPropParameterUpdate.java |     3 +
 .../RPropUpdateCalculator.java                  |     3 +
 .../SimpleGDParameterUpdate.java                |     3 +
 .../SimpleGDUpdateCalculator.java               |     3 +
 .../util/SparseDistributedMatrixMapReducer.java |    84 -
 .../ml/optimization/util/package-info.java      |    22 -
 .../ml/preprocessing/PreprocessingTrainer.java  |    41 +-
 .../normalization/NormalizationTrainer.java     |    35 +-
 .../linear/FeatureExtractorWrapper.java         |    55 +
 .../linear/LinearRegressionLSQRTrainer.java     |    31 +-
 .../linear/LinearRegressionQRTrainer.java       |    72 -
 .../linear/LinearRegressionSGDTrainer.java      |   121 +-
 .../apache/ignite/ml/structures/Dataset.java    |    10 +
 .../ignite/ml/structures/LabeledDataset.java    |    16 +-
 .../LabelPartitionDataBuilderOnHeap.java        |    65 +
 .../partition/LabelPartitionDataOnHeap.java     |    45 +
 ...abeledDatasetPartitionDataBuilderOnHeap.java |    86 +
 .../svm/SVMLinearBinaryClassificationModel.java |     9 +-
 .../SVMLinearBinaryClassificationTrainer.java   |   101 +-
 .../SVMLinearMultiClassClassificationModel.java |     8 +-
 ...VMLinearMultiClassClassificationTrainer.java |    92 +-
 .../ignite/ml/trainers/DatasetTrainer.java      |    89 +
 .../ml/trainers/MultiLabelDatasetTrainer.java   |    28 +
 .../ml/trainers/SingleLabelDatasetTrainer.java  |    28 +
 .../org/apache/ignite/ml/trainers/Trainer.java  |    33 -
 .../trainers/group/BaseLocalProcessorJob.java   |   146 -
 .../ignite/ml/trainers/group/ConstModel.java    |    46 -
 .../ignite/ml/trainers/group/GroupTrainer.java  |   208 -
 .../group/GroupTrainerBaseProcessorTask.java    |   144 -
 .../ml/trainers/group/GroupTrainerCacheKey.java |   125 -
 .../group/GroupTrainerEntriesProcessorTask.java |    64 -
 .../ml/trainers/group/GroupTrainerInput.java    |    37 -
 .../group/GroupTrainerKeysProcessorTask.java    |    62 -
 .../ml/trainers/group/GroupTrainingContext.java |    98 -
 .../group/LocalEntriesProcessorJob.java         |    85 -
 .../trainers/group/LocalKeysProcessorJob.java   |    78 -
 .../ignite/ml/trainers/group/Metaoptimizer.java |    93 -
 .../group/MetaoptimizerDistributedStep.java     |    97 -
 .../group/MetaoptimizerGroupTrainer.java        |   132 -
 .../ml/trainers/group/ResultAndUpdates.java     |   178 -
 .../ml/trainers/group/UpdateStrategies.java     |    47 -
 .../ml/trainers/group/UpdatesStrategy.java      |    94 -
 .../ignite/ml/trainers/group/chain/Chains.java  |    56 -
 .../trainers/group/chain/ComputationsChain.java |   246 -
 .../chain/DistributedEntryProcessingStep.java   |    34 -
 .../chain/DistributedKeyProcessingStep.java     |    33 -
 .../trainers/group/chain/DistributedStep.java   |    70 -
 .../trainers/group/chain/EntryAndContext.java   |    70 -
 .../trainers/group/chain/HasTrainingUUID.java   |    32 -
 .../ml/trainers/group/chain/KeyAndContext.java  |    67 -
 .../ml/trainers/group/chain/package-info.java   |    22 -
 .../ignite/ml/trainers/group/package-info.java  |    22 -
 .../ml/trainers/local/LocalBatchTrainer.java    |   178 -
 .../trainers/local/LocalBatchTrainerInput.java  |    42 -
 .../ignite/ml/trainers/local/package-info.java  |    22 -
 .../org/apache/ignite/ml/tree/DecisionTree.java |   252 +
 .../tree/DecisionTreeClassificationTrainer.java |    93 +
 .../ml/tree/DecisionTreeConditionalNode.java    |    78 +
 .../ignite/ml/tree/DecisionTreeLeafNode.java    |    48 +
 .../apache/ignite/ml/tree/DecisionTreeNode.java |    26 +
 .../ml/tree/DecisionTreeRegressionTrainer.java  |    60 +
 .../org/apache/ignite/ml/tree/TreeFilter.java   |    38 +
 .../ignite/ml/tree/data/DecisionTreeData.java   |   128 +
 .../ml/tree/data/DecisionTreeDataBuilder.java   |    73 +
 .../ignite/ml/tree/data/package-info.java       |    22 +
 .../ml/tree/impurity/ImpurityMeasure.java       |    55 +
 .../impurity/ImpurityMeasureCalculator.java     |    38 +
 .../tree/impurity/gini/GiniImpurityMeasure.java |   115 +
 .../gini/GiniImpurityMeasureCalculator.java     |   110 +
 .../ml/tree/impurity/gini/package-info.java     |    22 +
 .../tree/impurity/mse/MSEImpurityMeasure.java   |   133 +
 .../mse/MSEImpurityMeasureCalculator.java       |    80 +
 .../ml/tree/impurity/mse/package-info.java      |    22 +
 .../ignite/ml/tree/impurity/package-info.java   |    22 +
 .../util/SimpleStepFunctionCompressor.java      |   149 +
 .../ml/tree/impurity/util/StepFunction.java     |   162 +
 .../impurity/util/StepFunctionCompressor.java   |    55 +
 .../ml/tree/impurity/util/package-info.java     |    22 +
 .../ml/tree/leaf/DecisionTreeLeafBuilder.java   |    38 +
 .../tree/leaf/MeanDecisionTreeLeafBuilder.java  |    73 +
 .../leaf/MostCommonDecisionTreeLeafBuilder.java |    86 +
 .../ignite/ml/tree/leaf/package-info.java       |    22 +
 .../org/apache/ignite/ml/tree/package-info.java |    22 +
 .../ignite/ml/trees/CategoricalRegionInfo.java  |    72 -
 .../ignite/ml/trees/CategoricalSplitInfo.java   |    68 -
 .../ignite/ml/trees/ContinuousRegionInfo.java   |    74 -
 .../ml/trees/ContinuousSplitCalculator.java     |    51 -
 .../org/apache/ignite/ml/trees/RegionInfo.java  |    62 -
 .../ml/trees/models/DecisionTreeModel.java      |    44 -
 .../ignite/ml/trees/models/package-info.java    |    22 -
 .../ml/trees/nodes/CategoricalSplitNode.java    |    50 -
 .../ml/trees/nodes/ContinuousSplitNode.java     |    56 -
 .../ignite/ml/trees/nodes/DecisionTreeNode.java |    33 -
 .../org/apache/ignite/ml/trees/nodes/Leaf.java  |    49 -
 .../apache/ignite/ml/trees/nodes/SplitNode.java |   100 -
 .../ignite/ml/trees/nodes/package-info.java     |    22 -
 .../apache/ignite/ml/trees/package-info.java    |    22 -
 .../ml/trees/trainers/columnbased/BiIndex.java  |   113 -
 ...exedCacheColumnDecisionTreeTrainerInput.java |    57 -
 .../CacheColumnDecisionTreeTrainerInput.java    |   141 -
 .../columnbased/ColumnDecisionTreeTrainer.java  |   568 -
 .../ColumnDecisionTreeTrainerInput.java         |    55 -
 .../MatrixColumnDecisionTreeTrainerInput.java   |    83 -
 .../trainers/columnbased/RegionProjection.java  |   109 -
 .../trainers/columnbased/TrainingContext.java   |   166 -
 .../columnbased/caches/ContextCache.java        |    68 -
 .../columnbased/caches/FeaturesCache.java       |   151 -
 .../columnbased/caches/ProjectionsCache.java    |   286 -
 .../trainers/columnbased/caches/SplitCache.java |   206 -
 .../columnbased/caches/package-info.java        |    22 -
 .../ContinuousSplitCalculators.java             |    34 -
 .../contsplitcalcs/GiniSplitCalculator.java     |   234 -
 .../contsplitcalcs/VarianceSplitCalculator.java |   179 -
 .../contsplitcalcs/package-info.java            |    22 -
 .../trainers/columnbased/package-info.java      |    22 -
 .../columnbased/regcalcs/RegionCalculators.java |    85 -
 .../columnbased/regcalcs/package-info.java      |    22 -
 .../vectors/CategoricalFeatureProcessor.java    |   212 -
 .../vectors/ContinuousFeatureProcessor.java     |   111 -
 .../vectors/ContinuousSplitInfo.java            |    71 -
 .../columnbased/vectors/FeatureProcessor.java   |    82 -
 .../vectors/FeatureVectorProcessorUtils.java    |    57 -
 .../columnbased/vectors/SampleInfo.java         |    80 -
 .../trainers/columnbased/vectors/SplitInfo.java |   106 -
 .../columnbased/vectors/package-info.java       |    22 -
 .../org/apache/ignite/ml/util/MnistUtils.java   |    95 +-
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |    14 +-
 .../org/apache/ignite/ml/LocalModelsTest.java   |    32 +-
 .../impl/cache/CacheBasedDatasetTest.java       |    11 +-
 .../ml/genetic/GAGridCalculateFitnessTest.java  |   145 +
 .../genetic/GAGridInitializePopulationTest.java |   156 +
 .../ignite/ml/genetic/GAGridTestSuite.java      |    33 +
 .../ml/genetic/PasswordFitnessFunction.java     |    61 +
 .../org/apache/ignite/ml/knn/BaseKNNTest.java   |    92 -
 .../ignite/ml/knn/KNNClassificationTest.java    |   210 +-
 .../ml/knn/KNNMultipleLinearRegressionTest.java |   157 -
 .../apache/ignite/ml/knn/KNNRegressionTest.java |   155 +
 .../org/apache/ignite/ml/knn/KNNTestSuite.java  |     2 +-
 .../ignite/ml/knn/LabeledDatasetHelper.java     |    56 +
 .../ignite/ml/knn/LabeledDatasetTest.java       |    55 +-
 .../ml/math/isolve/lsqr/LSQROnHeapTest.java     |    17 +-
 .../ignite/ml/nn/MLPGroupTrainerTest.java       |   148 -
 .../ignite/ml/nn/MLPLocalTrainerTest.java       |    97 -
 .../java/org/apache/ignite/ml/nn/MLPTest.java   |    12 +-
 .../org/apache/ignite/ml/nn/MLPTestSuite.java   |     4 +-
 .../ignite/ml/nn/MLPTrainerIntegrationTest.java |   182 +
 .../org/apache/ignite/ml/nn/MLPTrainerTest.java |   193 +
 .../ml/nn/SimpleMLPLocalBatchTrainerInput.java  |    99 -
 .../MLPTrainerMnistIntegrationTest.java         |   132 +
 .../ml/nn/performance/MLPTrainerMnistTest.java  |   102 +
 .../ml/nn/performance/MnistDistributed.java     |   154 -
 .../ignite/ml/nn/performance/MnistLocal.java    |    95 -
 .../ml/nn/performance/MnistMLPTestUtil.java     |    53 +-
 .../ml/optimization/GradientDescentTest.java    |    64 -
 .../ml/optimization/OptimizationTestSuite.java  |    33 -
 .../SparseDistributedMatrixMapReducerTest.java  |   135 -
 .../normalization/NormalizationTrainerTest.java |    10 +-
 .../ml/regressions/RegressionsTestSuite.java    |    18 +-
 .../linear/ArtificialRegressionDatasets.java    |   404 -
 ...istributedLinearRegressionQRTrainerTest.java |    36 -
 ...stributedLinearRegressionSGDTrainerTest.java |    35 -
 ...istributedLinearRegressionQRTrainerTest.java |    36 -
 ...stributedLinearRegressionSGDTrainerTest.java |    35 -
 .../GenericLinearRegressionTrainerTest.java     |   206 -
 ...wareAbstractLinearRegressionTrainerTest.java |   124 -
 .../linear/LinearRegressionLSQRTrainerTest.java |    24 +-
 .../linear/LinearRegressionSGDTrainerTest.java  |    94 +
 .../LocalLinearRegressionQRTrainerTest.java     |    36 -
 .../LocalLinearRegressionSGDTrainerTest.java    |    35 -
 .../org/apache/ignite/ml/svm/BaseSVMTest.java   |    58 -
 .../ignite/ml/svm/SVMBinaryTrainerTest.java     |    74 +
 .../org/apache/ignite/ml/svm/SVMModelTest.java  |     6 +-
 .../ignite/ml/svm/SVMMultiClassTrainerTest.java |    77 +
 .../org/apache/ignite/ml/svm/SVMTestSuite.java  |    14 +-
 ...inearSVMBinaryClassificationTrainerTest.java |    35 -
 ...inearSVMBinaryClassificationTrainerTest.java |   141 -
 ...inearSVMBinaryClassificationTrainerTest.java |    38 -
 ...rSVMMultiClassClassificationTrainerTest.java |    35 -
 ...rSVMMultiClassClassificationTrainerTest.java |    76 -
 ...rSVMMultiClassClassificationTrainerTest.java |    38 -
 .../group/DistributedWorkersChainTest.java      |   189 -
 .../ml/trainers/group/GroupTrainerTest.java     |    90 -
 .../trainers/group/SimpleGroupTrainerInput.java |    63 -
 .../ml/trainers/group/TestGroupTrainer.java     |   144 -
 .../group/TestGroupTrainerLocalContext.java     |    85 -
 .../trainers/group/TestGroupTrainingCache.java  |    70 -
 .../group/TestGroupTrainingSecondCache.java     |    56 -
 .../ml/trainers/group/TestLocalContext.java     |    51 -
 .../ml/trainers/group/TestTrainingLoopStep.java |    65 -
 .../trainers/group/TrainersGroupTestSuite.java  |    32 -
 ...reeClassificationTrainerIntegrationTest.java |   101 +
 .../DecisionTreeClassificationTrainerTest.java  |    87 +
 ...ionTreeRegressionTrainerIntegrationTest.java |   101 +
 .../tree/DecisionTreeRegressionTrainerTest.java |    87 +
 .../ignite/ml/tree/DecisionTreeTestSuite.java   |    48 +
 .../ml/tree/data/DecisionTreeDataTest.java      |    59 +
 .../gini/GiniImpurityMeasureCalculatorTest.java |   103 +
 .../impurity/gini/GiniImpurityMeasureTest.java  |   131 +
 .../mse/MSEImpurityMeasureCalculatorTest.java   |    59 +
 .../impurity/mse/MSEImpurityMeasureTest.java    |   109 +
 .../util/SimpleStepFunctionCompressorTest.java  |    75 +
 .../ml/tree/impurity/util/StepFunctionTest.java |    71 +
 .../tree/impurity/util/TestImpurityMeasure.java |    88 +
 .../DecisionTreeMNISTIntegrationTest.java       |   106 +
 .../tree/performance/DecisionTreeMNISTTest.java |    75 +
 .../ignite/ml/trees/BaseDecisionTreeTest.java   |    70 -
 .../ml/trees/ColumnDecisionTreeTrainerTest.java |   191 -
 .../ignite/ml/trees/DecisionTreesTestSuite.java |    33 -
 .../ml/trees/GiniSplitCalculatorTest.java       |   141 -
 .../ignite/ml/trees/SplitDataGenerator.java     |   390 -
 .../ml/trees/VarianceSplitCalculatorTest.java   |    84 -
 .../ColumnDecisionTreeTrainerBenchmark.java     |   456 -
 .../trees/columntrees.manualrun.properties      |     2 +-
 .../osgi-karaf/src/main/resources/features.xml  |    11 +-
 .../include/ignite/common/platform_utils.h      |     7 +
 .../os/linux/src/common/platform_utils.cpp      |    15 +
 .../common/os/win/src/common/platform_utils.cpp |     5 +
 .../cpp/core-test/config/cache-identity-32.xml  |     5 +-
 .../cpp/core-test/config/cache-query-32.xml     |     6 +-
 .../config/cache-query-continuous-32.xml        |     6 +-
 .../cpp/core-test/config/cache-store-32.xml     |     4 +-
 .../cpp/core-test/config/cache-test-32.xml      |     6 +-
 .../cpp/core-test/config/isolated-32.xml        |     5 +-
 .../core-test/config/persistence-store-32.xml   |     5 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   155 +-
 modules/platforms/cpp/odbc-test/Makefile.am     |     8 +-
 .../cpp/odbc-test/config/queries-auth-32.xml    |    62 +
 .../cpp/odbc-test/config/queries-auth.xml       |    44 +
 .../cpp/odbc-test/config/queries-ssl-32.xml     |     4 +-
 .../cpp/odbc-test/config/queries-test-32.xml    |     4 +-
 .../cpp/odbc-test/include/odbc_test_suite.h     |    58 +
 .../cpp/odbc-test/project/vs/odbc-test.vcxproj  |     9 +
 .../project/vs/odbc-test.vcxproj.filters        |    29 +-
 .../cpp/odbc-test/src/api_robustness_test.cpp   |    25 -
 .../cpp/odbc-test/src/attributes_test.cpp       |   190 +-
 .../cpp/odbc-test/src/authentication_test.cpp   |   175 +
 .../cpp/odbc-test/src/configuration_test.cpp    |   223 +-
 .../cpp/odbc-test/src/odbc_test_suite.cpp       |    65 +-
 .../cpp/odbc-test/src/queries_ssl_test.cpp      |    55 +-
 .../cpp/odbc-test/src/queries_test.cpp          |    61 +-
 .../cpp/odbc-test/src/sql_get_info_test.cpp     |    84 +-
 .../src/sql_value_expressions_test.cpp          |     1 -
 modules/platforms/cpp/odbc/Makefile.am          |     2 +
 .../include/ignite/odbc/config/config_tools.h   |    92 +
 .../include/ignite/odbc/config/configuration.h  |   559 +-
 .../odbc/config/connection_string_parser.h      |   194 +
 .../include/ignite/odbc/config/settable_value.h |   101 +
 .../cpp/odbc/include/ignite/odbc/connection.h   |    74 +-
 .../odbc/diagnostic/diagnosable_adapter.h       |    26 +-
 .../odbc/diagnostic/diagnostic_record_storage.h |    13 +-
 .../cpp/odbc/include/ignite/odbc/end_point.h    |    72 +
 .../cpp/odbc/include/ignite/odbc/message.h      |    34 +-
 .../odbc/include/ignite/odbc/protocol_version.h |     3 +
 .../odbc/include/ignite/odbc/socket_client.h    |     8 +-
 .../ignite/odbc/ssl/secure_socket_client.h      |     4 +-
 .../cpp/odbc/include/ignite/odbc/ssl/ssl_mode.h |    13 +-
 .../ignite/odbc/system/tcp_socket_client.h      |    11 +-
 .../odbc/system/ui/dsn_configuration_window.h   |    38 +
 .../cpp/odbc/include/ignite/odbc/utility.h      |     3 +-
 .../os/linux/src/system/tcp_socket_client.cpp   |     5 +-
 .../os/win/src/system/tcp_socket_client.cpp     |    12 +-
 .../src/system/ui/dsn_configuration_window.cpp  |   211 +-
 .../cpp/odbc/os/win/src/system_dsn.cpp          |    24 +-
 .../platforms/cpp/odbc/project/vs/odbc.vcxproj  |    10 +-
 .../cpp/odbc/project/vs/odbc.vcxproj.filters    |    18 +
 .../cpp/odbc/src/config/config_tools.cpp        |   250 +
 .../cpp/odbc/src/config/configuration.cpp       |   544 +-
 .../src/config/connection_string_parser.cpp     |   462 +
 modules/platforms/cpp/odbc/src/connection.cpp   |   219 +-
 .../diagnostic/diagnostic_record_storage.cpp    |     5 +
 modules/platforms/cpp/odbc/src/dsn_config.cpp   |   193 +-
 modules/platforms/cpp/odbc/src/environment.cpp  |     7 +-
 modules/platforms/cpp/odbc/src/message.cpp      |    30 +-
 modules/platforms/cpp/odbc/src/odbc.cpp         |    12 +-
 .../platforms/cpp/odbc/src/protocol_version.cpp |     4 +-
 .../cpp/odbc/src/ssl/secure_socket_client.cpp   |     3 +-
 .../platforms/cpp/odbc/src/ssl/ssl_gateway.cpp  |     8 +-
 modules/platforms/cpp/odbc/src/ssl/ssl_mode.cpp |    18 +-
 .../IgniteSessionStateStoreProviderTest.cs      |     7 +-
 .../Apache.Ignite.Benchmarks/BenchmarkRunner.cs |     1 -
 .../ApiParity/CacheConfigurationParityTest.cs   |     3 +-
 .../ApiParity/IgniteConfigurationParityTest.cs  |     6 +-
 .../QueryEntityConfigurationParityTest.cs       |     5 +-
 .../Binary/BinaryDynamicRegistrationTest.cs     |     2 +-
 .../Cache/CacheAbstractTransactionalTest.cs     |     5 +-
 .../Cache/CacheConfigurationTest.cs             |     2 +
 .../Cache/DataStorageMetricsTest.cs             |     2 +-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |    11 +-
 .../Cache/Query/CacheQueriesTest.cs             |     6 +-
 .../Cache/Query/Linq/CacheLinqTest.Base.cs      |     4 +
 .../Cache/Query/Linq/CacheLinqTest.Custom.cs    |   195 +
 .../Cache/Query/Linq/CacheLinqTest.Functions.cs |    48 +
 .../Cache/Query/Linq/CacheLinqTest.Misc.cs      |     4 +-
 .../Cache/Query/Linq/CacheLinqTest.Numerics.cs  |     4 +-
 .../Cache/Store/CacheStoreTest.cs               |    32 +
 .../Client/Cache/CacheTestNoMeta.cs             |     2 +-
 .../Client/ClientConnectionTest.cs              |   169 +-
 .../Config/full-config.xml                      |     2 +-
 .../Apache.Ignite.Core.Tests/EventsTest.cs      |     9 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |     3 +-
 .../IgniteConfigurationSerializerTest.cs        |     1 +
 .../IgniteConfigurationTest.cs                  |     4 +-
 .../IgniteStartStopTest.cs                      |     5 +-
 .../Apache.Ignite.Core.Tests/MessagingTest.cs   |     5 +-
 .../Apache.Ignite.Core.csproj                   |     1 +
 .../Cache/Configuration/CacheConfiguration.cs   |     2 +
 .../Cache/Configuration/QueryEntity.cs          |     4 +-
 .../Cache/Configuration/QueryField.cs           |    16 +-
 .../Configuration/QuerySqlFieldAttribute.cs     |    10 +
 .../Apache.Ignite.Core/Cache/ICacheMetrics.cs   |     9 +
 .../Client/ClientStatusCode.cs                  |    12 +-
 .../Client/IgniteClientConfiguration.cs         |    13 +
 .../Apache.Ignite.Core/Compute/IComputeTask.cs  |     2 +
 .../IgniteClientConfigurationSection.xsd        |    10 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |    20 +
 .../IgniteConfigurationSection.xsd              |    15 +
 .../Impl/Binary/BinaryReader.cs                 |    10 +-
 .../Impl/Binary/BinaryTypeId.cs                 |     4 +-
 .../Impl/Cache/CacheMetricsImpl.cs              |     7 +
 .../Query/Continuous/ContinuousQueryFilter.cs   |     2 +
 .../Impl/Client/ClientProtocolVersion.cs        |    22 +-
 .../Impl/Client/ClientSocket.cs                 |    89 +-
 .../Impl/Common/DelegateTypeDescriptor.cs       |     1 +
 .../Impl/Common/IgniteArgumentCheck.cs          |     1 +
 .../Impl/Common/TaskRunner.cs                   |    70 +
 .../Impl/Datastream/DataStreamerBatch.cs        |     2 +-
 .../Impl/Datastream/DataStreamerImpl.cs         |     2 +-
 .../Apache.Ignite.Core/Impl/Events/Events.cs    |     2 +-
 .../Impl/Handle/HandleRegistry.cs               |     1 +
 .../Impl/Transactions/TransactionImpl.cs        |     3 +-
 .../Impl/Unmanaged/Jni/Jvm.cs                   |     1 +
 .../Apache.Ignite.Core/Ssl/SslContextFactory.cs |     8 +-
 .../Apache.Ignite.Linq.csproj                   |     4 +
 .../Apache.Ignite.Linq/CacheLinqExtensions.cs   |    28 +
 .../Apache.Ignite.Linq/IUpdateDescriptor.cs     |    51 +
 .../Apache.Ignite.Linq/Impl/AliasDictionary.cs  |     8 +-
 .../Impl/CacheFieldsQueryProvider.cs            |    23 +-
 .../Impl/CacheQueryExpressionVisitor.cs         |    43 +-
 .../Impl/CacheQueryModelVisitor.cs              |   147 +-
 .../Apache.Ignite.Linq/Impl/CacheQueryParser.cs |     3 +
 .../Impl/Dml/MemberUpdateContainer.cs           |    38 +
 .../Impl/Dml/UpdateAllExpressionNode.cs         |   138 +
 .../Impl/Dml/UpdateAllResultOperator.cs         |    75 +
 .../Apache.Ignite.Linq/Impl/ExpressionWalker.cs |     1 +
 modules/platforms/dotnet/DEVNOTES.txt           |     4 +-
 modules/platforms/dotnet/build-mono.sh          |     5 +
 modules/platforms/dotnet/build.ps1              |     6 +-
 modules/platforms/dotnet/build.sh               |     5 +
 .../Datagrid/QueryExample.cs                    |     2 +-
 .../Apache.Ignite.Examples/Sql/LinqExample.cs   |     5 +-
 .../Apache.Ignite.Examples/Sql/SqlExample.cs    |     5 +-
 .../http/jetty/GridJettyObjectMapper.java       |    61 +-
 .../http/jetty/GridJettyRestHandler.java        |    67 +-
 modules/spark-2.10/pom.xml                      |   167 +-
 modules/spark/pom.xml                           |   200 +-
 .../org/apache/ignite/spark/IgniteRDD.scala     |     9 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |     4 -
 .../repository/query/IgniteQueryGenerator.java  |    19 +-
 .../IgniteSpringDataQueriesSelfTest.java        |     9 +
 .../springdata/misc/PersonRepository.java       |     4 +
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java |    11 +-
 .../ignite/internal/GridFactorySelfTest.java    |     3 +-
 .../resource/GridServiceInjectionSelfTest.java  |    64 +-
 .../GridSpringResourceInjectionSelfTest.java    |    58 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |     5 +-
 modules/sqlline/bin/sqlline.bat                 |     3 +-
 .../commands/cache/VisorCacheCommand.scala      |    34 +-
 .../cache/VisorCacheLostPartitionsCommand.scala |   170 +
 .../VisorCacheResetLostPartitionsCommand.scala  |   132 +
 modules/web-console/assembly/README.txt         |    26 +-
 modules/web-console/backend/app/apiServer.js    |     9 +-
 modules/web-console/backend/app/mongo.js        |     3 +-
 modules/web-console/backend/app/routes.js       |     5 +-
 modules/web-console/backend/app/schemas.js      |    38 +-
 .../backend/errors/AppErrorException.js         |     2 -
 .../backend/errors/AuthFailedException.js       |     2 +-
 .../backend/errors/IllegalAccessError.js        |     3 +-
 .../backend/errors/IllegalArgumentException.js  |     1 -
 .../backend/errors/MissingResourceException.js  |     2 +-
 .../backend/errors/ServerErrorException.js      |     1 -
 modules/web-console/backend/index.js            |     4 +-
 modules/web-console/backend/middlewares/api.js  |    15 +-
 modules/web-console/backend/middlewares/demo.js |    31 +
 .../1502249492000-invalidate_rename.js          |    28 +
 .../migrations/1502432624000-cache-index.js     |    32 +
 .../migrations/1504672035000-igfs-index.js      |    32 +
 .../migrations/1505114649000-models-index.js    |    32 +
 .../1508395969410-init-registered-date.js       |     7 +-
 .../migrations/1516948939797-migrate-configs.js |   346 +
 .../backend/migrations/migration-utils.js       |   153 +
 .../backend/migrations/recreate-index.js        |    30 -
 modules/web-console/backend/routes/caches.js    |    12 +
 modules/web-console/backend/routes/clusters.js  |    46 +-
 .../web-console/backend/routes/configuration.js |    12 +-
 modules/web-console/backend/routes/demo.js      |     2 +
 modules/web-console/backend/routes/domains.js   |     6 +
 modules/web-console/backend/routes/igfss.js     |    12 +
 modules/web-console/backend/services/caches.js  |    78 +-
 .../web-console/backend/services/clusters.js    |   153 +-
 .../backend/services/configurations.js          |    12 +
 modules/web-console/backend/services/domains.js |    96 +-
 modules/web-console/backend/services/igfss.js   |    44 +-
 .../web-console/backend/services/sessions.js    |     2 +-
 modules/web-console/backend/services/spaces.js  |     2 +-
 .../backend/test/unit/CacheService.test.js      |    45 +-
 .../backend/test/unit/ClusterService.test.js    |   233 +-
 .../backend/test/unit/DomainService.test.js     |     5 +
 .../docker/compose/backend/Dockerfile           |     2 +-
 .../docker/compose/frontend/DockerfileBuild     |     2 +-
 .../web-console/docker/standalone/Dockerfile    |     2 +-
 modules/web-console/e2e/docker-compose.yml      |     3 +-
 modules/web-console/e2e/testcafe/.eslintrc      |     1 +
 .../e2e/testcafe/components/FormField.js        |    72 +
 .../e2e/testcafe/components/ListEditable.js     |    83 +
 .../e2e/testcafe/components/PanelCollapsible.js |    28 +
 .../e2e/testcafe/components/Table.js            |    56 +
 .../e2e/testcafe/components/confirmation.js     |    39 +
 .../e2e/testcafe/components/modalInput.js       |    40 +
 .../e2e/testcafe/components/notifications.js    |    21 +
 .../components/pageAdvancedConfiguration.js     |    29 +
 .../testcafe/components/pageConfiguration.js    |    21 +
 .../e2e/testcafe/components/topNavigation.js    |    21 +
 .../e2e/testcafe/components/userMenu.js         |    34 +
 modules/web-console/e2e/testcafe/envtools.js    |     2 +-
 .../e2e/testcafe/fixtures/admin-panel.js        |     7 +-
 .../web-console/e2e/testcafe/fixtures/auth.js   |   186 -
 .../testcafe/fixtures/auth/forgot-password.js   |    58 +
 .../e2e/testcafe/fixtures/auth/logout.js        |    39 +
 .../e2e/testcafe/fixtures/auth/signup.js        |    91 +
 .../testcafe/fixtures/configuration/basic.js    |    89 +
 .../testcafe/fixtures/configuration/overview.js |   147 +
 .../e2e/testcafe/fixtures/menu-smoke.js         |    17 +-
 .../testcafe/fixtures/queries/notebooks-list.js |    59 +
 .../fixtures/user-profile/credentials.js        |    34 +-
 .../testcafe/fixtures/user-profile/profile.js   |    60 +-
 modules/web-console/e2e/testcafe/helpers.js     |    11 +-
 modules/web-console/e2e/testcafe/package.json   |     3 +-
 .../PageConfigurationAdvancedCluster.js         |    28 +
 .../page-models/PageConfigurationBasic.js       |    68 +
 .../page-models/PageConfigurationOverview.js    |    36 +
 .../e2e/testcafe/page-models/PageQueries.js     |    69 +
 .../e2e/testcafe/page-models/PageSignIn.js      |    52 +-
 .../pageConfigurationAdvancedIGFS.js            |    21 +
 .../pageConfigurationAdvancedModels.js          |    28 +
 .../e2e/testcafe/page-models/pageProfile.js     |    40 +
 modules/web-console/e2e/testcafe/roles.js       |     1 -
 modules/web-console/e2e/testcafe/testcafe.js    |     2 +-
 modules/web-console/e2e/testenv/Dockerfile      |     2 +-
 modules/web-console/frontend/.babelrc           |     2 +-
 modules/web-console/frontend/.eslintrc          |     3 +-
 modules/web-console/frontend/.gitignore         |     5 +
 modules/web-console/frontend/app/app.config.js  |    47 +-
 modules/web-console/frontend/app/app.d.ts       |    29 +
 modules/web-console/frontend/app/app.js         |    43 +-
 .../app/components/breadcrumbs/component.js     |    43 +
 .../app/components/breadcrumbs/index.js         |    23 +
 .../app/components/breadcrumbs/style.scss       |    48 +
 .../app/components/breadcrumbs/template.pug     |    17 +
 .../app/components/bs-select-menu/style.scss    |     4 +-
 .../components/cluster-selector/controller.js   |     4 +-
 .../directives.js                               |    53 +
 .../expose-ignite-form-field-control/index.js   |    23 +
 .../grid-column-selector/template.pug           |     1 +
 .../components/grid-item-selected/controller.js |     2 +-
 .../app/components/ignite-icon/directive.js     |     2 +-
 .../app/components/ignite-icon/style.scss       |     8 +-
 .../list-editable-add-item-button/component.js  |    86 +
 .../component.spec.js                           |    72 +
 .../has-items-template.pug                      |    23 +
 .../list-editable-add-item-button/index.js      |    24 +
 .../no-items-template.pug                       |    18 +
 .../list-editable-add-item-button/style.scss    |    21 +
 .../list-editable-cols/cols.directive.js        |     5 +-
 .../list-editable-cols/cols.style.scss          |    16 +-
 .../list-editable-cols/cols.template.pug        |     2 +-
 .../components/list-editable-cols/index.js      |     3 +-
 .../list-editable-cols/row.directive.js         |     4 +-
 .../list-editable-one-way/directive.js          |    54 +
 .../components/list-editable-one-way/index.js   |    24 +
 .../list-editable-save-on-changes/directives.js |    76 +
 .../list-editable-save-on-changes/index.js      |    24 +
 .../list-editable-transclude/directive.js       |     3 +
 .../app/components/list-editable/controller.js  |    59 +-
 .../app/components/list-editable/index.js       |     8 +-
 .../app/components/list-editable/style.scss     |    31 +-
 .../app/components/list-editable/template.pug   |     9 +-
 .../list-of-registered-users/column-defs.js     |    20 +-
 .../list-of-registered-users/controller.js      |    14 +-
 .../frontend/app/components/page-admin/index.js |    39 +
 .../app/components/page-admin/style.scss        |    62 +
 .../app/components/page-admin/template.tpl.pug  |    28 +
 .../components/cache-edit-form/component.js     |    32 +
 .../components/cache-edit-form/controller.js    |   103 +
 .../components/cache-edit-form/index.js         |    21 +
 .../components/cache-edit-form/style.scss       |    20 +
 .../components/cache-edit-form/template.tpl.pug |    46 +
 .../cache-edit-form/templates/affinity.pug      |    86 +
 .../cache-edit-form/templates/concurrency.pug   |    64 +
 .../cache-edit-form/templates/general.pug       |   113 +
 .../cache-edit-form/templates/memory.pug        |   158 +
 .../templates/near-cache-client.pug             |    50 +
 .../templates/near-cache-server.pug             |    51 +
 .../cache-edit-form/templates/node-filter.pug   |    53 +
 .../cache-edit-form/templates/query.pug         |   114 +
 .../cache-edit-form/templates/rebalance.pug     |    66 +
 .../cache-edit-form/templates/statistics.pug    |    34 +
 .../cache-edit-form/templates/store.pug         |   310 +
 .../components/cluster-edit-form/component.js   |    31 +
 .../components/cluster-edit-form/controller.js  |   116 +
 .../components/cluster-edit-form/index.js       |    21 +
 .../components/cluster-edit-form/style.scss     |    20 +
 .../cluster-edit-form/template.tpl.pug          |    87 +
 .../cluster-edit-form/templates/atomic.pug      |    75 +
 .../cluster-edit-form/templates/attributes.pug  |    40 +
 .../cluster-edit-form/templates/binary.pug      |    80 +
 .../templates/cache-key-cfg.pug                 |    63 +
 .../cluster-edit-form/templates/checkpoint.pug  |    82 +
 .../templates/checkpoint/fs.pug                 |    36 +
 .../templates/checkpoint/jdbc.pug               |    47 +
 .../templates/checkpoint/s3.pug                 |   204 +
 .../templates/client-connector.pug              |    76 +
 .../cluster-edit-form/templates/collision.pug   |    58 +
 .../templates/collision/custom.pug              |    23 +
 .../templates/collision/fifo-queue.pug          |    26 +
 .../templates/collision/job-stealing.pug        |    51 +
 .../templates/collision/priority-queue.pug      |    41 +
 .../templates/communication.pug                 |   134 +
 .../cluster-edit-form/templates/connector.pug   |   100 +
 .../templates/data-storage.pug                  |   301 +
 .../cluster-edit-form/templates/deployment.pug  |   192 +
 .../cluster-edit-form/templates/discovery.pug   |    97 +
 .../cluster-edit-form/templates/events.pug      |    66 +
 .../cluster-edit-form/templates/failover.pug    |    89 +
 .../cluster-edit-form/templates/general.pug     |    89 +
 .../templates/general/discovery/cloud.pug       |    78 +
 .../templates/general/discovery/google.pug      |    38 +
 .../templates/general/discovery/jdbc.pug        |    35 +
 .../templates/general/discovery/kubernetes.pug  |    38 +
 .../templates/general/discovery/multicast.pug   |    63 +
 .../templates/general/discovery/s3.pug          |    38 +
 .../templates/general/discovery/shared.pug      |    24 +
 .../templates/general/discovery/vm.pug          |    55 +
 .../templates/general/discovery/zookeeper.pug   |    84 +
 .../retrypolicy/bounded-exponential-backoff.pug |    26 +
 .../discovery/zookeeper/retrypolicy/custom.pug  |    25 +
 .../retrypolicy/exponential-backoff.pug         |    26 +
 .../discovery/zookeeper/retrypolicy/forever.pug |    23 +
 .../discovery/zookeeper/retrypolicy/n-times.pug |    24 +
 .../zookeeper/retrypolicy/one-time.pug          |    23 +
 .../zookeeper/retrypolicy/until-elapsed.pug     |    24 +
 .../cluster-edit-form/templates/hadoop.pug      |    87 +
 .../cluster-edit-form/templates/igfs.pug        |    34 +
 .../templates/load-balancing.pug                |   115 +
 .../cluster-edit-form/templates/logger.pug      |    60 +
 .../templates/logger/custom.pug                 |    24 +
 .../templates/logger/log4j.pug                  |    49 +
 .../templates/logger/log4j2.pug                 |    38 +
 .../cluster-edit-form/templates/marshaller.pug  |    75 +
 .../cluster-edit-form/templates/memory.pug      |   195 +
 .../cluster-edit-form/templates/metrics.pug     |    46 +
 .../cluster-edit-form/templates/misc.pug        |    58 +
 .../cluster-edit-form/templates/odbc.pug        |    70 +
 .../cluster-edit-form/templates/persistence.pug |    82 +
 .../cluster-edit-form/templates/service.pug     |    89 +
 .../templates/sql-connector.pug                 |    58 +
 .../cluster-edit-form/templates/ssl.pug         |    89 +
 .../cluster-edit-form/templates/swap.pug        |    74 +
 .../cluster-edit-form/templates/thread.pug      |   144 +
 .../cluster-edit-form/templates/time.pug        |    44 +
 .../templates/transactions.pug                  |    65 +
 .../components/igfs-edit-form/component.js      |    30 +
 .../components/igfs-edit-form/controller.js     |    57 +
 .../components/igfs-edit-form/index.js          |    21 +
 .../components/igfs-edit-form/style.scss        |    20 +
 .../components/igfs-edit-form/template.tpl.pug  |    38 +
 .../igfs-edit-form/templates/dual.pug           |    42 +
 .../igfs-edit-form/templates/fragmentizer.pug   |    37 +
 .../igfs-edit-form/templates/general.pug        |    72 +
 .../components/igfs-edit-form/templates/ipc.pug |    55 +
 .../igfs-edit-form/templates/misc.pug           |   110 +
 .../igfs-edit-form/templates/secondary.pug      |    55 +
 .../components/model-edit-form/component.js     |    31 +
 .../components/model-edit-form/controller.js    |   187 +
 .../components/model-edit-form/index.js         |    21 +
 .../components/model-edit-form/style.scss       |    20 +
 .../components/model-edit-form/template.tpl.pug |    32 +
 .../model-edit-form/templates/general.pug       |    57 +
 .../model-edit-form/templates/query.pug         |   255 +
 .../model-edit-form/templates/store.pug         |   123 +
 .../page-configure-advanced-caches/component.js |    25 +
 .../controller.js                               |   174 +
 .../page-configure-advanced-caches/index.js     |    23 +
 .../page-configure-advanced-caches/template.pug |    57 +
 .../component.js                                |    25 +
 .../controller.js                               |    51 +
 .../page-configure-advanced-cluster/index.js    |    23 +
 .../template.pug                                |    25 +
 .../page-configure-advanced-igfs/component.js   |    25 +
 .../page-configure-advanced-igfs/controller.js  |   139 +
 .../page-configure-advanced-igfs/index.js       |    23 +
 .../page-configure-advanced-igfs/template.pug   |    51 +
 .../page-configure-advanced-models/component.js |    26 +
 .../controller.js                               |   171 +
 .../hasIndex.template.pug                       |    23 +
 .../page-configure-advanced-models/index.js     |    23 +
 .../keyCell.template.pug                        |    21 +
 .../page-configure-advanced-models/style.scss   |    37 +
 .../page-configure-advanced-models/template.pug |    51 +
 .../valueCell.template.pug                      |    18 +
 .../page-configure-advanced/controller.js       |    15 +-
 .../components/page-configure-advanced/index.js |    23 +-
 .../page-configure-advanced/service.js          |    31 -
 .../page-configure-advanced/style.scss          |   136 +-
 .../page-configure-advanced/template.pug        |    14 +-
 .../components/pcbScaleNumber.js                |    46 -
 .../page-configure-basic/controller.js          |   242 +-
 .../page-configure-basic/controller.spec.js     |    19 +-
 .../components/page-configure-basic/index.js    |    11 +-
 .../mixins/pcb-form-field-size.pug              |    71 -
 .../components/page-configure-basic/reducer.js  |    17 +-
 .../page-configure-basic/reducer.spec.js        |     2 +-
 .../components/page-configure-basic/service.js  |   134 -
 .../page-configure-basic/service.spec.js        |   323 -
 .../components/page-configure-basic/style.scss  |   131 +-
 .../page-configure-basic/template.pug           |   299 +-
 .../page-configure-overview/component.js        |    25 +
 .../pco-grid-column-categories/directive.js     |    67 +
 .../page-configure-overview/controller.js       |   163 +
 .../components/page-configure-overview/index.js |    26 +
 .../page-configure-overview/style.scss          |    33 +
 .../page-configure-overview/template.pug        |    40 +
 .../app/components/page-configure/component.js  |     5 +-
 .../button-download-project/component.js        |    36 +
 .../components/button-download-project/index.js |    23 +
 .../button-download-project/template.pug        |    22 +
 .../button-import-models/component.js           |    37 +
 .../components/button-import-models/index.js    |    23 +
 .../components/button-import-models/style.scss  |    25 +
 .../button-import-models/template.pug           |    20 +
 .../button-preview-project/component.js         |    36 +
 .../components/button-preview-project/index.js  |    23 +
 .../button-preview-project/template.pug         |    22 +
 .../page-configure/components/fakeUICanExit.js  |    48 +
 .../components/formUICanExitGuard.js            |    59 +
 .../components/modal-import-models/component.js |  1151 ++
 .../components/modal-import-models/index.js     |    31 +
 .../component.js                                |    27 +
 .../selected-items-amount-indicator/style.scss  |    24 +
 .../template.pug                                |    17 +
 .../components/modal-import-models/service.js   |    56 +
 .../step-indicator/component.js                 |    35 +
 .../step-indicator/style.scss                   |   101 +
 .../step-indicator/template.pug                 |    31 +
 .../components/modal-import-models/style.scss   |    53 +
 .../tables-action-cell/component.js             |    62 +
 .../tables-action-cell/style.scss               |    49 +
 .../tables-action-cell/template.pug             |    45 +
 .../modal-import-models/template.tpl.pug        |   181 +
 .../modal-preview-project/component.js          |    31 +
 .../modal-preview-project/controller.js         |   120 +
 .../components/modal-preview-project/index.js   |    27 +
 .../components/modal-preview-project/service.js |    52 +
 .../components/modal-preview-project/style.scss |    67 +
 .../modal-preview-project/template.pug          |    47 +
 .../components/pc-form-field-size/component.js  |    41 +
 .../components/pc-form-field-size/controller.js |   131 +
 .../components/pc-form-field-size/index.js      |    23 +
 .../components/pc-form-field-size/style.scss    |    52 +
 .../components/pc-form-field-size/template.pug  |    61 +
 .../components/pc-items-table/component.js      |    45 +
 .../components/pc-items-table/controller.js     |   125 +
 .../components/pc-items-table/decorator.js      |    34 +
 .../components/pc-items-table/index.js          |    25 +
 .../components/pc-items-table/style.scss        |    71 +
 .../components/pc-items-table/template.pug      |    49 +
 .../components/pc-ui-grid-filters/directive.js  |    62 +
 .../components/pc-ui-grid-filters/index.js      |    43 +
 .../components/pc-ui-grid-filters/style.scss    |    22 +
 .../components/pc-ui-grid-filters/template.pug  |    39 +
 .../components/pcIsInCollection.js              |    41 +
 .../page-configure/components/pcValidation.js   |   193 +
 .../components/preview-panel/directive.js       |   246 +
 .../components/preview-panel/index.js           |    23 +
 .../app/components/page-configure/controller.js |    35 +-
 .../components/page-configure/defaultNames.js   |    23 +
 .../app/components/page-configure/index.d.ts    |   151 +
 .../app/components/page-configure/index.js      |   159 +-
 .../app/components/page-configure/reducer.js    |   353 +-
 .../components/page-configure/reducer.spec.js   |    21 +-
 .../page-configure/reduxDevtoolsIntegration.js  |    75 +
 .../services/ConfigChangesGuard.js              |   100 +
 .../services/ConfigChangesGuard.spec.js         |    40 +
 .../services/ConfigSelectionManager.js          |    93 +
 .../services/ConfigurationDownload.js           |    23 +-
 .../services/ConfigurationDownload.spec.js      |     2 +-
 .../services/ConfigurationResource.js           |    49 +
 .../page-configure/services/ConfigureState.js   |    90 +-
 .../page-configure/services/PageConfigure.js    |    86 +-
 .../services/PageConfigure.spec.js              |   244 +
 .../page-configure/services/SummaryZipper.js    |    44 +
 .../page-configure/services/summary.worker.js   |   147 +
 .../app/components/page-configure/states.js     |   270 +
 .../page-configure/store/actionCreators.js      |   170 +
 .../page-configure/store/actionTypes.js         |    31 +
 .../components/page-configure/store/effects.js  |   664 +
 .../page-configure/store/selectors.js           |   170 +
 .../app/components/page-configure/style.scss    |   285 +-
 .../app/components/page-configure/template.pug  |    43 +-
 .../transitionHooks/errorState.js               |    55 +
 .../page-configure/types/uirouter.d.ts          |    20 +
 .../app/components/page-landing/index.js        |     4 +-
 .../app/components/page-profile/controller.js   |    29 +-
 .../app/components/page-profile/style.scss      |     2 +-
 .../app/components/page-profile/template.pug    |    32 +-
 .../components/page-queries/Notebook.data.js    |   168 -
 .../components/page-queries/Notebook.service.js |    74 -
 .../app/components/page-queries/component.js    |    50 +
 .../components/queries-notebook/controller.js   |  1949 +++
 .../components/queries-notebook/index.js        |    29 +
 .../components/queries-notebook/style.scss      |   111 +
 .../queries-notebook/template.tpl.pug           |   414 +
 .../queries-notebooks-list/controller.js        |   224 +
 .../components/queries-notebooks-list/index.js  |    28 +
 .../queries-notebooks-list/style.scss           |    36 +
 .../queries-notebooks-list/template.tpl.pug     |    53 +
 .../app/components/page-queries/controller.js   |  1946 ---
 .../app/components/page-queries/index.js        |   118 +-
 .../page-queries/notebook.controller.js         |    62 -
 .../components/page-queries/notebook.data.js    |   172 +
 .../components/page-queries/notebook.service.js |    91 +
 .../app/components/page-queries/style.scss      |    91 -
 .../components/page-queries/template.tpl.pug    |   408 +-
 .../app/components/page-signin/controller.js    |   103 +-
 .../app/components/page-signin/template.pug     |    73 +-
 .../components/panel-collapsible/component.js   |    39 +
 .../components/panel-collapsible/controller.js  |    52 +
 .../app/components/panel-collapsible/index.js   |    25 +
 .../components/panel-collapsible/index.spec.js  |   140 +
 .../app/components/panel-collapsible/style.scss |    79 +
 .../components/panel-collapsible/template.pug   |    24 +
 .../panel-collapsible/transcludeDirective.js    |    51 +
 .../app/components/version-picker/style.scss    |     3 +-
 .../app/components/version-picker/template.pug  |    14 +-
 .../app/core/activities/Activities.data.d.ts    |    37 +
 .../app/core/activities/Activities.data.js      |    10 +-
 .../frontend/app/data/getting-started.json      |    17 -
 modules/web-console/frontend/app/data/i18n.js   |    18 +-
 .../app/directives/on-focus-out.directive.js    |   100 +-
 .../directives/ui-ace-docker/ui-ace-docker.pug  |    10 +-
 .../app/directives/ui-ace-java/index.js         |    26 +
 .../ui-ace-java/ui-ace-java.controller.js       |   108 +-
 .../ui-ace-java/ui-ace-java.directive.js        |    43 +-
 .../ui-ace-pojos/ui-ace-pojos.controller.js     |     4 +-
 .../app/directives/ui-ace-spring/index.js       |    26 +
 .../ui-ace-spring/ui-ace-spring.controller.js   |   104 +-
 .../ui-ace-spring/ui-ace-spring.directive.js    |    46 +-
 .../app/directives/ui-ace.controller.js         |   123 +
 .../frontend/app/filters/id8.filter.js          |     8 +-
 .../frontend/app/helpers/jade/form.pug          |     2 -
 .../helpers/jade/form/form-field-checkbox.pug   |    43 +-
 .../helpers/jade/form/form-field-datalist.pug   |    25 +-
 .../helpers/jade/form/form-field-dropdown.pug   |    34 +-
 .../helpers/jade/form/form-field-feedback.pug   |    16 +-
 .../app/helpers/jade/form/form-field-label.pug  |     8 +-
 .../app/helpers/jade/form/form-field-number.pug |    41 +-
 .../helpers/jade/form/form-field-password.pug   |    26 +-
 .../app/helpers/jade/form/form-field-text.pug   |    28 +-
 .../app/helpers/jade/form/form-group.pug        |    23 -
 .../frontend/app/helpers/jade/mixins.pug        |   400 +-
 .../frontend/app/modules/ace.module.js          |    47 +-
 .../app/modules/agent/AgentManager.service.js   |    13 +-
 .../generator/AbstractTransformer.js            |     8 +-
 .../modules/configuration/generator/Beans.js    |    60 +-
 .../generator/ConfigurationGenerator.js         |    88 +-
 .../generator/JavaTransformer.service.js        |    24 +-
 .../generator/PlatformGenerator.js              |     8 +-
 .../generator/SpringTransformer.service.js      |     2 +-
 .../generator/defaults/Cluster.service.js       |    15 +-
 .../frontend/app/modules/demo/Demo.module.js    |     6 +-
 .../field/bs-select-placeholder.directive.js    |    20 +-
 .../frontend/app/modules/form/form.module.js    |     6 -
 .../app/modules/form/panel/chevron.directive.js |    17 +-
 .../app/modules/form/panel/field.directive.js   |    69 -
 .../app/modules/form/panel/panel.directive.js   |    37 -
 .../app/modules/form/panel/revert.directive.js  |    54 -
 .../form/validator/java-identifier.directive.js |     5 +-
 .../modules/form/validator/unique.directive.js  |    78 +-
 .../modules/nodes/nodes-dialog.controller.js    |     2 +-
 .../frontend/app/modules/states/admin.state.js  |    15 +-
 .../app/modules/states/configuration.state.js   |   152 -
 .../configuration/Configuration.resource.js     |    42 -
 .../states/configuration/caches/affinity.pug    |    89 -
 .../configuration/caches/client-near-cache.pug  |    50 -
 .../states/configuration/caches/concurrency.pug |    68 -
 .../states/configuration/caches/general.pug     |   100 -
 .../states/configuration/caches/memory.pug      |   136 -
 .../configuration/caches/near-cache-client.pug  |    51 -
 .../configuration/caches/near-cache-server.pug  |    52 -
 .../states/configuration/caches/node-filter.pug |    52 -
 .../states/configuration/caches/query.pug       |   131 -
 .../states/configuration/caches/rebalance.pug   |    66 -
 .../states/configuration/caches/statistics.pug  |    39 -
 .../states/configuration/caches/store.pug       |   255 -
 .../states/configuration/clusters/atomic.pug    |    79 -
 .../configuration/clusters/attributes.pug       |    57 -
 .../states/configuration/clusters/binary.pug    |    77 -
 .../configuration/clusters/cache-key-cfg.pug    |    50 -
 .../configuration/clusters/checkpoint.pug       |    86 -
 .../configuration/clusters/checkpoint/fs.pug    |    66 -
 .../configuration/clusters/checkpoint/jdbc.pug  |    48 -
 .../configuration/clusters/checkpoint/s3.pug    |   178 -
 .../configuration/clusters/client-connector.pug |    84 -
 .../states/configuration/clusters/collision.pug |    63 -
 .../configuration/clusters/collision/custom.pug |    24 -
 .../clusters/collision/fifo-queue.pug           |    27 -
 .../clusters/collision/job-stealing.pug         |    63 -
 .../clusters/collision/priority-queue.pug       |    42 -
 .../configuration/clusters/communication.pug    |   100 -
 .../states/configuration/clusters/connector.pug |   104 -
 .../configuration/clusters/data-storage.pug     |   264 -
 .../configuration/clusters/deployment.pug       |   243 -
 .../states/configuration/clusters/discovery.pug |    92 -
 .../states/configuration/clusters/events.pug    |    70 -
 .../states/configuration/clusters/failover.pug  |    82 -
 .../states/configuration/clusters/general.pug   |    89 -
 .../clusters/general/discovery/cloud.pug        |   138 -
 .../clusters/general/discovery/google.pug       |    38 -
 .../clusters/general/discovery/jdbc.pug         |    32 -
 .../clusters/general/discovery/kubernetes.pug   |    38 -
 .../clusters/general/discovery/multicast.pug    |   102 -
 .../clusters/general/discovery/s3.pug           |    28 -
 .../clusters/general/discovery/shared.pug       |    24 -
 .../clusters/general/discovery/vm.pug           |    82 -
 .../clusters/general/discovery/zookeeper.pug    |    89 -
 .../retrypolicy/bounded-exponential-backoff.pug |    27 -
 .../discovery/zookeeper/retrypolicy/custom.pug  |    24 -
 .../retrypolicy/exponential-backoff.pug         |    27 -
 .../discovery/zookeeper/retrypolicy/forever.pug |    22 -
 .../discovery/zookeeper/retrypolicy/n-times.pug |    25 -
 .../zookeeper/retrypolicy/one-time.pug          |    23 -
 .../zookeeper/retrypolicy/until-elapsed.pug     |    25 -
 .../states/configuration/clusters/hadoop.pug    |   119 -
 .../states/configuration/clusters/igfs.pug      |    38 -
 .../configuration/clusters/load-balancing.pug   |   107 -
 .../states/configuration/clusters/logger.pug    |    66 -
 .../configuration/clusters/logger/custom.pug    |    25 -
 .../configuration/clusters/logger/log4j.pug     |    50 -
 .../configuration/clusters/logger/log4j2.pug    |    39 -
 .../configuration/clusters/marshaller.pug       |    84 -
 .../states/configuration/clusters/memory.pug    |   124 -
 .../states/configuration/clusters/metrics.pug   |    51 -
 .../states/configuration/clusters/misc.pug      |    64 -
 .../states/configuration/clusters/odbc.pug      |    59 -
 .../configuration/clusters/persistence.pug      |    82 -
 .../states/configuration/clusters/service.pug   |    88 -
 .../configuration/clusters/sql-connector.pug    |    60 -
 .../states/configuration/clusters/ssl.pug       |   110 -
 .../states/configuration/clusters/swap.pug      |    72 -
 .../states/configuration/clusters/thread.pug    |    92 -
 .../states/configuration/clusters/time.pug      |    50 -
 .../configuration/clusters/transactions.pug     |    69 -
 .../states/configuration/domains/general.pug    |    52 -
 .../states/configuration/domains/query.pug      |   190 -
 .../states/configuration/domains/store.pug      |   127 -
 .../modules/states/configuration/igfs/dual.pug  |    42 -
 .../states/configuration/igfs/fragmentizer.pug  |    43 -
 .../states/configuration/igfs/general.pug       |    57 -
 .../modules/states/configuration/igfs/ipc.pug   |    60 -
 .../modules/states/configuration/igfs/misc.pug  |   123 -
 .../states/configuration/igfs/secondary.pug     |    45 -
 .../configuration/preview-panel.directive.js    |   239 -
 .../summary/summary-tabs.directive.js           |    50 -
 .../summary/summary-zipper.service.js           |    37 -
 .../configuration/summary/summary.controller.js |   350 -
 .../configuration/summary/summary.worker.js     |   142 -
 .../frontend/app/modules/user/Auth.service.js   |   107 +-
 .../frontend/app/modules/user/user.module.js    |     2 +-
 .../frontend/app/primitives/btn/index.scss      |    21 +
 .../frontend/app/primitives/checkbox/index.scss |    52 +
 .../app/primitives/datepicker/index.pug         |     8 +-
 .../frontend/app/primitives/dropdown/index.pug  |     6 +-
 .../frontend/app/primitives/file/index.pug      |     2 +-
 .../app/primitives/form-field/index.scss        |   108 +-
 .../frontend/app/primitives/index.js            |     1 +
 .../frontend/app/primitives/modal/index.scss    |     1 +
 .../frontend/app/primitives/radio/index.pug     |    12 +-
 .../frontend/app/primitives/tabs/index.scss     |    10 +-
 .../app/primitives/timepicker/index.pug         |     8 +-
 .../frontend/app/primitives/tooltip/index.pug   |     3 +-
 .../frontend/app/primitives/ui-grid/index.scss  |    35 +-
 .../services/AngularStrapSelect.decorator.js    |     5 +-
 .../services/AngularStrapTooltip.decorator.js   |     8 +-
 .../web-console/frontend/app/services/Caches.js |   206 +-
 .../frontend/app/services/Clusters.js           |   483 +-
 .../frontend/app/services/Confirm.service.js    |    38 +
 .../app/services/ConfirmBatch.service.js        |   125 +-
 .../app/services/ErrorPopover.service.js        |    12 +-
 .../frontend/app/services/FormUtils.service.js  |    24 +-
 .../web-console/frontend/app/services/IGFSs.js  |    77 +
 .../frontend/app/services/JavaTypes.service.js  |    27 +-
 .../app/services/LegacyUtils.service.js         |     2 +
 .../frontend/app/services/Messages.service.js   |     6 +-
 .../web-console/frontend/app/services/Models.js |   181 +
 .../frontend/app/services/Version.service.js    |     1 +
 .../web-console/frontend/app/services/index.js  |     2 +
 modules/web-console/frontend/app/utils/id8.js   |    20 +
 .../frontend/app/utils/lodashMixins.js          |    23 +
 .../frontend/app/utils/uniqueName.js            |    27 +
 modules/web-console/frontend/app/vendor.js      |     1 +
 .../frontend/controllers/caches-controller.js   |   653 -
 .../frontend/controllers/clusters-controller.js |  1041 --
 .../frontend/controllers/domains-controller.js  |  1897 ---
 .../frontend/controllers/igfs-controller.js     |   415 -
 modules/web-console/frontend/package-lock.json  | 14506 +++++++++++++++++
 modules/web-console/frontend/package.json       |   101 +-
 .../frontend/public/images/checkbox-active.svg  |     2 +-
 .../frontend/public/images/collapse.svg         |     3 +
 .../frontend/public/images/expand.svg           |     3 +
 .../frontend/public/images/icons/collapse.svg   |     2 +-
 .../frontend/public/images/icons/expand.svg     |     2 +-
 .../frontend/public/images/icons/home.svg       |     3 +
 .../frontend/public/images/icons/index.js       |     9 +-
 .../frontend/public/images/icons/plus.svg       |     2 +
 .../frontend/public/images/icons/structure.svg  |     3 +
 .../stylesheets/_bootstrap-variables.scss       |     2 +-
 .../frontend/public/stylesheets/style.scss      |   173 +-
 .../frontend/test/check-doc-links/Dockerfile    |    31 +
 .../test/check-doc-links/check-doc-links.js     |   150 +
 .../test/check-doc-links/docker-compose.yml     |    25 +
 .../web-console/frontend/test/ci/.dockerignore  |     4 +
 modules/web-console/frontend/test/ci/Dockerfile |    39 +
 .../frontend/test/ci/docker-compose.yml         |    25 +
 .../frontend/test/karma.conf.babel.js           |    20 +-
 .../frontend/test/protractor.conf.js            |    50 -
 modules/web-console/frontend/tsconfig.json      |     6 +-
 modules/web-console/frontend/views/base2.pug    |     4 +-
 .../frontend/views/configuration/caches.tpl.pug |    55 -
 .../views/configuration/clusters.tpl.pug        |    95 -
 .../views/configuration/domains-import.tpl.pug  |   182 -
 .../views/configuration/domains.tpl.pug         |    65 -
 .../frontend/views/configuration/igfs.tpl.pug   |    54 -
 .../summary-project-structure.tpl.pug           |    28 -
 .../views/configuration/summary-tabs.pug        |    25 -
 .../views/configuration/summary.tpl.pug         |    90 -
 .../frontend/views/includes/header-left.pug     |    31 +-
 .../frontend/views/settings/admin.tpl.pug       |    28 -
 .../frontend/views/sql/notebook-new.tpl.pug     |    33 -
 .../views/templates/batch-confirm.tpl.pug       |    29 +-
 .../frontend/views/templates/confirm.tpl.pug    |     2 +-
 .../frontend/webpack/webpack.common.js          |    24 +-
 .../frontend/webpack/webpack.dev.babel.js       |     7 +-
 .../frontend/webpack/webpack.prod.babel.js      |    32 +-
 .../frontend/webpack/webpack.test.js            |    17 +-
 modules/web-console/web-agent/README.txt        |    27 +-
 .../console/agent/AgentConfiguration.java       |     2 +-
 .../ignite/console/agent/AgentLauncher.java     |    11 +-
 .../ignite/console/agent/rest/RestExecutor.java |    94 +-
 .../demo/service/DemoCachesLoadService.java     |    22 +-
 .../ignite-localhost-persistence-config.xml     |    71 +
 .../benchmark-jdbc-thin-inmemory.properties     |   121 +
 .../benchmark-jdbc-thin-persistence.properties  |   128 +
 modules/yardstick/pom-standalone.xml            |     6 +
 modules/yardstick/pom.xml                       |     6 +
 .../yardstick/IgniteBenchmarkArguments.java     |     9 +-
 .../yardstick/jdbc/AbstractJdbcBenchmark.java   |    23 +-
 .../ignite/yardstick/ml/knn/Datasets.java       |   453 -
 .../knn/IgniteKNNClassificationBenchmark.java   |    73 -
 .../ml/knn/IgniteKNNRegressionBenchmark.java    |    82 -
 .../ignite/yardstick/ml/knn/package-info.java   |    22 -
 ...iteOLSMultipleLinearRegressionBenchmark.java |    69 -
 .../yardstick/ml/regression/package-info.java   |    22 -
 .../IgniteColumnDecisionTreeGiniBenchmark.java  |    70 -
 ...niteColumnDecisionTreeVarianceBenchmark.java |    71 -
 .../yardstick/ml/trees/SplitDataGenerator.java  |   426 -
 .../ignite/yardstick/ml/trees/package-info.java |    22 -
 .../upload/AbstractNativeBenchmark.java         |   114 +
 .../upload/AbstractUploadBenchmark.java         |   208 +
 .../upload/BatchedInsertBenchmark.java          |    65 +
 .../ignite/yardstick/upload/CopyBenchmark.java  |   125 +
 .../yardstick/upload/InsertBenchmark.java       |    52 +
 .../yardstick/upload/NativePutBenchmark.java    |    39 +
 .../upload/NativeStreamerBenchmark.java         |    69 +
 .../ignite/yardstick/upload/StreamerParams.java |    46 +
 .../upload/UploadBenchmarkArguments.java        |   180 +
 .../yardstick/upload/model/QueryFactory.java    |   213 +
 .../ignite/yardstick/upload/model/Values10.java |    76 +
 .../org/yardstickframework/package-info.java    |    25 +
 .../probes/TotalTimeProbe.java                  |   137 +
 modules/zookeeper/pom.xml                       |    40 +
 .../spi/discovery/zk/ZookeeperDiscoverySpi.java |   557 +
 .../zk/internal/ZkAbstractCallabck.java         |    83 +
 .../zk/internal/ZkAbstractChildrenCallback.java |    61 +
 .../zk/internal/ZkAbstractWatcher.java          |    55 +
 .../discovery/zk/internal/ZkAliveNodeData.java  |    40 +
 .../zk/internal/ZkBulkJoinContext.java          |    50 +
 .../discovery/zk/internal/ZkClusterNodes.java   |   103 +
 .../internal/ZkCommunicationErrorNodeState.java |    46 +
 .../ZkCommunicationErrorProcessFuture.java      |   411 +
 ...kCommunicationErrorResolveFinishMessage.java |    69 +
 .../ZkCommunicationErrorResolveResult.java      |    45 +
 ...ZkCommunicationErrorResolveStartMessage.java |    61 +
 .../internal/ZkCommunicationFailureContext.java |   188 +
 .../zk/internal/ZkDiscoveryCustomEventData.java |    89 +
 .../zk/internal/ZkDiscoveryEventData.java       |   165 +
 .../zk/internal/ZkDiscoveryEventsData.java      |   121 +
 .../internal/ZkDiscoveryNodeFailEventData.java  |    55 +
 .../internal/ZkDiscoveryNodeJoinEventData.java  |    60 +
 .../ZkDistributedCollectDataFuture.java         |   250 +
 .../zk/internal/ZkForceNodeFailMessage.java     |    65 +
 .../discovery/zk/internal/ZkIgnitePaths.java    |   307 +
 .../zk/internal/ZkInternalJoinErrorMessage.java |    44 +
 .../zk/internal/ZkInternalMessage.java          |    27 +
 .../zk/internal/ZkJoinEventDataForJoined.java   |    83 +
 .../zk/internal/ZkJoinedNodeEvtData.java        |    79 +
 .../zk/internal/ZkJoiningNodeData.java          |    87 +
 .../zk/internal/ZkNoServersMessage.java         |    50 +
 .../zk/internal/ZkNodeValidateResult.java       |    43 +
 .../spi/discovery/zk/internal/ZkRunnable.java   |    51 +
 .../discovery/zk/internal/ZkRuntimeState.java   |   135 +
 .../discovery/zk/internal/ZkTimeoutObject.java  |    54 +
 .../discovery/zk/internal/ZookeeperClient.java  |  1219 ++
 .../ZookeeperClientFailedException.java         |    40 +
 .../zk/internal/ZookeeperClusterNode.java       |   362 +
 .../zk/internal/ZookeeperDiscoveryImpl.java     |  4464 +++++
 .../java/org/apache/ZookeeperNodeStart.java     |    46 +
 ...CacheEntryListenerWithZkDiscoAtomicTest.java |    32 +
 .../ZookeeperDiscoverySpiAbstractTestSuite.java |   118 +
 .../zk/ZookeeperDiscoverySpiTestSuite1.java     |    44 +
 .../zk/ZookeeperDiscoverySpiTestSuite2.java     |    94 +
 ...ZookeeperDiscoverySuitePreprocessorTest.java |   101 +
 .../zk/internal/ZookeeperClientTest.java        |   495 +
 ...okeeperDiscoverySpiSaslAuthAbstractTest.java |   247 +
 ...ZookeeperDiscoverySpiSaslFailedAuthTest.java |    44 +
 ...eeperDiscoverySpiSaslSuccessfulAuthTest.java |    48 +
 .../zk/internal/ZookeeperDiscoverySpiTest.java  |  4847 ++++++
 .../zookeeper/ZkTestClientCnxnSocketNIO.java    |   137 +
 parent/pom.xml                                  |     9 +-
 1866 files changed, 130014 insertions(+), 45323 deletions(-)
----------------------------------------------------------------------



[29/50] [abbrv] ignite git commit: IGNITE-7691: Provide info about DECIMAL column precision and scale. - Fixes #3691.

Posted by ag...@apache.org.
IGNITE-7691: Provide info about DECIMAL column precision and scale. - Fixes #3691.

Signed-off-by: Nikolay Izhikov <ni...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 6df5f99bec46d9b0eabd2e623f0d26b32d031e4c
Parents: 08a700d
Author: Nikolay Izhikov <ni...@apache.org>
Authored: Fri Apr 13 11:53:50 2018 +0300
Committer: Nikolay Izhikov <ni...@apache.org>
Committed: Fri Apr 13 11:53:50 2018 +0300

----------------------------------------------------------------------
 .../jdbc/thin/JdbcThinMetadataSelfTest.java     |  22 +-
 .../org/apache/ignite/cache/QueryEntity.java    |  41 ++-
 .../cache/query/annotations/QuerySqlField.java  |  16 +-
 .../internal/client/thin/ClientUtils.java       |  38 ++-
 .../jdbc/thin/JdbcThinDatabaseMetadata.java     |   6 +-
 .../cache/query/QueryEntityTypeDescriptor.java  |  21 ++
 .../processors/odbc/jdbc/JdbcColumnMeta.java    |  14 +
 .../processors/odbc/jdbc/JdbcColumnMetaV4.java  |  90 +++++++
 .../odbc/jdbc/JdbcConnectionContext.java        |   4 +-
 .../odbc/jdbc/JdbcMetaColumnsResultV4.java      |  50 ++++
 .../odbc/jdbc/JdbcRequestHandler.java           |  14 +-
 .../processors/odbc/jdbc/JdbcResult.java        |   8 +
 .../utils/PlatformConfigurationUtils.java       |  20 ++
 .../processors/query/GridQueryProcessor.java    |   2 +-
 .../processors/query/GridQueryProperty.java     |  14 +
 .../internal/processors/query/QueryField.java   |  26 +-
 .../internal/processors/query/QueryUtils.java   |  28 +-
 .../query/property/QueryBinaryProperty.java     |  23 +-
 .../query/property/QueryClassProperty.java      |  10 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |  13 +-
 .../processors/query/h2/sql/GridSqlColumn.java  |  17 +-
 .../cache/index/IgniteDecimalSelfTest.java      | 265 +++++++++++++++++++
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  12 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   2 +
 .../QueryEntityConfigurationParityTest.cs       |   5 +-
 .../Cache/CacheConfigurationTest.cs             |   2 +
 .../Cache/Configuration/QueryEntity.cs          |   4 +-
 .../Cache/Configuration/QueryField.cs           |  16 +-
 .../Configuration/QuerySqlFieldAttribute.cs     |  10 +
 .../IgniteConfigurationSection.xsd              |  10 +
 30 files changed, 773 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
index 4a34be6..cc6790a 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinMetadataSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.jdbc.thin;
 
 import java.io.Serializable;
+import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 import java.sql.DriverManager;
@@ -143,6 +144,7 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
             stmt.execute("CREATE TABLE \"Quoted\" (\"Id\" INT primary key, \"Name\" VARCHAR(50)) WITH WRAP_KEY");
             stmt.execute("CREATE INDEX \"MyTestIndex quoted\" on \"Quoted\" (\"Id\" DESC)");
             stmt.execute("CREATE INDEX IDX ON TEST (ID ASC)");
+            stmt.execute("CREATE TABLE TEST_DECIMAL_COLUMN (ID INT primary key, DEC_COL DECIMAL(8, 3))");
         }
     }
 
@@ -244,7 +246,8 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "org.ORGANIZATION",
                 "pers.PERSON",
                 "PUBLIC.TEST",
-                "PUBLIC.Quoted"));
+                "PUBLIC.Quoted",
+                "PUBLIC.TEST_DECIMAL_COLUMN"));
 
             Set<String> actualTbls = new HashSet<>(expectedTbls.size());
 
@@ -390,15 +393,25 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "PUBLIC.TEST.VAL.null",
                 "PUBLIC.TEST.AGE.21",
                 "PUBLIC.Quoted.Id.null",
-                "PUBLIC.Quoted.Name.null"));
+                "PUBLIC.Quoted.Name.null",
+                "PUBLIC.TEST_DECIMAL_COLUMN.ID.null",
+                "PUBLIC.TEST_DECIMAL_COLUMN.DEC_COL.null.8.3"
+            ));
 
             Set<String> actualCols = new HashSet<>(expectedCols.size());
 
             while(rs.next()) {
+                int precision = rs.getInt("COLUMN_SIZE");
+
+                int scale = rs.getInt("DECIMAL_DIGITS");
+
                 actualCols.add(rs.getString("TABLE_SCHEM") + '.'
                     + rs.getString("TABLE_NAME") + "."
                     + rs.getString("COLUMN_NAME") + "."
-                    + rs.getString("COLUMN_DEF"));
+                    + rs.getString("COLUMN_DEF")
+                    + (precision == 0 ? "" : ("." + precision))
+                    + (scale == 0 ? "" : ("." + scale))
+                );
             }
 
             assert expectedCols.equals(actualCols) : "expectedCols=" + expectedCols +
@@ -532,7 +545,8 @@ public class JdbcThinMetadataSelfTest extends JdbcThinAbstractSelfTest {
                 "pers.PERSON.PK_pers_PERSON._KEY",
                 "PUBLIC.TEST.PK_PUBLIC_TEST.ID",
                 "PUBLIC.TEST.PK_PUBLIC_TEST.NAME",
-                "PUBLIC.Quoted.PK_PUBLIC_Quoted.Id"));
+                "PUBLIC.Quoted.PK_PUBLIC_Quoted.Id",
+                "PUBLIC.TEST_DECIMAL_COLUMN.ID._KEY"));
 
             Set<String> actualPks = new HashSet<>(expectedPks.size());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
index 0065bae..aff3461 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/QueryEntity.java
@@ -19,6 +19,7 @@ package org.apache.ignite.cache;
 
 import java.io.Serializable;
 import java.lang.reflect.Field;
+import java.math.BigDecimal;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -42,8 +43,11 @@ import org.apache.ignite.internal.util.typedef.F;
 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.lang.IgniteBiTuple;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.Collections.unmodifiableMap;
+
 /**
  * Query entity is a description of {@link org.apache.ignite.IgniteCache cache} entry (composed of key and value)
  * in a way of how it must be indexed and can be queried.
@@ -89,6 +93,9 @@ public class QueryEntity implements Serializable {
     /** Fields default values. */
     private Map<String, Object> defaultFieldValues = new HashMap<>();
 
+    /** Decimal fields information. */
+    private Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = new HashMap<>();
+
     /**
      * Creates an empty query entity.
      */
@@ -120,6 +127,8 @@ public class QueryEntity implements Serializable {
 
         defaultFieldValues = other.defaultFieldValues != null ? new HashMap<>(other.defaultFieldValues)
             : new HashMap<String, Object>();
+
+        decimalInfo = other.decimalInfo != null ? new HashMap<>(other.decimalInfo) : new HashMap<>();
     }
 
     /**
@@ -391,6 +400,27 @@ public class QueryEntity implements Serializable {
     }
 
     /**
+     * Gets set of field name to precision and scale.
+     *
+     * @return Set of names of fields that must have non-null values.
+     */
+    public Map<String, IgniteBiTuple<Integer, Integer>> getDecimalInfo() {
+        return unmodifiableMap(decimalInfo);
+    }
+
+    /**
+     * Sets decimal fields info.
+     *
+     * @param decimalInfo Set of name to precision and scale for decimal fields.
+     * @return {@code this} for chaining.
+     */
+    public QueryEntity setDecimalInfo(Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo) {
+        this.decimalInfo = decimalInfo;
+
+        return this;
+    }
+
+    /**
      * Gets fields default values.
      *
      * @return Field's name to default value map.
@@ -504,6 +534,9 @@ public class QueryEntity implements Serializable {
         if (!F.isEmpty(desc.notNullFields()))
             entity.setNotNullFields(desc.notNullFields());
 
+        if (!F.isEmpty(desc.decimalInfo()))
+            entity.setDecimalInfo(desc.decimalInfo());
+
         return entity;
     }
 
@@ -629,6 +662,9 @@ public class QueryEntity implements Serializable {
             if (sqlAnn.notNull())
                 desc.addNotNullField(prop.fullName());
 
+            if (BigDecimal.class == fldCls && sqlAnn.precision() != -1 && sqlAnn.scale() != -1)
+                desc.addDecimalInfo(prop.fullName(), F.t(sqlAnn.precision(), sqlAnn.scale()));
+
             if ((!F.isEmpty(sqlAnn.groups()) || !F.isEmpty(sqlAnn.orderedGroups()))
                 && sqlAnn.inlineSize() != QueryIndex.DFLT_INLINE_SIZE) {
                 throw new CacheException("Inline size cannot be set on a field with group index [" +
@@ -670,13 +706,14 @@ public class QueryEntity implements Serializable {
             F.eqNotOrdered(idxs, entity.idxs) &&
             F.eq(tableName, entity.tableName) &&
             F.eq(_notNullFields, entity._notNullFields) &&
-            F.eq(defaultFieldValues, entity.defaultFieldValues);
+            F.eq(defaultFieldValues, entity.defaultFieldValues) &&
+            F.eq(decimalInfo, entity.decimalInfo);
     }
 
     /** {@inheritDoc} */
     @Override public int hashCode() {
         return Objects.hash(keyType, valType, keyFieldName, valueFieldName, fields, keyFields, aliases, idxs,
-            tableName, _notNullFields, defaultFieldValues);
+            tableName, _notNullFields, defaultFieldValues, decimalInfo);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
index 0343474..35b7575 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/annotations/QuerySqlField.java
@@ -64,6 +64,20 @@ public @interface QuerySqlField {
     boolean notNull() default false;
 
     /**
+     * Specifies precision for a decimal field.
+     *
+     * @return precision for a decimal field.
+     */
+    int precision() default -1;
+
+    /**
+     * Specifies scale for a decimal field.
+     *
+     * @return scale for a decimal field.
+     */
+    int scale() default -1;
+
+    /**
      * Array of index groups this field belongs to. Groups are used for compound indexes,
      * whenever index should be created on more than one field. All fields within the same
      * group will belong to the same index.
@@ -155,4 +169,4 @@ public @interface QuerySqlField {
          */
         boolean descending() default false;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
index e21cc4e..a50d78e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientUtils.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.binary.BinarySchema;
 import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
+import org.apache.ignite.lang.IgniteBiTuple;
 
 /**
  * Shared serialization/deserialization utils.
@@ -309,6 +310,8 @@ final class ClientUtils {
                                 w.writeBoolean(qf.isKey());
                                 w.writeBoolean(qf.isNotNull());
                                 w.writeObject(qf.getDefaultValue());
+                                w.writeInt(qf.getPrecision());
+                                w.writeInt(qf.getScale());
                             }
                         );
                         ClientUtils.collection(
@@ -392,7 +395,9 @@ final class ClientUtils {
                                 reader.readString(),
                                 reader.readBoolean(),
                                 reader.readBoolean(),
-                                reader.readObject()
+                                reader.readObject(),
+                                reader.readInt(),
+                                reader.readInt()
                             )
                         );
 
@@ -494,6 +499,12 @@ final class ClientUtils {
         /** Default value. */
         private final Object dfltVal;
 
+        /** Precision. */
+        private final int precision;
+
+        /** Scale. */
+        private final int scale;
+
         /** Serialization constructor. */
         QueryField(QueryEntity e, Map.Entry<String, String> nameAndTypeName) {
             name = nameAndTypeName.getKey();
@@ -502,19 +513,28 @@ final class ClientUtils {
             Set<String> keys = e.getKeyFields();
             Set<String> notNulls = e.getNotNullFields();
             Map<String, Object> dflts = e.getDefaultFieldValues();
+            Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = e.getDecimalInfo();
 
             isKey = keys != null && keys.contains(name);
             isNotNull = notNulls != null && notNulls.contains(name);
             dfltVal = dflts == null ? null : dflts.get(name);
+
+            IgniteBiTuple<Integer, Integer> precisionAndScale = decimalInfo == null ? null : decimalInfo.get(name);
+
+            precision = precisionAndScale == null? -1 : precisionAndScale.get1();
+            scale = precisionAndScale == null? -1 : precisionAndScale.get2();
         }
 
         /** Deserialization constructor. */
-        public QueryField(String name, String typeName, boolean isKey, boolean isNotNull, Object dfltVal) {
+        public QueryField(String name, String typeName, boolean isKey, boolean isNotNull, Object dfltVal,
+            int precision, int scale) {
             this.name = name;
             this.typeName = typeName;
             this.isKey = isKey;
             this.isNotNull = isNotNull;
             this.dfltVal = dfltVal;
+            this.precision = precision;
+            this.scale = scale;
         }
 
         /**
@@ -551,6 +571,20 @@ final class ClientUtils {
         Object getDefaultValue() {
             return dfltVal;
         }
+
+        /**
+         * @return Precision.
+         */
+        public int getPrecision() {
+            return precision;
+        }
+
+        /**
+         * @return Scale.
+         */
+        public int getScale() {
+            return scale;
+        }
     }
 
     /** Thin client protocol cache configuration item codes. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
index dd8b733..7adc301 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/JdbcThinDatabaseMetadata.java
@@ -846,9 +846,9 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
         row.add(colMeta.columnName());          // 4. COLUMN_NAME
         row.add(colMeta.dataType());            // 5. DATA_TYPE
         row.add(colMeta.dataTypeName());        // 6. TYPE_NAME
-        row.add((Integer)null);                 // 7. COLUMN_SIZE
+        row.add(colMeta.precision() == -1 ? null : colMeta.precision());                 // 7. COLUMN_SIZE
         row.add((Integer)null);                 // 8. BUFFER_LENGTH
-        row.add((Integer)null);                 // 9. DECIMAL_DIGITS
+        row.add(colMeta.scale() == -1 ? null : colMeta.scale());           // 9. DECIMAL_DIGITS
         row.add(10);                            // 10. NUM_PREC_RADIX
         row.add(colMeta.isNullable() ? columnNullable : columnNoNulls);  // 11. NULLABLE
         row.add((String)null);                  // 12. REMARKS
@@ -1542,4 +1542,4 @@ public class JdbcThinDatabaseMetadata implements DatabaseMetaData {
     @Override public boolean generatedKeyAlwaysReturned() throws SQLException {
         return false;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryEntityTypeDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryEntityTypeDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryEntityTypeDescriptor.java
index fd0ef2b..9f8abc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryEntityTypeDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/QueryEntityTypeDescriptor.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.query.GridQueryIndexDescriptor;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.lang.IgniteBiTuple;
 
 /**
  * Descriptor of type.
@@ -54,6 +55,9 @@ public class QueryEntityTypeDescriptor {
     /** */
     private Set<String> notNullFields = new HashSet<>();
 
+    /** Decimal fields information. */
+    private Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = new HashMap<>();
+
     /** */
     private QueryEntityIndexDescriptor fullTextIdx;
 
@@ -187,6 +191,16 @@ public class QueryEntityTypeDescriptor {
     }
 
     /**
+     * Adds decimal info.
+     *
+     * @param field Field.
+     * @param info Decimal column info.
+     */
+    public void addDecimalInfo(String field, IgniteBiTuple<Integer, Integer> info) {
+        decimalInfo.put(field, info);
+    }
+
+    /**
      * @return notNull fields.
      */
     public Set<String> notNullFields() {
@@ -194,6 +208,13 @@ public class QueryEntityTypeDescriptor {
     }
 
     /**
+     * @return Decimal info for fields.
+     */
+    public Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo() {
+        return decimalInfo;
+    }
+
+    /**
      * @return Class properties.
      */
     public Map<String, QueryEntityClassProperty> properties() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
index c0ac322..5b6304d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMeta.java
@@ -135,6 +135,20 @@ public class JdbcColumnMeta implements JdbcRawBinarylizable {
     }
 
     /**
+     * @return Column's precision.
+     */
+    public int precision() {
+        return -1;
+    }
+
+    /**
+     * @return Column's scale.
+     */
+    public int scale() {
+        return -1;
+    }
+
+    /**
      * Return 'nullable' flag in compatibility mode (according with column name and column type).
      *
      * @return {@code true} in case the column allows null values. Otherwise returns {@code false}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java
new file mode 100644
index 0000000..ec76983
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcColumnMetaV4.java
@@ -0,0 +1,90 @@
+/*
+ * 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.odbc.jdbc;
+
+import org.apache.ignite.internal.binary.BinaryReaderExImpl;
+import org.apache.ignite.internal.binary.BinaryWriterExImpl;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC column metadata V4.
+ */
+public class JdbcColumnMetaV4 extends JdbcColumnMetaV3 {
+    /** Decimal field precision. */
+    private int precision;
+
+    /** Decimal field scale. */
+    private int scale;
+
+    /**
+     * Default constructor is used for serialization.
+     */
+    JdbcColumnMetaV4() {
+        // No-op.
+    }
+
+    /**
+     * @param schemaName Schema.
+     * @param tblName Table.
+     * @param colName Column.
+     * @param cls Type.
+     * @param nullable Allow nulls.
+     * @param dfltVal Default value.
+     * @param precision Decimal column precision.
+     * @param scale Decimal column scale.
+     */
+    public JdbcColumnMetaV4(String schemaName, String tblName, String colName, Class<?> cls, boolean nullable,
+        Object dfltVal, int precision, int scale) {
+        super(schemaName, tblName, colName, cls, nullable, dfltVal);
+
+        this.precision = precision;
+
+        this.scale = scale;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return precision;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return scale;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBinary(BinaryWriterExImpl writer) {
+        super.writeBinary(writer);
+
+        writer.writeInt(precision);
+        writer.writeInt(scale);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readBinary(BinaryReaderExImpl reader) {
+        super.readBinary(reader);
+
+        precision = reader.readInt();
+        scale = reader.readInt();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcColumnMetaV4.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
index 2fe3b9c..ed37e0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcConnectionContext.java
@@ -47,8 +47,8 @@ public class JdbcConnectionContext implements ClientListenerConnectionContext {
     /** Version 2.4.0: adds default values for columns feature. */
     static final ClientListenerProtocolVersion VER_2_4_0 = ClientListenerProtocolVersion.create(2, 4, 0);
 
-    /** Version 2.5.0. */
-    private static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0);
+    /** Version 2.5.0: adds precision and scale for columns feature. */
+    static final ClientListenerProtocolVersion VER_2_5_0 = ClientListenerProtocolVersion.create(2, 5, 0);
 
     /** Current version. */
     private static final ClientListenerProtocolVersion CURRENT_VER = VER_2_5_0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV4.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV4.java
new file mode 100644
index 0000000..9c8e2b7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcMetaColumnsResultV4.java
@@ -0,0 +1,50 @@
+/*
+ * 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.odbc.jdbc;
+
+import java.util.Collection;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * JDBC columns metadata result.
+ */
+public class JdbcMetaColumnsResultV4 extends JdbcMetaColumnsResult {
+    /**
+     * Default constructor is used for deserialization.
+     */
+    JdbcMetaColumnsResultV4() {
+        super(META_COLUMNS_V4);
+    }
+
+    /**
+     * @param meta Columns metadata.
+     */
+    JdbcMetaColumnsResultV4(Collection<JdbcColumnMeta> meta) {
+        super(META_COLUMNS_V4, meta);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected JdbcColumnMeta createMetaColumn() {
+        return new JdbcColumnMetaV4();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(JdbcMetaColumnsResultV4.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
index cf0e98b..b9c9cdd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcRequestHandler.java
@@ -67,6 +67,7 @@ import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchR
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcBulkLoadBatchRequest.CMD_FINISHED_ERROR;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_3_0;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_4_0;
+import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext.VER_2_5_0;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BATCH_EXEC;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.BULK_LOAD_BATCH;
 import static org.apache.ignite.internal.processors.odbc.jdbc.JdbcRequest.META_COLUMNS;
@@ -744,7 +745,14 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
 
                         JdbcColumnMeta columnMeta;
 
-                        if (protocolVer.compareTo(VER_2_4_0) >= 0) {
+                        if (protocolVer.compareTo(VER_2_5_0) >= 0) {
+                            GridQueryProperty prop = table.property(colName);
+
+                            columnMeta = new JdbcColumnMetaV4(table.schemaName(), table.tableName(),
+                                field.getKey(), field.getValue(), !prop.notNull(), prop.defaultValue(),
+                                prop.precision(), prop.scale());
+                        }
+                        else if (protocolVer.compareTo(VER_2_4_0) >= 0) {
                             GridQueryProperty prop = table.property(colName);
 
                             columnMeta = new JdbcColumnMetaV3(table.schemaName(), table.tableName(),
@@ -768,7 +776,9 @@ public class JdbcRequestHandler implements ClientListenerRequestHandler {
 
             JdbcMetaColumnsResult res;
 
-            if (protocolVer.compareTo(VER_2_4_0) >= 0)
+            if (protocolVer.compareTo(VER_2_5_0) >= 0)
+                res = new JdbcMetaColumnsResultV4(meta);
+            else if (protocolVer.compareTo(VER_2_4_0) >= 0)
                 res = new JdbcMetaColumnsResultV3(meta);
             else if (protocolVer.compareTo(VER_2_3_0) >= 0)
                 res = new JdbcMetaColumnsResultV2(meta);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
index 43631e9..4fea207 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/jdbc/JdbcResult.java
@@ -68,6 +68,9 @@ public class JdbcResult implements JdbcRawBinarylizable {
     /** A request to send file from client to server. */
     static final byte BULK_LOAD_ACK = 16;
 
+    /** Columns metadata result V4. */
+    static final byte META_COLUMNS_V4 = 17;
+
     /** Success status. */
     private byte type;
 
@@ -171,6 +174,11 @@ public class JdbcResult implements JdbcRawBinarylizable {
 
                 break;
 
+            case META_COLUMNS_V4:
+                res = new JdbcMetaColumnsResultV4();
+
+                break;
+
             default:
                 throw new IgniteException("Unknown SQL listener request ID: [request ID=" + resId + ']');
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
index 8ebf09e..fa8e509 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java
@@ -76,6 +76,8 @@ import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAff
 import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicyFactory;
 import org.apache.ignite.internal.processors.platform.events.PlatformLocalEventListener;
 import org.apache.ignite.internal.processors.platform.plugin.cache.PlatformCachePluginConfiguration;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.platform.dotnet.PlatformDotNetAffinityFunction;
 import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryConfiguration;
@@ -492,6 +494,7 @@ public class PlatformConfigurationUtils {
         Set<String> keyFields = new HashSet<>(cnt);
         Set<String> notNullFields = new HashSet<>(cnt);
         Map<String, Object> defVals = new HashMap<>(cnt);
+        Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = new HashMap<>(cnt);
 
         if (cnt > 0) {
             LinkedHashMap<String, String> fields = new LinkedHashMap<>(cnt);
@@ -511,6 +514,13 @@ public class PlatformConfigurationUtils {
                 Object defVal = in.readObject();
                 if (defVal != null)
                     defVals.put(fieldName, defVal);
+
+                int precision = in.readInt();
+
+                int scale = in.readInt();
+
+                if (precision != -1 || scale != -1)
+                    decimalInfo.put(fieldName, F.t(precision, scale));
             }
 
             res.setFields(fields);
@@ -523,6 +533,9 @@ public class PlatformConfigurationUtils {
 
             if (!defVals.isEmpty())
                 res.setDefaultFieldValues(defVals);
+
+            if (!decimalInfo.isEmpty())
+                res.setDecimalInfo(decimalInfo);
         }
 
         // Aliases
@@ -1012,6 +1025,7 @@ public class PlatformConfigurationUtils {
             Set<String> keyFields = qryEntity.getKeyFields();
             Set<String> notNullFields = qryEntity.getNotNullFields();
             Map<String, Object> defVals = qryEntity.getDefaultFieldValues();
+            Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = qryEntity.getDecimalInfo();
 
             writer.writeInt(fields.size());
 
@@ -1021,6 +1035,12 @@ public class PlatformConfigurationUtils {
                 writer.writeBoolean(keyFields != null && keyFields.contains(field.getKey()));
                 writer.writeBoolean(notNullFields != null && notNullFields.contains(field.getKey()));
                 writer.writeObject(defVals != null ? defVals.get(field.getKey()) : null);
+
+                IgniteBiTuple<Integer, Integer> precisionAndScale =
+                    decimalInfo == null ? null : decimalInfo.get(field.getKey());
+
+                writer.writeInt(precisionAndScale == null ? -1 : precisionAndScale.get1());
+                writer.writeInt(precisionAndScale == null ? -1 : precisionAndScale.get2());
             }
         }
         else

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index 03e5254..a74548a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -2400,7 +2400,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         for (QueryField col : cols) {
             try {
                 props.add(new QueryBinaryProperty(ctx, col.name(), null, Class.forName(col.typeName()),
-                    false, null, !col.isNullable(), null));
+                    false, null, !col.isNullable(), null, -1, -1));
             }
             catch (ClassNotFoundException e) {
                 throw new SchemaOperationException("Class not found for new property: " + col.typeName());

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
index b258b7c..448c844 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProperty.java
@@ -77,4 +77,18 @@ public interface GridQueryProperty {
      * @return {@code null} if a default value is not set for the property.
      */
     public Object defaultValue();
+
+    /**
+     * Gets precision for this property.
+     *
+     * @return Precision for a decimal property or -1.
+     */
+    public int precision();
+
+    /**
+     * Gets scale for this property.
+     *
+     * @return Scale for a decimal property or -1.
+     */
+    public int scale();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryField.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryField.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryField.java
index 1a75ef1..882d816 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryField.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryField.java
@@ -40,13 +40,19 @@ public class QueryField implements Serializable {
     /** Default value. */
     private final Object dfltValue;
 
+    /** Precision. */
+    private final int precision;
+
+    /** Scale. */
+    private final int scale;
+
     /**
      * @param name Field name.
      * @param typeName Class name for this field's values.
      * @param nullable Nullable flag.
      */
     public QueryField(String name, String typeName, boolean nullable) {
-        this(name, typeName, nullable, null);
+        this(name, typeName, nullable, null, -1, -1);
     }
 
     /**
@@ -55,11 +61,13 @@ public class QueryField implements Serializable {
      * @param nullable Nullable flag.
      * @param dfltValue Default value.
      */
-    public QueryField(String name, String typeName, boolean nullable, Object dfltValue) {
+    public QueryField(String name, String typeName, boolean nullable, Object dfltValue, int precision, int scale) {
         this.name = name;
         this.typeName = typeName;
         this.nullable = nullable;
         this.dfltValue = dfltValue;
+        this.precision = precision;
+        this.scale = scale;
     }
 
     /**
@@ -90,6 +98,20 @@ public class QueryField implements Serializable {
         return dfltValue;
     }
 
+    /**
+     * @return Precision.
+     */
+    public int precision() {
+        return precision;
+    }
+
+    /**
+     * @return Scale.
+     */
+    public int scale() {
+        return scale;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(QueryField.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
index 2ee37a1..12eacef 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryUtils.java
@@ -61,6 +61,7 @@ import org.apache.ignite.internal.util.Jsr310Java8DateTimeApiUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
@@ -264,6 +265,7 @@ public class QueryUtils {
         normalEntity.setValueFieldName(entity.getValueFieldName());
         normalEntity.setNotNullFields(entity.getNotNullFields());
         normalEntity.setDefaultFieldValues(entity.getDefaultFieldValues());
+        normalEntity.setDecimalInfo(entity.getDecimalInfo());
 
         // Normalize table name.
         String normalTblName = entity.getTableName();
@@ -541,6 +543,7 @@ public class QueryUtils {
         Set<String> keyFields = qryEntity.getKeyFields();
         Set<String> notNulls = qryEntity.getNotNullFields();
         Map<String, Object> dlftVals = qryEntity.getDefaultFieldValues();
+        Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo  = qryEntity.getDecimalInfo();
 
         // We have to distinguish between empty and null keyFields when the key is not of SQL type -
         // when a key is not of SQL type, absence of a field in nonnull keyFields tell us that this field
@@ -571,9 +574,14 @@ public class QueryUtils {
 
             Object dfltVal = dlftVals != null ? dlftVals.get(entry.getKey()) : null;
 
+            IgniteBiTuple<Integer, Integer> precisionAndScale =
+                decimalInfo != null ? decimalInfo.get(entry.getKey()) : null;
+
             QueryBinaryProperty prop = buildBinaryProperty(ctx, entry.getKey(),
                 U.classForName(entry.getValue(), Object.class, true),
-                d.aliases(), isKeyField, notNull, dfltVal);
+                d.aliases(), isKeyField, notNull, dfltVal,
+                precisionAndScale != null ? precisionAndScale.get1() : -1,
+                precisionAndScale != null ? precisionAndScale.get2() : -1);
 
             d.addProperty(prop, false);
         }
@@ -717,11 +725,14 @@ public class QueryUtils {
      *      to key, {@code false} if it belongs to value, {@code null} if QueryEntity#keyFields is null.
      * @param notNull {@code true} if {@code null} value is not allowed.
      * @param dlftVal Default value.
+     * @param precision Precision.
+     * @param scale Scale.
      * @return Binary property.
      * @throws IgniteCheckedException On error.
      */
     public static QueryBinaryProperty buildBinaryProperty(GridKernalContext ctx, String pathStr, Class<?> resType,
-        Map<String, String> aliases, @Nullable Boolean isKeyField, boolean notNull, Object dlftVal) throws IgniteCheckedException {
+        Map<String, String> aliases, @Nullable Boolean isKeyField, boolean notNull, Object dlftVal,
+        int precision, int scale) throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
         QueryBinaryProperty res = null;
@@ -737,7 +748,8 @@ public class QueryUtils {
             String alias = aliases.get(fullName.toString());
 
             // The key flag that we've found out is valid for the whole path.
-            res = new QueryBinaryProperty(ctx, prop, res, resType, isKeyField, alias, notNull, dlftVal);
+            res = new QueryBinaryProperty(ctx, prop, res, resType, isKeyField, alias, notNull, dlftVal,
+                precision, scale);
         }
 
         return res;
@@ -1400,5 +1412,15 @@ public class QueryUtils {
         @Override public Object defaultValue() {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public int precision() {
+            return -1;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int scale() {
+            return -1;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
index f440d12..7a47c2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryBinaryProperty.java
@@ -71,6 +71,12 @@ public class QueryBinaryProperty implements GridQueryProperty {
     /** */
     private final Object defaultValue;
 
+    /** */
+    private final int precision;
+
+    /** */
+    private final int scale;
+
     /**
      * Constructor.
      *
@@ -82,9 +88,12 @@ public class QueryBinaryProperty implements GridQueryProperty {
      * @param alias Field alias.
      * @param notNull {@code true} if null value is not allowed.
      * @param defaultValue Default value.
+     * @param precision Precision.
+     * @param scale Scale.
      */
     public QueryBinaryProperty(GridKernalContext ctx, String propName, QueryBinaryProperty parent,
-        Class<?> type, @Nullable Boolean key, String alias, boolean notNull, Object defaultValue) {
+        Class<?> type, @Nullable Boolean key, String alias, boolean notNull, Object defaultValue,
+        int precision, int scale) {
         this.ctx = ctx;
 
         log = ctx.log(QueryBinaryProperty.class);
@@ -99,6 +108,8 @@ public class QueryBinaryProperty implements GridQueryProperty {
             this.isKeyProp = key ? 1 : -1;
 
         this.defaultValue = defaultValue;
+        this.precision = precision;
+        this.scale = scale;
     }
 
     /** {@inheritDoc} */
@@ -286,4 +297,14 @@ public class QueryBinaryProperty implements GridQueryProperty {
     @Override public Object defaultValue() {
         return defaultValue;
     }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return precision;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return scale;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
index 575fe17..487ee5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/property/QueryClassProperty.java
@@ -146,4 +146,14 @@ public class QueryClassProperty implements GridQueryProperty {
     @Override public Object defaultValue() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Override public int precision() {
+        return -1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int scale() {
+        return -1;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index bc5c1e0..f907138 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -71,6 +71,7 @@ import org.apache.ignite.internal.sql.command.SqlIndexColumn;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.plugin.security.SecurityPermission;
 import org.h2.command.Prepared;
 import org.h2.command.ddl.AlterTableAlterColumn;
@@ -80,6 +81,7 @@ import org.h2.command.ddl.DropIndex;
 import org.h2.command.ddl.DropTable;
 import org.h2.table.Column;
 import org.h2.value.DataType;
+import org.h2.value.Value;
 
 import static org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing.UPDATE_RESULT_META;
 import static org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser.PARAM_WRAP_VALUE;
@@ -420,7 +422,8 @@ public class DdlStatementsProcessor {
 
                         QueryField field = new QueryField(col.columnName(),
                             DataType.getTypeClassName(col.column().getType()),
-                            col.column().isNullable(), col.defaultValue());
+                            col.column().isNullable(), col.defaultValue(),
+                            col.precision(), col.scale());
 
                         cols.add(field);
 
@@ -595,6 +598,8 @@ public class DdlStatementsProcessor {
 
         HashMap<String, Object> dfltValues = new HashMap<>();
 
+        Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = new HashMap<>();
+
         for (Map.Entry<String, GridSqlColumn> e : createTbl.columns().entrySet()) {
             GridSqlColumn gridCol = e.getValue();
 
@@ -613,11 +618,17 @@ public class DdlStatementsProcessor {
 
             if (dfltVal != null)
                 dfltValues.put(e.getKey(), dfltVal);
+
+            if (col.getType() == Value.DECIMAL)
+                decimalInfo.put(e.getKey(), F.t((int)col.getPrecision(), col.getScale()));
         }
 
         if (!F.isEmpty(dfltValues))
             res.setDefaultFieldValues(dfltValues);
 
+        if (!F.isEmpty(decimalInfo))
+            res.setDecimalInfo(decimalInfo);
+
         String valTypeName = QueryUtils.createTableValueTypeName(createTbl.schemaName(), createTbl.tableName());
         String keyTypeName = QueryUtils.createTableKeyTypeName(valTypeName);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
index bc14ae2..0efbd4c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlColumn.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.h2.command.Parser;
 import org.h2.expression.Expression;
 import org.h2.table.Column;
+import org.h2.value.Value;
 
 /**
  * Column.
@@ -129,9 +130,23 @@ public class GridSqlColumn extends GridSqlElement {
     }
 
     /**
+     * @return Precision.
+     */
+    public int precision() {
+        return (int) col.getPrecision();
+    }
+
+    /**
+     * @return Scale.
+     */
+    public int scale() {
+        return col.getScale();
+    }
+
+    /**
      * @return H2 Column.
      */
     public Column column() {
         return col;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IgniteDecimalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IgniteDecimalSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IgniteDecimalSelfTest.java
new file mode 100644
index 0000000..9e65276
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/IgniteDecimalSelfTest.java
@@ -0,0 +1,265 @@
+/*
+ * 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.index;
+
+import java.math.BigDecimal;
+import java.math.MathContext;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.QueryEntity;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.lang.IgniteBiTuple;
+import org.jetbrains.annotations.NotNull;
+
+import static java.math.RoundingMode.HALF_UP;
+import static java.util.Arrays.asList;
+
+/**
+ * Test to check decimal columns.
+ */
+public class IgniteDecimalSelfTest extends AbstractSchemaSelfTest {
+    /** */
+    private static final int PRECISION = 9;
+
+    /** */
+    private static final int SCALE = 8;
+
+    /** */
+    private static final String DEC_TAB_NAME = "DECIMAL_TABLE";
+
+    /** */
+    private static final String VALUE = "VALUE";
+
+    /** */
+    private static final String SALARY_TAB_NAME = "SALARY";
+
+    /** */
+    private static final MathContext MATH_CTX = new MathContext(PRECISION);
+
+    /** */
+    private static final BigDecimal VAL_1 = new BigDecimal("123456789", MATH_CTX).setScale(SCALE, HALF_UP);
+
+    /** */
+    private static final BigDecimal VAL_2 = new BigDecimal("12345678.12345678", MATH_CTX).setScale(SCALE, HALF_UP);
+
+    /** */
+    private static final BigDecimal VAL_3 = new BigDecimal(".123456789", MATH_CTX).setScale(SCALE, HALF_UP);
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        IgniteEx grid = startGrid(0);
+
+        execute(grid, "CREATE TABLE " + DEC_TAB_NAME +
+            "(id LONG PRIMARY KEY, " + VALUE + " DECIMAL(" + PRECISION + ", " + SCALE + "))");
+
+        String insertQry = "INSERT INTO " + DEC_TAB_NAME + " VALUES (?, ?)";
+
+        execute(grid, insertQry, 1, VAL_1);
+        execute(grid, insertQry, 2, VAL_2);
+        execute(grid, insertQry, 3, VAL_3);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        CacheConfiguration<Integer, Salary> ccfg = cacheCfg(SALARY_TAB_NAME, "salary_cache");
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** */
+    @NotNull private CacheConfiguration<Integer, Salary> cacheCfg(String tabName, String cacheName) {
+        CacheConfiguration<Integer, Salary> ccfg = new CacheConfiguration<>(cacheName);
+
+        QueryEntity queryEntity = new QueryEntity(Integer.class.getName(), Salary.class.getName());
+
+        queryEntity.setTableName(tabName);
+
+        queryEntity.addQueryField("id", Integer.class.getName(), null);
+        queryEntity.addQueryField("amount", BigDecimal.class.getName(), null);
+
+        Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = new HashMap<>();
+
+        decimalInfo.put("amount", F.t(PRECISION, SCALE));
+
+        queryEntity.setDecimalInfo(decimalInfo);
+
+        ccfg.setQueryEntities(Collections.singletonList(queryEntity));
+
+        return ccfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredFromDdl() throws Exception {
+        checkDecimalInfo(DEC_TAB_NAME, VALUE, PRECISION, SCALE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredFromQueryEntity() throws Exception {
+        checkDecimalInfo(SALARY_TAB_NAME, "amount", PRECISION, SCALE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredFromQueryEntityInDynamicallyCreatedCache() throws Exception {
+        IgniteEx grid = grid(0);
+
+        String tabName = SALARY_TAB_NAME + "2";
+
+        CacheConfiguration<Integer, Salary> ccfg = cacheCfg(tabName, "SalaryCache-2");
+
+        IgniteCache<Integer, Salary> cache = grid.createCache(ccfg);
+
+        checkDecimalInfo(tabName, "amount", PRECISION, SCALE);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConfiguredFromAnnotations() throws Exception {
+        IgniteEx grid = grid(0);
+
+        CacheConfiguration<Integer, Salary> ccfg = new CacheConfiguration<>("SalaryCache-3");
+
+        ccfg.setIndexedTypes(Integer.class, SalaryWithAnnotations.class);
+
+        grid.createCache(ccfg);
+
+        checkDecimalInfo(SalaryWithAnnotations.class.getSimpleName().toUpperCase(), "amount", PRECISION, SCALE);
+    }
+
+    /** */
+    public void testSelectDecimal() throws Exception {
+        IgniteEx grid = grid(0);
+
+        List rows = execute(grid, "SELECT id, value FROM " + DEC_TAB_NAME + " order by id");
+
+        assertEquals(rows.size(), 3);
+
+        assertEquals(asList(1L, VAL_1), rows.get(0));
+        assertEquals(asList(2L, VAL_2), rows.get(1));
+        assertEquals(asList(3L, VAL_3), rows.get(2));
+    }
+
+    /** */
+    private void checkDecimalInfo(String tabName, String colName, Integer precision, Integer scale) {
+        QueryEntity queryEntity = findTableInfo(tabName);
+
+        assertNotNull(queryEntity);
+
+        Map<String, IgniteBiTuple<Integer, Integer>> decimalInfo = queryEntity.getDecimalInfo();
+
+        assertNotNull(decimalInfo);
+
+        IgniteBiTuple<Integer, Integer> columnInfo = decimalInfo.get(colName);
+
+        assertNotNull(columnInfo);
+
+        assertEquals(columnInfo.get1(), precision);
+        assertEquals(columnInfo.get2(), scale);
+    }
+
+    /**
+     * @param tabName Table name.
+     * @return QueryEntity of table.
+     */
+    private QueryEntity findTableInfo(String tabName) {
+        IgniteEx ignite = grid(0);
+
+        Collection<String> cacheNames = ignite.cacheNames();
+
+        for (String cacheName : cacheNames) {
+            CacheConfiguration ccfg = ignite.cache(cacheName).getConfiguration(CacheConfiguration.class);
+
+            Collection<QueryEntity> entities = ccfg.getQueryEntities();
+
+            for (QueryEntity entity : entities)
+                if (entity.getTableName().equalsIgnoreCase(tabName))
+                    return entity;
+        }
+
+        return null;
+    }
+
+    /**
+     * Execute DDL statement on given node.
+     *
+     * @param node Node.
+     * @param sql Statement.
+     */
+    private List<List<?>> execute(Ignite node, String sql, Object... args) {
+        SqlFieldsQuery qry = new SqlFieldsQuery(sql)
+            .setArgs(args)
+            .setSchema("PUBLIC");
+
+        return queryProcessor(node).querySqlFields(qry, true).getAll();
+    }
+
+    /** */
+    private static class Salary {
+        /** */
+        private BigDecimal amount;
+
+        /** */
+        public BigDecimal getAmount() {
+            return amount;
+        }
+
+        /** */
+        public void setAmount(BigDecimal amount) {
+            this.amount = amount;
+        }
+    }
+
+    /** */
+    private static class SalaryWithAnnotations {
+        /** */
+        @QuerySqlField(index = true, precision = PRECISION, scale = SCALE)
+        private BigDecimal amount;
+
+        /** */
+        public BigDecimal getAmount() {
+            return amount;
+        }
+
+        /** */
+        public void setAmount(BigDecimal amount) {
+            this.amount = amount;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 1da695b..235b28b 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -565,6 +565,16 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
                 @Override public Object defaultValue() {
                     return null;
                 }
+
+                /** */
+                @Override public int precision() {
+                    return -1;
+                }
+
+                /** */
+                @Override public int scale() {
+                    return -1;
+                }
             };
         }
 
@@ -783,4 +793,4 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
             return false;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index e10fff1..619e7cf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -108,6 +108,7 @@ import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoord
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest;
+import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest;
 import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsClientBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerBasicSelfTest;
 import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerCoordinatorBasicSelfTest;
@@ -397,6 +398,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(OptimizedMarshallerIndexNameTest.class);
 
         suite.addTestSuite(IgniteSqlDefaultValueTest.class);
+        suite.addTestSuite(IgniteDecimalSelfTest.class);
 
         // H2 Rows on-heap cache
         suite.addTestSuite(H2RowCacheSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/QueryEntityConfigurationParityTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/QueryEntityConfigurationParityTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/QueryEntityConfigurationParityTest.cs
index a9830d8..ba10cda 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/QueryEntityConfigurationParityTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/ApiParity/QueryEntityConfigurationParityTest.cs
@@ -32,7 +32,8 @@ namespace Apache.Ignite.Core.Tests.ApiParity
             "findValueType",
             "KeyFields",
             "NotNullFields",
-            "DefaultFieldValues"
+            "DefaultFieldValues",
+            "DecimalInfo"
         };
 
         /// <summary>
@@ -47,4 +48,4 @@ namespace Apache.Ignite.Core.Tests.ApiParity
                 UnneededProperties);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
index 6834d5d..9d59a50 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -551,6 +551,8 @@ namespace Apache.Ignite.Core.Tests.Cache
             Assert.AreEqual(x.IsKeyField, y.IsKeyField);
             Assert.AreEqual(x.NotNull, y.NotNull);
             Assert.AreEqual(x.DefaultValue, y.DefaultValue);
+            Assert.AreEqual(x.Precision, y.Precision);
+            Assert.AreEqual(x.Scale, y.Scale);
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
index 8855535..32173ba 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryEntity.cs
@@ -458,7 +458,9 @@ namespace Apache.Ignite.Core.Cache.Configuration
                     {
                         IsKeyField = isKey,
                         NotNull = attr.NotNull,
-                        DefaultValue = attr.DefaultValue
+                        DefaultValue = attr.DefaultValue,
+                        Precision = attr.Precision,
+                        Scale = attr.Scale
                     });
 
                     ScanAttributes(memberInfo.Value, fields, indexes, columnName, visitedTypes, isKey);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
index a96f1bf..869ce7d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QueryField.cs
@@ -85,6 +85,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
             IsKeyField = reader.ReadBoolean();
             NotNull = reader.ReadBoolean();
             DefaultValue = reader.ReadObject<object>();
+            Precision = reader.ReadInt();
+            Scale = reader.ReadInt();
         }
 
         /// <summary>
@@ -99,6 +101,8 @@ namespace Apache.Ignite.Core.Cache.Configuration
             writer.WriteBoolean(IsKeyField);
             writer.WriteBoolean(NotNull);
             writer.WriteObject(DefaultValue);
+            writer.WriteInt(Precision);
+            writer.WriteInt(Scale);
         }
 
         /// <summary>
@@ -154,6 +158,16 @@ namespace Apache.Ignite.Core.Cache.Configuration
         public object DefaultValue { get; set; }
 
         /// <summary>
+        /// Gets or sets the precision for the field.
+        /// </summary>
+        public int Precision { get; set; }
+
+        /// <summary>
+        /// Gets or sets the scale for the field.
+        /// </summary>
+        public int Scale { get; set; }
+
+        /// <summary>
         /// Validates this instance and outputs information to the log, if necessary.
         /// </summary>
         internal void Validate(ILogger log, string logInfo)
@@ -179,4 +193,4 @@ namespace Apache.Ignite.Core.Cache.Configuration
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
index dd131f9..bfd3575 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Cache/Configuration/QuerySqlFieldAttribute.cs
@@ -81,5 +81,15 @@ namespace Apache.Ignite.Core.Cache.Configuration
         /// Gets or sets the default value for the field (has effect when inserting with DML).
         /// </summary>
         public object DefaultValue { get; set; }
+
+        /// <summary>
+        /// Gets or sets the precision for the field.
+        /// </summary>
+        public int Precision { get; set; }
+
+        /// <summary>
+        /// Gets or sets the scale for the field.
+        /// </summary>
+        public int Scale { get; set; }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6df5f99b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
index a97136f..8707272 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd
@@ -340,6 +340,16 @@
                                                                                             <xs:documentation>Indicates whether null value is allowed for the field.</xs:documentation>
                                                                                         </xs:annotation>
                                                                                     </xs:attribute>
+                                                                                    <xs:attribute name="precision" type="xs:int">
+                                                                                        <xs:annotation>
+                                                                                            <xs:documentation>Precision of field</xs:documentation>
+                                                                                        </xs:annotation>
+                                                                                    </xs:attribute>
+                                                                                    <xs:attribute name="scale" type="xs:int">
+                                                                                        <xs:annotation>
+                                                                                            <xs:documentation>Scale of field</xs:documentation>
+                                                                                        </xs:annotation>
+                                                                                    </xs:attribute>
                                                                                 </xs:complexType>
                                                                             </xs:element>
                                                                         </xs:sequence>


[43/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
IGNITE-8232: ML package cleanup for 2.5 release

this closes #3823


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

Branch: refs/heads/ignite-7708
Commit: c6ab036dcdb5a190b8577c938dbc420ec030c7ad
Parents: 47cfdc2
Author: YuriBabak <y....@gmail.com>
Authored: Fri Apr 13 20:27:15 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Fri Apr 13 20:27:15 2018 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6ab036d/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
index 45fd035..560be4b 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
@@ -27,7 +27,7 @@ import java.util.function.BiFunction;
  * @see java.util.function.BiFunction
  */
 public interface IgniteBiFunction<T, U, R> extends BiFunction<T, U, R>, Serializable {
-    /** {@inheritDoc} */
+    /** */
     default <V> IgniteBiFunction<T, U, V> andThen(IgniteFunction<? super R, ? extends V> after) {
         Objects.requireNonNull(after);
         return (T t, U u) -> after.apply(apply(t, u));


[19/50] [abbrv] ignite git commit: IGNITE-8042: .NET thin client: authentication support. This closes #3790.

Posted by ag...@apache.org.
IGNITE-8042: .NET thin client: authentication support. This closes #3790.


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

Branch: refs/heads/ignite-7708
Commit: 67023a88b8c9cf647196b6deb1759b4d01e33890
Parents: e333f30
Author: devozerov <vo...@gridgain.com>
Authored: Thu Apr 12 10:37:36 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Apr 12 10:37:36 2018 +0300

----------------------------------------------------------------------
 .../client/ClientAuthenticationException.java   |  16 +-
 .../internal/client/thin/TcpClientChannel.java  |   9 +-
 .../odbc/ClientListenerNioListener.java         |  19 +++
 .../client/ClientConnectionContext.java         |   8 +-
 .../platform/client/ClientStatus.java           |   5 +-
 .../Client/ClientConnectionTest.cs              | 161 +++++++++++++++++++
 .../Client/ClientStatusCode.cs                  |  12 +-
 .../Client/IgniteClientConfiguration.cs         |  13 ++
 .../IgniteClientConfigurationSection.xsd        |  10 ++
 .../Impl/Client/ClientProtocolVersion.cs        |  22 ++-
 .../Impl/Client/ClientSocket.cs                 |  85 +++++++++-
 11 files changed, 329 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
index 0c24db8..526690a 100644
--- a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
@@ -24,22 +24,10 @@ public class ClientAuthenticationException extends ClientException {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
-    /** Message. */
-    private static final String MSG = "Invalid user name or password";
-
     /**
      * Default constructor.
      */
-    public ClientAuthenticationException() {
-        super(MSG);
-    }
-
-    /**
-     * Constructs a new exception with the specified cause.
-     *
-     * @param cause the cause.
-     */
-    public ClientAuthenticationException(Throwable cause) {
-        super(MSG, cause);
+    public ClientAuthenticationException(String msg) {
+        super(msg);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
index 8e8294f..10dc865 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
@@ -272,8 +272,13 @@ class TcpClientChannel implements ClientChannel {
             try (BinaryReaderExImpl r = new BinaryReaderExImpl(null, res, null, true)) {
                 String err = r.readString();
 
-                if (err != null && err.toUpperCase().matches(".*USER.*INCORRECT.*"))
-                    throw new ClientAuthenticationException();
+                int errCode = ClientStatus.FAILED;
+
+                if (res.remaining() > 0)
+                    errCode = r.readInt();
+
+                if (errCode == ClientStatus.AUTH_FAILED)
+                    throw new ClientAuthenticationException(err);
                 else if (ver.equals(srvVer))
                     throw new ClientProtocolError(err);
                 else if (!supportedVers.contains(srvVer) ||

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
index 53b14d7..407c1a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerNioListener.java
@@ -26,9 +26,11 @@ import org.apache.ignite.internal.binary.BinaryWriterExImpl;
 import org.apache.ignite.internal.binary.streams.BinaryHeapInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
+import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
 import org.apache.ignite.internal.processors.odbc.jdbc.JdbcConnectionContext;
 import org.apache.ignite.internal.processors.odbc.odbc.OdbcConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.nio.GridNioServerListenerAdapter;
 import org.apache.ignite.internal.util.nio.GridNioSession;
@@ -228,6 +230,18 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
 
             connCtx.handler().writeHandshake(writer);
         }
+        catch (IgniteAccessControlException authEx) {
+            writer.writeBoolean(false);
+
+            writer.writeShort((short)0);
+            writer.writeShort((short)0);
+            writer.writeShort((short)0);
+
+            writer.doWriteString(authEx.getMessage());
+
+            if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0)
+                writer.writeInt(ClientStatus.AUTH_FAILED);
+        }
         catch (IgniteCheckedException e) {
             U.warn(log, "Error during handshake [rmtAddr=" + ses.remoteAddress() + ", msg=" + e.getMessage() + ']');
 
@@ -239,10 +253,15 @@ public class ClientListenerNioListener extends GridNioServerListenerAdapter<byte
                 currVer = connCtx.currentVersion();
 
             writer.writeBoolean(false);
+
             writer.writeShort(currVer.major());
             writer.writeShort(currVer.minor());
             writer.writeShort(currVer.maintenance());
+
             writer.doWriteString(e.getMessage());
+
+            if (ver.compareTo(ClientConnectionContext.VER_1_1_0) >= 0)
+                writer.writeInt(ClientStatus.FAILED);
         }
 
         ses.send(writer.array());

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 061aab3..056ea83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -44,10 +44,10 @@ import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIEN
  */
 public class ClientConnectionContext implements ClientListenerConnectionContext {
     /** Version 1.0.0. */
-    private static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0);
+    public static final ClientListenerProtocolVersion VER_1_0_0 = ClientListenerProtocolVersion.create(1, 0, 0);
 
     /** Version 1.1.0. */
-    private static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0);
+    public static final ClientListenerProtocolVersion VER_1_1_0 = ClientListenerProtocolVersion.create(1, 1, 0);
 
     /** Supported versions. */
     private static final Collection<ClientListenerProtocolVersion> SUPPORTED_VERS = Arrays.asList(VER_1_1_0, VER_1_0_0);
@@ -144,12 +144,12 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
             authCtx = thirdPartyAuthentication(user, pwd).authorizationContext();
         else if (kernalCtx.authentication().enabled()) {
             if (user == null || user.length() == 0)
-                throw new IgniteCheckedException("Unauthenticated sessions are prohibited.");
+                throw new IgniteAccessControlException("Unauthenticated sessions are prohibited.");
 
             authCtx = kernalCtx.authentication().authenticate(user, pwd);
 
             if (authCtx == null)
-                throw new IgniteCheckedException("Unknown authentication error.");
+                throw new IgniteAccessControlException("Unknown authentication error.");
         }
 
         handler = new ClientRequestHandler(this, authCtx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
index b8dfb1f..e63812c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -49,6 +49,9 @@ public final class ClientStatus {
     /** Resource does not exist. */
     public static final int RESOURCE_DOES_NOT_EXIST = 1011;
 
-    /** Resource does not exist. */
+    /** Authorization failure. */
     public static final int SECURITY_VIOLATION = 1012;
+
+    /** Authentication failed. */
+    public static final int AUTH_FAILED = 2000;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 9da9a03..2ea17a8 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -19,13 +19,17 @@ namespace Apache.Ignite.Core.Tests.Client
 {
     using System;
     using System.Collections.Generic;
+    using System.IO;
     using System.Linq;
     using System.Net;
     using System.Net.Sockets;
     using System.Text.RegularExpressions;
     using System.Threading;
     using System.Threading.Tasks;
+    using Apache.Ignite.Core.Cache.Configuration;
+    using Apache.Ignite.Core.Cache.Query;
     using Apache.Ignite.Core.Client;
+    using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Configuration;
     using NUnit.Framework;
 
@@ -34,6 +38,18 @@ namespace Apache.Ignite.Core.Tests.Client
     /// </summary>
     public class ClientConnectionTest
     {
+        /** Temp dir for WAL. */
+        private readonly string _tempDir = TestUtils.GetTempDirectoryName();
+
+        /// <summary>
+        /// Sets up the test.
+        /// </summary>
+        [SetUp]
+        public void SetUp()
+        {
+            TestUtils.ClearWorkDir();
+        }
+
         /// <summary>
         /// Fixture tear down.
         /// </summary>
@@ -41,6 +57,13 @@ namespace Apache.Ignite.Core.Tests.Client
         public void TearDown()
         {
             Ignition.StopAll(true);
+
+            if (Directory.Exists(_tempDir))
+            {
+                Directory.Delete(_tempDir, true);
+            }
+
+            TestUtils.ClearWorkDir();
         }
 
         /// <summary>
@@ -55,6 +78,107 @@ namespace Apache.Ignite.Core.Tests.Client
         }
 
         /// <summary>
+        /// Tests that empty username or password are not allowed.
+        /// </summary>
+        [Test]
+        public void TestAuthenticationEmptyCredentials()
+        {
+            using (Ignition.Start(SecureServerConfig()))
+            {
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Password = null;
+                var ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be null"));
+
+                cliCfg.Password = "";
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Password cannot be empty"));
+
+                cliCfg.Password = "ignite";
+
+                cliCfg.Username = null;
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null"));
+
+                cliCfg.Username = "";
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be empty"));
+            }
+        }
+
+        /// <summary>
+        /// Test invalid username or password.
+        /// </summary>
+        [Test]
+        public void TestAuthenticationInvalidCredentials()
+        {
+            using (Ignition.Start(SecureServerConfig()))
+            {
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Username = "invalid";
+
+                var ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed);
+
+                cliCfg.Username = "ignite";
+                cliCfg.Password = "invalid";
+
+                ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
+                Assert.True(ex.StatusCode == ClientStatusCode.AuthenticationFailed);
+            }
+        }
+
+        /// <summary>
+        /// Test authentication.
+        /// </summary>
+        [Test]
+        public void TestAuthentication()
+        {
+            using (var srv = Ignition.Start(SecureServerConfig()))
+            {
+                srv.GetCluster().SetActive(true);
+
+                using (var cli = Ignition.StartClient(SecureClientConfig()))
+                {
+                    CacheClientConfiguration ccfg = new CacheClientConfiguration()
+                    {
+                        Name = "TestCache",
+                        QueryEntities = new[]
+                        {
+                            new QueryEntity
+                            {
+                                KeyType = typeof(string),
+                                ValueType = typeof(string),
+                            },
+                        },
+                    };
+
+                    ICacheClient<string, string> cache = cli.GetOrCreateCache<string, string>(ccfg);
+
+                    cache.Put("key1", "val1");
+
+                    cache.Query(new SqlFieldsQuery("CREATE USER \"my_User\" WITH PASSWORD 'my_Password'")).GetAll();
+                }
+
+                var cliCfg = SecureClientConfig();
+
+                cliCfg.Username = "my_User";
+                cliCfg.Password = "my_Password";
+
+                using (var cli = Ignition.StartClient(cliCfg))
+                {
+                    ICacheClient<string, string> cache = cli.GetCache<string, string>("TestCache");
+
+                    string val = cache.Get("key1");
+
+                    Assert.True(val == "val1");
+                }
+            }
+        }
+
+        /// <summary>
         /// Tests that multiple clients can connect to one server.
         /// </summary>
         [Test]
@@ -374,5 +498,42 @@ namespace Apache.Ignite.Core.Tests.Client
             
             throw new Exception("SocketException not found.", origEx);
         }
+
+        /// <summary>
+        /// Create server configuration with enabled authentication.
+        /// </summary>
+        /// <returns>Server configuration.</returns>
+        private IgniteConfiguration SecureServerConfig()
+        {
+            return new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                AuthenticationEnabled = true,
+                DataStorageConfiguration = new DataStorageConfiguration()
+                {
+                    StoragePath = Path.Combine(_tempDir, "Store"),
+                    WalPath = Path.Combine(_tempDir, "WalStore"),
+                    WalArchivePath = Path.Combine(_tempDir, "WalArchive"),
+                    DefaultDataRegionConfiguration = new DataRegionConfiguration()
+                    {
+                        Name = "default",
+                        PersistenceEnabled = true
+                    }
+                }
+            };
+        }
+
+        /// <summary>
+        /// Create client configuration with enabled authentication.
+        /// </summary>
+        /// <returns>Client configuration.</returns>
+        private static IgniteClientConfiguration SecureClientConfig()
+        {
+            return new IgniteClientConfiguration()
+            {
+                Host = "localhost",
+                Username = "ignite",
+                Password = "ignite"
+            };
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
index 3f5ee8e..3bdd9e1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/ClientStatusCode.cs
@@ -52,6 +52,16 @@ namespace Apache.Ignite.Core.Client
         /// <summary>
         /// The too many cursors (see <see cref="ClientConnectorConfiguration.MaxOpenCursorsPerConnection"/>).
         /// </summary>
-        TooManyCursors = 1010
+        TooManyCursors = 1010,
+
+        /// <summary>
+        /// Authorization failure.
+        /// </summary>
+        SecurityViolation = 1012,
+
+        /// <summary>
+        /// Authentication failed.
+        /// </summary>
+        AuthenticationFailed = 2000
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
index 8730f39..3252495 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/IgniteClientConfiguration.cs
@@ -90,6 +90,9 @@ namespace Apache.Ignite.Core.Client
 
             BinaryProcessor = cfg.BinaryProcessor;
             SslStreamFactory = cfg.SslStreamFactory;
+
+            Username = cfg.Username;
+            Password = cfg.Password;
         }
 
         /// <summary>
@@ -146,6 +149,16 @@ namespace Apache.Ignite.Core.Client
         public ISslStreamFactory SslStreamFactory { get; set; }
 
         /// <summary>
+        /// Username to be used to connect to secured cluster.
+        /// </summary>
+        public string Username { get; set; }
+
+        /// <summary>
+        /// Password to be used to connect to secured cluster.
+        /// </summary>
+        public string Password { get; set; }
+
+        /// <summary>
         /// Gets or sets custom binary processor. Internal property for tests.
         /// </summary>
         internal IBinaryProcessor BinaryProcessor { get; set; }

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
index 569ee6f..7e6caff 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteClientConfigurationSection.xsd
@@ -237,6 +237,16 @@
                     <xs:documentation>Socket operation timeout. Zero or negative for infinite timeout.</xs:documentation>
                 </xs:annotation>
             </xs:attribute>
+            <xs:attribute name="username" type="xs:string">
+                <xs:annotation>
+                    <xs:documentation>Username to be used to connect to secured cluster.</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
+            <xs:attribute name="password" type="xs:string">
+                <xs:annotation>
+                    <xs:documentation>Password to be used to connect to secured cluster.</xs:documentation>
+                </xs:annotation>
+            </xs:attribute>
         </xs:complexType>
     </xs:element>
 </xs:schema>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
index bfdf5a3..4fe5c71 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientProtocolVersion.cs
@@ -22,7 +22,7 @@ namespace Apache.Ignite.Core.Impl.Client
     /// <summary>
     /// Client protocol version.
     /// </summary>
-    internal struct ClientProtocolVersion : IEquatable<ClientProtocolVersion>
+    internal struct ClientProtocolVersion : IEquatable<ClientProtocolVersion>, IComparable<ClientProtocolVersion>
     {
         /** */
         private readonly short _major;
@@ -68,6 +68,26 @@ namespace Apache.Ignite.Core.Impl.Client
         }
 
         /// <summary>
+        /// Compare this version to other version.
+        /// </summary>
+        /// <param name="other"></param>
+        /// <returns></returns>
+        public int CompareTo(ClientProtocolVersion other)
+        {
+            int res = Major - other.Major;
+
+            if (res == 0)
+            {
+                res = Minor - other.Minor;
+
+                if (res == 0)
+                    res = Maintenance - other.Maintenance;
+            }
+
+            return res;
+        }
+
+        /// <summary>
         /// Returns a value indicating whether specified instance equals to current.
         /// </summary>
         public bool Equals(ClientProtocolVersion other)

http://git-wip-us.apache.org/repos/asf/ignite/blob/67023a88/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
index fca5dab..27d8f0b 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientSocket.cs
@@ -28,6 +28,7 @@ namespace Apache.Ignite.Core.Impl.Client
     using System.Net.Sockets;
     using System.Threading;
     using System.Threading.Tasks;
+    using System.Xml.Schema;
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary;
@@ -38,8 +39,14 @@ namespace Apache.Ignite.Core.Impl.Client
     /// </summary>
     internal sealed class ClientSocket : IDisposable
     {
+        /** Version 1.0.0. */
+        private static readonly ClientProtocolVersion Ver100 = new ClientProtocolVersion(1, 0, 0);
+
+        /** Version 1.1.0. */
+        private static readonly ClientProtocolVersion Ver110 = new ClientProtocolVersion(1, 1, 0);
+
         /** Current version. */
-        private static readonly ClientProtocolVersion CurrentProtocolVersion = new ClientProtocolVersion(1, 0, 0);
+        private static readonly ClientProtocolVersion CurrentProtocolVersion = Ver110;
 
         /** Handshake opcode. */
         private const byte OpHandshake = 1;
@@ -98,7 +105,9 @@ namespace Apache.Ignite.Core.Impl.Client
             _socket = Connect(clientConfiguration);
             _stream = GetSocketStream(_socket, clientConfiguration);
 
-            Handshake(version ?? CurrentProtocolVersion);
+            Validate(clientConfiguration);
+
+            Handshake(clientConfiguration, version ?? CurrentProtocolVersion);
 
             // Check periodically if any request has timed out.
             if (_timeout > TimeSpan.Zero)
@@ -112,6 +121,31 @@ namespace Apache.Ignite.Core.Impl.Client
         }
 
         /// <summary>
+        /// Validate configuration.
+        /// </summary>
+        /// <param name="cfg">Configuration.</param>
+        private void Validate(IgniteClientConfiguration cfg)
+        {
+            if (cfg.Username != null)
+            {
+                if (cfg.Username.Length == 0)
+                    throw new IgniteClientException("IgniteClientConfiguration.Username cannot be empty.");
+
+                if (cfg.Password == null)
+                    throw new IgniteClientException("IgniteClientConfiguration.Password cannot be null when Username is set.");
+            }
+
+            if (cfg.Password != null)
+            {
+                if (cfg.Password.Length == 0)
+                    throw new IgniteClientException("IgniteClientConfiguration.Password cannot be empty.");
+
+                if (cfg.Username == null)
+                    throw new IgniteClientException("IgniteClientConfiguration.Username cannot be null when Password is set.");
+            }
+        }
+
+        /// <summary>
         /// Performs a send-receive operation.
         /// </summary>
         public T DoOutInOp<T>(ClientOp opId, Action<IBinaryStream> writeAction,
@@ -226,8 +260,10 @@ namespace Apache.Ignite.Core.Impl.Client
         /// <summary>
         /// Performs client protocol handshake.
         /// </summary>
-        private void Handshake(ClientProtocolVersion version)
+        private void Handshake(IgniteClientConfiguration clientConfiguration, ClientProtocolVersion version)
         {
+            bool auth = version.CompareTo(Ver110) >= 0 && clientConfiguration.Username != null;
+
             // Send request.
             int messageLen;
             var buf = WriteMessage(stream =>
@@ -242,10 +278,19 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 // Client type: platform.
                 stream.WriteByte(ClientType);
-            }, 12, out messageLen);
 
-            Debug.Assert(messageLen == 12);
+                // Authentication data.
+                if (auth)
+                {
+                    var writer = BinaryUtils.Marshaller.StartMarshal(stream);
+
+                    writer.WriteString(clientConfiguration.Username);
+                    writer.WriteString(clientConfiguration.Password);
 
+                    BinaryUtils.Marshaller.FinishMarshal(writer);
+                }
+            }, 12, out messageLen);
+            
             _stream.Write(buf, 0, messageLen);
 
             // Decode response.
@@ -253,6 +298,7 @@ namespace Apache.Ignite.Core.Impl.Client
 
             using (var stream = new BinaryHeapStream(res))
             {
+                // Read input.
                 var success = stream.ReadBool();
 
                 if (success)
@@ -265,9 +311,32 @@ namespace Apache.Ignite.Core.Impl.Client
 
                 var errMsg = BinaryUtils.Marshaller.Unmarshal<string>(stream);
 
-                throw new IgniteClientException(string.Format(
-                    "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}",
-                    errMsg, version, serverVersion));
+                ClientStatusCode errCode = ClientStatusCode.Fail;
+
+                if (stream.Remaining > 0)
+                {
+                    errCode = (ClientStatusCode) stream.ReadInt();
+                }
+
+                // Authentication error is handled immediately.
+                if (errCode == ClientStatusCode.AuthenticationFailed)
+                {
+                    throw new IgniteClientException(errMsg, null, ClientStatusCode.AuthenticationFailed);
+                }
+
+                // Re-try if possible.
+                bool retry = serverVersion.CompareTo(version) < 0 && serverVersion.Equals(Ver100);
+
+                if (retry)
+                {
+                    Handshake(clientConfiguration, serverVersion);
+                }
+                else
+                {
+                    throw new IgniteClientException(string.Format(
+                        "Client handshake failed: '{0}'. Client version: {1}. Server version: {2}",
+                        errMsg, version, serverVersion), null, errCode);
+                }
             }
         }
 


[25/50] [abbrv] ignite git commit: IGNITE-8230: SQL: Fixed backup number propagation in CREATE TABLE command. This closes #3803.

Posted by ag...@apache.org.
IGNITE-8230: SQL: Fixed backup number propagation in CREATE TABLE command. This closes #3803.


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

Branch: refs/heads/ignite-7708
Commit: b93595cbe67bb3313d472db58b6d113e10a2f97e
Parents: a57c9e1
Author: devozerov <vo...@gridgain.com>
Authored: Thu Apr 12 15:13:51 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Apr 12 15:13:51 2018 +0300

----------------------------------------------------------------------
 .../processors/query/GridQueryProcessor.java    |  7 ++--
 .../query/h2/sql/GridSqlCreateTable.java        |  7 ++--
 .../cache/index/H2DynamicTableSelfTest.java     | 37 ++++++++++++++++++++
 3 files changed, 45 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b93595cb/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index bde9427..03e5254 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -1483,10 +1483,10 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     @SuppressWarnings("unchecked")
     public void dynamicTableCreate(String schemaName, QueryEntity entity, String templateName, String cacheName,
         String cacheGroup, @Nullable String dataRegion, String affinityKey, @Nullable CacheAtomicityMode atomicityMode,
-        @Nullable CacheWriteSynchronizationMode writeSyncMode, int backups, boolean ifNotExists)
+        @Nullable CacheWriteSynchronizationMode writeSyncMode, @Nullable Integer backups, boolean ifNotExists)
         throws IgniteCheckedException {
         assert !F.isEmpty(templateName);
-        assert backups >= 0;
+        assert backups == null || backups >= 0;
 
         CacheConfiguration<?, ?> ccfg = ctx.cache().getConfigFromTemplate(templateName);
 
@@ -1525,7 +1525,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
         if (writeSyncMode != null)
             ccfg.setWriteSynchronizationMode(writeSyncMode);
 
-        ccfg.setBackups(backups);
+        if (backups != null)
+            ccfg.setBackups(backups);
 
         ccfg.setSqlSchema(schemaName);
         ccfg.setSqlEscapeAll(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/b93595cb/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
index 3608aed..de86d6a 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlCreateTable.java
@@ -22,6 +22,7 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * CREATE TABLE statement.
@@ -57,7 +58,7 @@ public class GridSqlCreateTable extends GridSqlStatement {
     private CacheWriteSynchronizationMode writeSyncMode;
 
     /** Backups number for new cache. */
-    private int backups;
+    private Integer backups;
 
     /** Quietly ignore this command if table already exists. */
     private boolean ifNotExists;
@@ -184,14 +185,14 @@ public class GridSqlCreateTable extends GridSqlStatement {
     /**
      * @return Backups number for new cache.
      */
-    public int backups() {
+    @Nullable public Integer backups() {
         return backups;
     }
 
     /**
      * @param backups Backups number for new cache.
      */
-    public void backups(int backups) {
+    public void backups(Integer backups) {
         this.backups = backups;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b93595cb/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
index b20bb59..8224711 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicTableSelfTest.java
@@ -88,7 +88,14 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     /** Bad data region name. */
     public static final String DATA_REGION_NAME_BAD = "my_data_region_bad";
 
+    /** Cache with backups. */
+    private static final String CACHE_NAME_BACKUPS = CACHE_NAME + "_backups";
+
+    /** Number of backups for backup test. */
+    private static final int DFLT_BACKUPS = 2;
+
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
@@ -98,6 +105,8 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         client().addCacheConfiguration(cacheConfiguration());
         client().addCacheConfiguration(cacheConfiguration().setName(CACHE_NAME + "_async")
             .setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_ASYNC));
+
+        client().addCacheConfiguration(cacheConfiguration().setName(CACHE_NAME_BACKUPS).setBackups(DFLT_BACKUPS));
     }
 
     /** {@inheritDoc} */
@@ -108,6 +117,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
@@ -120,6 +130,7 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
         execute("DROP TABLE IF EXISTS PUBLIC.\"Person\"");
         execute("DROP TABLE IF EXISTS PUBLIC.\"City\"");
         execute("DROP TABLE IF EXISTS PUBLIC.\"NameTest\"");
+        execute("DROP TABLE IF EXISTS PUBLIC.\"BackupTest\"");
 
         super.afterTest();
     }
@@ -496,6 +507,32 @@ public class H2DynamicTableSelfTest extends AbstractSchemaSelfTest {
     }
 
     /**
+     * Test backups propagation.
+     *
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testBackups() throws Exception {
+        String cacheName = "BackupTestCache";
+
+        execute("CREATE TABLE \"BackupTest\" (id BIGINT PRIMARY KEY, name VARCHAR) WITH \"template=" +
+            CACHE_NAME_BACKUPS + ", cache_name=" + cacheName + "\"");
+
+        CacheConfiguration ccfg = grid(0).cache(cacheName).getConfiguration(CacheConfiguration.class);
+
+        assertEquals(DFLT_BACKUPS, ccfg.getBackups());
+
+        execute("DROP TABLE PUBLIC.\"BackupTest\"");
+
+        execute("CREATE TABLE \"BackupTest\" (id BIGINT PRIMARY KEY, name VARCHAR) WITH \"template=" +
+            CACHE_NAME_BACKUPS + ", cache_name=" + cacheName + ", backups=1\"");
+
+        ccfg = grid(0).cache(cacheName).getConfiguration(CacheConfiguration.class);
+
+        assertEquals(1, ccfg.getBackups());
+    }
+
+    /**
      * Test that {@code CREATE TABLE} with given template cache name actually creates new cache,
      * H2 table and type descriptor on all nodes, optionally with cache type check.
      * @param tplCacheName Template cache name.


[24/50] [abbrv] ignite git commit: IGNITE-8135: SQL: authentication for CREATE TABLE and DROP TABLE commands. This closes #3801.

Posted by ag...@apache.org.
IGNITE-8135: SQL: authentication for CREATE TABLE and DROP TABLE commands. This closes #3801.


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

Branch: refs/heads/ignite-7708
Commit: a57c9e1f2a00fea310de5eabba92a642942b9796
Parents: 7a1d0ea
Author: devozerov <vo...@gridgain.com>
Authored: Thu Apr 12 15:02:57 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Apr 12 15:02:57 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/client/ClientException.java   |  3 +-
 .../internal/client/thin/ClientQueryCursor.java |  6 ++-
 .../platform/client/ClientRequestHandler.java   |  7 ++-
 .../cache/ClientCacheSqlFieldsQueryRequest.java | 19 +++++--
 .../security/SecurityContextHolder.java         | 53 ++++++++++++++++++++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  9 ++++
 6 files changed, 91 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/core/src/main/java/org/apache/ignite/client/ClientException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientException.java
index 0555635..b0d9f6c 100644
--- a/modules/core/src/main/java/org/apache/ignite/client/ClientException.java
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientException.java
@@ -20,7 +20,7 @@ package org.apache.ignite.client;
 /**
  * Common thin client checked exception.
  */
-public class ClientException extends Exception {
+public class ClientException extends RuntimeException {
     /** Serial version uid. */
     private static final long serialVersionUID = 0L;
 
@@ -28,6 +28,7 @@ public class ClientException extends Exception {
      * Constructs a new exception with {@code null} as its detail message.
      */
     public ClientException() {
+        // No-op.
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java
index 9367cfd..086fab8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientQueryCursor.java
@@ -54,6 +54,7 @@ class ClientQueryCursor<T> implements QueryCursor<T> {
             pager.close();
         }
         catch (Exception ignored) {
+            // No-op.
         }
     }
 
@@ -76,7 +77,10 @@ class ClientQueryCursor<T> implements QueryCursor<T> {
                         currPageIt = currPage.iterator();
                     }
                     catch (ClientException e) {
-                        throw new RuntimeException("Failed to retrieve query results", e);
+                        throw e;
+                    }
+                    catch (Exception e) {
+                        throw new ClientException("Failed to retrieve query results", e);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
index faa50bc..5ed0d38 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequestHandler.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.authentication.AuthorizationContext
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 import org.apache.ignite.internal.processors.odbc.ClientListenerResponse;
+import org.apache.ignite.internal.processors.security.SecurityContextHolder;
 
 /**
  * Thin client request handler.
@@ -47,8 +48,10 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
 
     /** {@inheritDoc} */
     @Override public ClientListenerResponse handle(ClientListenerRequest req) {
-        if (authCtx != null)
+        if (authCtx != null) {
             AuthorizationContext.context(authCtx);
+            SecurityContextHolder.set(ctx.securityContext());
+        }
 
         try {
             return ((ClientRequest)req).process(ctx);
@@ -56,6 +59,8 @@ public class ClientRequestHandler implements ClientListenerRequestHandler {
         finally {
             if (authCtx != null)
                 AuthorizationContext.clear();
+
+            SecurityContextHolder.clear();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
index 3aa95bf..53f6353 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
@@ -28,8 +28,11 @@ import org.apache.ignite.internal.processors.odbc.jdbc.JdbcStatementType;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
+import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
 import org.apache.ignite.internal.processors.query.QueryUtils;
-import org.apache.ignite.plugin.security.SecurityPermission;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.plugin.security.SecurityException;
 
 /**
  * Sql query request.
@@ -95,7 +98,7 @@ public class ClientCacheSqlFieldsQueryRequest extends ClientCacheRequest {
 
                 if (qry.getSchema() == null) {
                     String schema = QueryUtils.normalizeSchemaName(desc.cacheName(),
-                            desc.cacheConfiguration().getSqlSchema());
+                        desc.cacheConfiguration().getSqlSchema());
 
                     qry.setSchema(schema);
                 }
@@ -108,7 +111,7 @@ public class ClientCacheSqlFieldsQueryRequest extends ClientCacheRequest {
             FieldsQueryCursor cur = curs.get(0);
 
             ClientCacheFieldsQueryCursor cliCur = new ClientCacheFieldsQueryCursor(
-                    cur, qry.getPageSize(), ctx);
+                cur, qry.getPageSize(), ctx);
 
             long cursorId = ctx.resources().put(cliCur);
 
@@ -119,6 +122,16 @@ public class ClientCacheSqlFieldsQueryRequest extends ClientCacheRequest {
         catch (Exception e) {
             ctx.decrementCursors();
 
+            SecurityException securityEx = X.cause(e, SecurityException.class);
+
+            if (securityEx != null) {
+                throw new IgniteClientException(
+                    ClientStatus.SECURITY_VIOLATION,
+                    "Client is not authorized to perform this operation",
+                    securityEx
+                );
+            }
+
             throw e;
         }
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java
new file mode 100644
index 0000000..14d70c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/security/SecurityContextHolder.java
@@ -0,0 +1,53 @@
+/*
+ * 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.security;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Thread-local security context.
+ */
+public class SecurityContextHolder {
+    /** Context. */
+    private static final ThreadLocal<SecurityContext> CTX = new ThreadLocal<>();
+
+    /**
+     * Get security context.
+     *
+     * @return Security context.
+     */
+    @Nullable public static SecurityContext get() {
+        return CTX.get();
+    }
+
+    /**
+     * Set security context.
+     *
+     * @param ctx Context.
+     */
+    public static void set(@Nullable SecurityContext ctx) {
+        CTX.set(ctx);
+    }
+
+    /**
+     * Clear security context.
+     */
+    public static void clear() {
+        set(null);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a57c9e1f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
index b148969..bc5c1e0 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/ddl/DdlStatementsProcessor.java
@@ -34,6 +34,8 @@ import org.apache.ignite.cache.query.FieldsQueryCursor;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
+import org.apache.ignite.internal.processors.authentication.UserManagementOperation;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
@@ -56,6 +58,8 @@ import org.apache.ignite.internal.processors.query.h2.sql.GridSqlDropTable;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlQueryParser;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlStatement;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.internal.processors.security.SecurityContextHolder;
 import org.apache.ignite.internal.sql.command.SqlAlterTableCommand;
 import org.apache.ignite.internal.sql.command.SqlAlterUserCommand;
 import org.apache.ignite.internal.sql.command.SqlCommand;
@@ -67,6 +71,7 @@ import org.apache.ignite.internal.sql.command.SqlIndexColumn;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.security.SecurityPermission;
 import org.h2.command.Prepared;
 import org.h2.command.ddl.AlterTableAlterColumn;
 import org.h2.command.ddl.CreateIndex;
@@ -316,6 +321,8 @@ public class DdlStatementsProcessor {
                 }
             }
             else if (stmt0 instanceof GridSqlCreateTable) {
+                ctx.security().authorize(null, SecurityPermission.CACHE_CREATE, SecurityContextHolder.get());
+
                 GridSqlCreateTable cmd = (GridSqlCreateTable)stmt0;
 
                 if (!F.eq(QueryUtils.DFLT_SCHEMA, cmd.schemaName()))
@@ -349,6 +356,8 @@ public class DdlStatementsProcessor {
                 }
             }
             else if (stmt0 instanceof GridSqlDropTable) {
+                ctx.security().authorize(null, SecurityPermission.CACHE_DESTROY, SecurityContextHolder.get());
+
                 GridSqlDropTable cmd = (GridSqlDropTable)stmt0;
 
                 if (!F.eq(QueryUtils.DFLT_SCHEMA, cmd.schemaName()))


[40/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java
deleted file mode 100644
index bebfe3e..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/MetaoptimizerGroupTrainer.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.ml.Model;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.Chains;
-import org.apache.ignite.ml.trainers.group.chain.ComputationsChain;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/**
- * Group trainer using {@link Metaoptimizer}.
- * Main purpose of this trainer is to extract various transformations (normalizations for example) of data which is processed
- * in the training loop step into distinct entity called metaoptimizer and only fix the main part of logic in
- * trainers extending this class. This way we'll be able to quickly switch between this transformations by using different metaoptimizers
- * without touching main logic.
- *
- * @param <LC> Type of local context.
- * @param <K> Type of data in {@link GroupTrainerCacheKey} keys on which the training is done.
- * @param <V> Type of values of cache used in group training.
- * @param <IN> Data type which is returned by distributed initializer.
- * @param <R> Type of final result returned by nodes on which training is done.
- * @param <I> Type of data which is fed into each training loop step and returned from it.
- * @param <M> Type of model returned after training.
- * @param <T> Type of input of this trainer.
- * @param <G> Type of distributed context which is needed for forming final result which is send from each node to trainer for final model creation.
- * @param <O> Type of output of postprocessor.
- * @param <X> Type of data which is processed by dataProcessor.
- * @param <Y> Type of data which is returned by postprocessor.
- */
-public abstract class MetaoptimizerGroupTrainer<LC extends HasTrainingUUID, K, V, IN extends Serializable,
-    R extends Serializable, I extends Serializable,
-    M extends Model, T extends GroupTrainerInput<K>,
-    G, O extends Serializable, X, Y> extends
-    GroupTrainer<LC, K, V, IN, R, I, M, T, G> {
-    /**
-     * Metaoptimizer.
-     */
-    private Metaoptimizer<LC, X, Y, I, IN, O> metaoptimizer;
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param cache Cache on which group trainer is done.
-     * @param ignite Ignite instance.
-     */
-    public MetaoptimizerGroupTrainer(Metaoptimizer<LC, X, Y, I, IN, O> metaoptimizer,
-        IgniteCache<GroupTrainerCacheKey<K>, V> cache,
-        Ignite ignite) {
-        super(cache, ignite);
-        this.metaoptimizer = metaoptimizer;
-    }
-
-    /**
-     * Get function used to map EntryAndContext to type which is processed by dataProcessor.
-     *
-     * @return Function used to map EntryAndContext to type which is processed by dataProcessor.
-     */
-    protected abstract IgniteFunction<EntryAndContext<K, V, G>, X> trainingLoopStepDataExtractor();
-
-    /**
-     * Get supplier of keys which should be processed by training loop.
-     *
-     * @param locCtx Local text.
-     * @return Supplier of keys which should be processed by training loop.
-     */
-    protected abstract IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysToProcessInTrainingLoop(LC locCtx);
-
-    /**
-     * Get supplier of context used in training loop step.
-     *
-     * @param input Input.
-     * @param ctx Local context.
-     * @return Supplier of context used in training loop step.
-     */
-    protected abstract IgniteSupplier<G> remoteContextExtractor(I input, LC ctx);
-
-    /** {@inheritDoc} */
-    @Override protected void init(T data, UUID trainingUUID) {
-    }
-
-    /**
-     * Get function used to process data in training loop step.
-     *
-     * @return Function used to process data in training loop step.
-     */
-    protected abstract IgniteFunction<X, ResultAndUpdates<Y>> dataProcessor();
-
-    /** {@inheritDoc} */
-    @Override protected ComputationsChain<LC, K, V, I, I> trainingLoopStep() {
-        ComputationsChain<LC, K, V, I, O> chain = Chains.create(new MetaoptimizerDistributedStep<>(metaoptimizer, this));
-        return chain.thenLocally(metaoptimizer::localProcessor);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected I locallyProcessInitData(IN data, LC locCtx) {
-        return metaoptimizer.locallyProcessInitData(data, locCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean shouldContinue(I data, LC locCtx) {
-        return metaoptimizer.shouldContinue(data, locCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteFunction<List<IN>, IN> reduceDistributedInitData() {
-        return metaoptimizer.initialReducer();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java
deleted file mode 100644
index 9ed18af..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/ResultAndUpdates.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.stream.Collectors;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-
-/**
- * Class containing result of computation and updates which should be made for caches.
- * Purpose of this class is mainly performance optimization: suppose we have multiple computations which run in parallel
- * and do some updates to caches. It is more efficient to collect all changes from all this computations and perform them
- * in batch.
- *
- * @param <R> Type of computation result.
- */
-public class ResultAndUpdates<R> {
-    /**
-     * Result of computation.
-     */
-    private R res;
-
-    /**
-     * Updates in the form cache name -> (key -> new value).
-     */
-    private Map<String, Map> updates = new ConcurrentHashMap<>();
-
-    /**
-     * Construct an instance of this class.
-     *
-     * @param res Computation result.
-     */
-    public ResultAndUpdates(R res) {
-        this.res = res;
-    }
-
-    /**
-     * Construct an instance of this class.
-     *
-     * @param res Computation result.
-     * @param updates Map of updates in the form cache name -> (key -> new value).
-     */
-    ResultAndUpdates(R res, Map<String, Map> updates) {
-        this.res = res;
-        this.updates = updates;
-    }
-
-    /**
-     * Construct an empty result.
-     *
-     * @param <R> Result type.
-     * @return Empty result.
-     */
-    public static <R> ResultAndUpdates<R> empty() {
-        return new ResultAndUpdates<>(null);
-    }
-
-    /**
-     * Construct {@link ResultAndUpdates} object from given result.
-     *
-     * @param res Result of computation.
-     * @param <R> Type of result of computation.
-     * @return ResultAndUpdates object.
-     */
-    public static <R> ResultAndUpdates<R> of(R res) {
-        return new ResultAndUpdates<>(res);
-    }
-
-    /**
-     * Add a cache update to this object.
-     *
-     * @param cache Cache to be updated.
-     * @param key Key of cache to be updated.
-     * @param val New value.
-     * @param <K> Type of key of cache to be updated.
-     * @param <V> New value.
-     * @return This object.
-     */
-    @SuppressWarnings("unchecked")
-    public <K, V> ResultAndUpdates<R> updateCache(IgniteCache<K, V> cache, K key, V val) {
-        String name = cache.getName();
-
-        updates.computeIfAbsent(name, s -> new ConcurrentHashMap());
-        updates.get(name).put(key, val);
-
-        return this;
-    }
-
-    /**
-     * Get result of computation.
-     *
-     * @return Result of computation.
-     */
-    public R result() {
-        return res;
-    }
-
-    /**
-     * Sum collection of ResultAndUpdate into one: results are reduced by specified binary operator and updates are merged.
-     *
-     * @param reducer Reducer used to combine computation results.
-     * @param resultsAndUpdates ResultAndUpdates to be combined with.
-     * @param <R> Type of computation result.
-     * @return Sum of collection ResultAndUpdate objects.
-     */
-    @SuppressWarnings("unchecked")
-    static <R> ResultAndUpdates<R> sum(IgniteFunction<List<R>, R> reducer,
-        Collection<ResultAndUpdates<R>> resultsAndUpdates) {
-        Map<String, Map> allUpdates = new HashMap<>();
-
-        for (ResultAndUpdates<R> ru : resultsAndUpdates) {
-            for (String cacheName : ru.updates.keySet()) {
-                allUpdates.computeIfAbsent(cacheName, s -> new HashMap());
-
-                allUpdates.get(cacheName).putAll(ru.updates.get(cacheName));
-            }
-        }
-
-        List<R> results = resultsAndUpdates.stream().map(ResultAndUpdates::result).filter(Objects::nonNull).collect(Collectors.toList());
-
-        return new ResultAndUpdates<>(reducer.apply(results), allUpdates);
-    }
-
-    /**
-     * Get updates map.
-     *
-     * @return Updates map.
-     */
-    public Map<String, Map> updates() {
-        return updates;
-    }
-
-    /**
-     * Set updates map.
-     *
-     * @param updates New updates map.
-     * @return This object.
-     */
-    ResultAndUpdates<R> setUpdates(Map<String, Map> updates) {
-        this.updates = updates;
-        return this;
-    }
-
-    /**
-     * Apply updates to caches.
-     *
-     * @param ignite Ignite instance.
-     */
-    void applyUpdates(Ignite ignite) {
-        for (Map.Entry<String, Map> entry : updates.entrySet()) {
-            IgniteCache<Object, Object> cache = ignite.getOrCreateCache(entry.getKey());
-
-            cache.putAll(entry.getValue());
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java
deleted file mode 100644
index 33ec96a..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdateStrategies.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import org.apache.ignite.ml.optimization.SmoothParametrized;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
-
-/**
- * Holder class for various update strategies.
- */
-public class UpdateStrategies {
-    /**
-     * Simple GD update strategy.
-     *
-     * @return GD update strategy.
-     */
-    public static UpdatesStrategy<SmoothParametrized, SimpleGDParameterUpdate> GD() {
-        return new UpdatesStrategy<>(new SimpleGDUpdateCalculator(), SimpleGDParameterUpdate::sumLocal, SimpleGDParameterUpdate::avg);
-    }
-
-    /**
-     * RProp update strategy.
-     *
-     * @return RProp update strategy.
-     */
-    public static UpdatesStrategy<SmoothParametrized, RPropParameterUpdate> RProp() {
-        return new UpdatesStrategy<>(new RPropUpdateCalculator(), RPropParameterUpdate::sumLocal, RPropParameterUpdate::avg);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java
deleted file mode 100644
index 5288dbf..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/UpdatesStrategy.java
+++ /dev/null
@@ -1,95 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.io.Serializable;
-import java.util.List;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator;
-
-/**
- * Class encapsulating update strategies for group trainers based on updates.
- *
- * @param <M> Type of model to be optimized.
- * @param <U> Type of update.
- */
-public class UpdatesStrategy<M, U extends Serializable> {
-    /**
-     * {@link ParameterUpdateCalculator}.
-     */
-    private ParameterUpdateCalculator<M, U> updatesCalculator;
-
-    /**
-     * Function used to reduce updates in one training (for example, sum all sequential gradient updates to get one
-     * gradient update).
-     */
-    private IgniteFunction<List<U>, U> locStepUpdatesReducer;
-
-    /**
-     * Function used to reduce updates from different trainings (for example, averaging of gradients of all parallel trainings).
-     */
-    private IgniteFunction<List<U>, U> allUpdatesReducer;
-
-    /**
-     * Construct instance of this class with given parameters.
-     *
-     * @param updatesCalculator Parameter update calculator.
-     * @param locStepUpdatesReducer Function used to reduce updates in one training
-     * (for example, sum all sequential gradient updates to get one gradient update).
-     * @param allUpdatesReducer Function used to reduce updates from different trainings
-     * (for example, averaging of gradients of all parallel trainings).
-     */
-    public UpdatesStrategy(
-        ParameterUpdateCalculator<M, U> updatesCalculator,
-        IgniteFunction<List<U>, U> locStepUpdatesReducer,
-        IgniteFunction<List<U>, U> allUpdatesReducer) {
-        this.updatesCalculator = updatesCalculator;
-        this.locStepUpdatesReducer = locStepUpdatesReducer;
-        this.allUpdatesReducer = allUpdatesReducer;
-    }
-
-    /**
-     * Get parameter update calculator (see {@link ParameterUpdateCalculator}).
-     *
-     * @return Parameter update calculator.
-     */
-    public ParameterUpdateCalculator<M, U> getUpdatesCalculator() {
-        return updatesCalculator;
-    }
-
-    /**
-     * Get function used to reduce updates in one training
-     * (for example, sum all sequential gradient updates to get one gradient update).
-     *
-     * @return Function used to reduce updates in one training
-     * (for example, sum all sequential gradient updates to get on gradient update).
-     */
-    public IgniteFunction<List<U>, U> locStepUpdatesReducer() {
-        return locStepUpdatesReducer;
-    }
-
-    /**
-     * Get function used to reduce updates from different trainings
-     * (for example, averaging of gradients of all parallel trainings).
-     *
-     * @return Function used to reduce updates from different trainings.
-     */
-    public IgniteFunction<List<U>, U> allUpdatesReducer() {
-        return allUpdatesReducer;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java
deleted file mode 100644
index db4f13f..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/Chains.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.io.Serializable;
-
-/**
- * Class containing methods creating {@link ComputationsChain}.
- */
-public class Chains {
-    /**
-     * Create computation chain consisting of one returning its input as output.
-     *
-     * @param <L> Type of local context of created chain.
-     * @param <K> Type of keys of cache used in computation chain.
-     * @param <V> Type of values of cache used in computation chain.
-     * @param <I> Type of input to computation chain.
-     * @return Computation chain consisting of one returning its input as output.
-     */
-    public static <L extends HasTrainingUUID, K, V, I> ComputationsChain<L, K, V, I, I> create() {
-        return (input, context) -> input;
-    }
-
-    /**
-     * Create {@link ComputationsChain} from {@link DistributedEntryProcessingStep}.
-     *
-     * @param step Distributed chain step.
-     * @param <L> Type of local context of created chain.
-     * @param <K> Type of keys of cache used in computation chain.
-     * @param <V> Type of values of cache used in computation chain.
-     * @param <C> Type of context used by worker in {@link DistributedEntryProcessingStep}.
-     * @param <I> Type of input to computation chain.
-     * @param <O> Type of output of computation chain.
-     * @return Computation created from {@link DistributedEntryProcessingStep}.
-     */
-    public static <L extends HasTrainingUUID, K, V, C, I, O extends Serializable> ComputationsChain<L, K, V, I, O> create(
-        DistributedEntryProcessingStep<L, K, V, C, I, O> step) {
-        ComputationsChain<L, K, V, I, I> chain = create();
-        return chain.thenDistributedForEntries(step);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java
deleted file mode 100644
index 3c3bdab..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/ComputationsChain.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import javax.cache.processor.EntryProcessor;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.cluster.ClusterGroup;
-import org.apache.ignite.lang.IgniteBiPredicate;
-import org.apache.ignite.ml.math.functions.Functions;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey;
-import org.apache.ignite.ml.trainers.group.GroupTrainerEntriesProcessorTask;
-import org.apache.ignite.ml.trainers.group.GroupTrainerKeysProcessorTask;
-import org.apache.ignite.ml.trainers.group.GroupTrainingContext;
-import org.apache.ignite.ml.trainers.group.ResultAndUpdates;
-
-/**
- * This class encapsulates convenient way for creating computations chain for distributed model training.
- * Chain is meant in the sense that output of each non-final computation is fed as input to next computation.
- * Chain is basically a bi-function from context and input to output, context is separated from input
- * because input is specific to each individual step and context is something which is convenient to have access to in each of steps.
- * Context is separated into two parts: local context and remote context.
- * There are two kinds of computations: local and distributed.
- * Local steps are just functions from two arguments: input and local context.
- * Distributed steps are more sophisticated, but basically can be thought as functions of form
- * localContext -> (function of remote context -> output), locally we fix local context and get function
- * (function of remote context -> output) which is executed distributed.
- * Chains are composable through 'then' method.
- *
- * @param <L> Type of local context.
- * @param <K> Type of cache keys.
- * @param <V> Type of cache values.
- * @param <I> Type of input of this chain.
- * @param <O> Type of output of this chain.
- * // TODO: IGNITE-7405 check if it is possible to integrate with {@link EntryProcessor}.
- */
-@FunctionalInterface
-public interface ComputationsChain<L extends HasTrainingUUID, K, V, I, O> {
-    /**
-     * Process given input and {@link GroupTrainingContext}.
-     *
-     * @param input Computation chain input.
-     * @param ctx {@link GroupTrainingContext}.
-     * @return Result of processing input and context.
-     */
-    O process(I input, GroupTrainingContext<K, V, L> ctx);
-
-    /**
-     * Add a local step to this chain.
-     *
-     * @param locStep Local step.
-     * @param <O1> Output of local step.
-     * @return Composition of this chain and local step.
-     */
-    default <O1> ComputationsChain<L, K, V, I, O1> thenLocally(IgniteBiFunction<O, L, O1> locStep) {
-        ComputationsChain<L, K, V, O, O1> nextStep = (input, context) -> locStep.apply(input, context.localContext());
-        return then(nextStep);
-    }
-
-    /**
-     * Add a distributed step which works in the following way:
-     * 1. apply local context and input to local context extractor and keys supplier to get corresponding suppliers;
-     * 2. on each node_n
-     * 2.1. get context object.
-     * 2.2. for each entry_i e located on node_n with key_i from keys stream compute worker((context, entry_i)) and get
-     * (cachesUpdates_i, result_i).
-     * 2.3. for all i on node_n merge cacheUpdates_i and apply them.
-     * 2.4. for all i on node_n, reduce result_i into result_n.
-     * 3. get all result_n, reduce them into result and return result.
-     *
-     * @param <O1> Type of worker output.
-     * @param <G> Type of context used by worker.
-     * @param workerCtxExtractor Extractor of context for worker.
-     * @param worker Function computed on each entry of cache used for training. Second argument is context:
-     * common part of data which is independent from key.
-     * @param ks Function from chain input and local context to supplier of keys for worker.
-     * @param reducer Function used for reducing results of worker.
-     * @return Combination of this chain and distributed step specified by given parameters.
-     */
-    default <O1 extends Serializable, G> ComputationsChain<L, K, V, I, O1> thenDistributedForEntries(
-        IgniteBiFunction<O, L, IgniteSupplier<G>> workerCtxExtractor,
-        IgniteFunction<EntryAndContext<K, V, G>, ResultAndUpdates<O1>> worker,
-        IgniteBiFunction<O, L, IgniteSupplier<Stream<GroupTrainerCacheKey<K>>>> ks,
-        IgniteFunction<List<O1>, O1> reducer) {
-        ComputationsChain<L, K, V, O, O1> nextStep = (input, context) -> {
-            L locCtx = context.localContext();
-            IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keysSupplier = ks.apply(input, locCtx);
-
-            Ignite ignite = context.ignite();
-            UUID trainingUUID = context.localContext().trainingUUID();
-            String cacheName = context.cache().getName();
-            ClusterGroup grp = ignite.cluster().forDataNodes(cacheName);
-
-            // Apply first two arguments locally because it is common for all nodes.
-            IgniteSupplier<G> extractor = Functions.curry(workerCtxExtractor).apply(input).apply(locCtx);
-
-            return ignite.compute(grp).execute(new GroupTrainerEntriesProcessorTask<>(trainingUUID, extractor, worker, keysSupplier, reducer, cacheName, ignite), null);
-        };
-        return then(nextStep);
-    }
-
-    /**
-     * Add a distributed step which works in the following way:
-     * 1. apply local context and input to local context extractor and keys supplier to get corresponding suppliers;
-     * 2. on each node_n
-     * 2.1. get context object.
-     * 2.2. for each key_i from keys stream such that key_i located on node_n compute worker((context, entry_i)) and get
-     * (cachesUpdates_i, result_i).
-     * 2.3. for all i on node_n merge cacheUpdates_i and apply them.
-     * 2.4. for all i on node_n, reduce result_i into result_n.
-     * 3. get all result_n, reduce them into result and return result.
-     *
-     * @param <O1> Type of worker output.
-     * @param <G> Type of context used by worker.
-     * @param workerCtxExtractor Extractor of context for worker.
-     * @param worker Function computed on each entry of cache used for training. Second argument is context:
-     * common part of data which is independent from key.
-     * @param keysSupplier Function from chain input and local context to supplier of keys for worker.
-     * @param reducer Function used for reducing results of worker.
-     * @return Combination of this chain and distributed step specified by given parameters.
-     */
-    default <O1 extends Serializable, G> ComputationsChain<L, K, V, I, O1> thenDistributedForKeys(
-        IgniteBiFunction<O, L, IgniteSupplier<G>> workerCtxExtractor,
-        IgniteFunction<KeyAndContext<K, G>, ResultAndUpdates<O1>> worker,
-        IgniteBiFunction<O, L, IgniteSupplier<Stream<GroupTrainerCacheKey<K>>>> keysSupplier,
-        IgniteFunction<List<O1>, O1> reducer) {
-        ComputationsChain<L, K, V, O, O1> nextStep = (input, context) -> {
-            L locCtx = context.localContext();
-            IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> ks = keysSupplier.apply(input, locCtx);
-
-            Ignite ignite = context.ignite();
-            UUID trainingUUID = context.localContext().trainingUUID();
-            String cacheName = context.cache().getName();
-            ClusterGroup grp = ignite.cluster().forDataNodes(cacheName);
-
-            // Apply first argument locally because it is common for all nodes.
-            IgniteSupplier<G> extractor = Functions.curry(workerCtxExtractor).apply(input).apply(locCtx);
-
-            return ignite.compute(grp).execute(new GroupTrainerKeysProcessorTask<>(trainingUUID, extractor, worker, ks, reducer, cacheName, ignite), null);
-        };
-        return then(nextStep);
-    }
-
-    /**
-     * Add a distributed step specified by {@link DistributedEntryProcessingStep}.
-     *
-     * @param step Distributed step.
-     * @param <O1> Type of output of distributed step.
-     * @param <G> Type of context of distributed step.
-     * @return Combination of this chain and distributed step specified by input.
-     */
-    default <O1 extends Serializable, G> ComputationsChain<L, K, V, I, O1> thenDistributedForEntries(
-        DistributedEntryProcessingStep<L, K, V, G, O, O1> step) {
-        return thenDistributedForEntries(step::remoteContextSupplier, step.worker(), step::keys, step.reducer());
-    }
-
-    /**
-     * Add a distributed step specified by {@link DistributedKeyProcessingStep}.
-     *
-     * @param step Distributed step.
-     * @param <O1> Type of output of distributed step.
-     * @param <G> Type of context of distributed step.
-     * @return Combination of this chain and distributed step specified by input.
-     */
-    default <O1 extends Serializable, G> ComputationsChain<L, K, V, I, O1> thenDistributedForKeys(
-        DistributedKeyProcessingStep<L, K, G, O, O1> step) {
-        return thenDistributedForKeys(step::remoteContextSupplier, step.worker(), step::keys, step.reducer());
-    }
-
-    /**
-     * Version of 'thenDistributedForKeys' where worker does not depend on context.
-     *
-     * @param worker Worker.
-     * @param kf Function providing supplier
-     * @param reducer Function from chain input and local context to supplier of keys for worker.
-     * @param <O1> Type of worker output.
-     * @return Combination of this chain and distributed step specified by given parameters.
-     */
-    default <O1 extends Serializable> ComputationsChain<L, K, V, I, O1> thenDistributedForKeys(
-        IgniteFunction<GroupTrainerCacheKey<K>, ResultAndUpdates<O1>> worker,
-        IgniteBiFunction<O, L, IgniteSupplier<Stream<GroupTrainerCacheKey<K>>>> kf,
-        IgniteFunction<List<O1>, O1> reducer) {
-
-        return thenDistributedForKeys((o, lc) -> () -> o, (context) -> worker.apply(context.key()), kf, reducer);
-    }
-
-    /**
-     * Combine this computation chain with other computation chain in the following way:
-     * 1. perform this calculations chain and get result r.
-     * 2. while 'cond(r)' is true, r = otherChain(r, context)
-     * 3. return r.
-     *
-     * @param cond Condition checking if 'while' loop should continue.
-     * @param otherChain Chain to be combined with this chain.
-     * @return Combination of this chain and otherChain.
-     */
-    default ComputationsChain<L, K, V, I, O> thenWhile(IgniteBiPredicate<O, L> cond,
-        ComputationsChain<L, K, V, O, O> otherChain) {
-        ComputationsChain<L, K, V, I, O> me = this;
-        return (input, context) -> {
-            O res = me.process(input, context);
-
-            while (cond.apply(res, context.localContext()))
-                res = otherChain.process(res, context);
-
-            return res;
-        };
-    }
-
-    /**
-     * Combine this chain with other: feed this chain as input to other, pass same context as second argument to both chains
-     * process method.
-     *
-     * @param next Next chain.
-     * @param <O1> Type of next chain output.
-     * @return Combined chain.
-     */
-    default <O1> ComputationsChain<L, K, V, I, O1> then(ComputationsChain<L, K, V, O, O1> next) {
-        ComputationsChain<L, K, V, I, O> me = this;
-        return (input, context) -> {
-            O myRes = me.process(input, context);
-            return next.process(myRes, context);
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java
deleted file mode 100644
index 8fd1264..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedEntryProcessingStep.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.io.Serializable;
-
-/**
- * {@link DistributedStep} specialized to {@link EntryAndContext}.
- *
- * @param <L> Local context.
- * @param <K> Type of keys of cache used for group training.
- * @param <V> Type of values of cache used for group training.
- * @param <C> Context used by worker.
- * @param <I> Type of input to this step.
- * @param <O> Type of output of this step.
- */
-public interface DistributedEntryProcessingStep<L, K, V, C, I, O extends Serializable> extends
-    DistributedStep<EntryAndContext<K, V, C>, L, K, C, I, O> {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java
deleted file mode 100644
index fb8d867..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedKeyProcessingStep.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.io.Serializable;
-
-/**
- * {@link DistributedStep} specialized to {@link KeyAndContext}.
- *
- * @param <L> Local context.
- * @param <K> Type of keys of cache used for group training.
- * @param <C> Context used by worker.
- * @param <I> Type of input to this step.
- * @param <O> Type of output of this step.
- */
-public interface DistributedKeyProcessingStep<L, K, C, I, O extends Serializable> extends
-    DistributedStep<KeyAndContext<K, C>, L, K, C, I, O> {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java
deleted file mode 100644
index 804a886..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/DistributedStep.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey;
-import org.apache.ignite.ml.trainers.group.ResultAndUpdates;
-
-/**
- * Class encapsulating logic of distributed step in {@link ComputationsChain}.
- *
- * @param <T> Type of elements to be processed by worker.
- * @param <L> Local context.
- * @param <K> Type of keys of cache used for group training.
- * @param <C> Context used by worker.
- * @param <I> Type of input to this step.
- * @param <O> Type of output of this step.
- */
-public interface DistributedStep<T, L, K, C, I, O extends Serializable> {
-    /**
-     * Create supplier of context used by worker.
-     *
-     * @param input Input.
-     * @param locCtx Local context.
-     * @return Context used by worker.
-     */
-    IgniteSupplier<C> remoteContextSupplier(I input, L locCtx);
-
-    /**
-     * Get function applied to each cache element specified by keys.
-     *
-     * @return Function applied to each cache entry specified by keys..
-     */
-    IgniteFunction<T, ResultAndUpdates<O>> worker();
-
-    /**
-     * Get supplier of keys for worker.
-     *
-     * @param input Input to this step.
-     * @param locCtx Local context.
-     * @return Keys for worker.
-     */
-    IgniteSupplier<Stream<GroupTrainerCacheKey<K>>> keys(I input, L locCtx);
-
-    /**
-     * Get function used to reduce results returned by worker.
-     *
-     * @return Function used to reduce results returned by worker..
-     */
-    IgniteFunction<List<O>, O> reducer();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java
deleted file mode 100644
index 59c3b34..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/EntryAndContext.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.util.Map;
-import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey;
-
-/**
- * Entry of cache used for group training and context.
- * This class is used as input for workers of distributed steps of {@link ComputationsChain}.
- *
- * @param <K> Type of cache keys used for training.
- * @param <V> Type of cache values used for training.
- * @param <C> Type of context.
- */
-public class EntryAndContext<K, V, C> {
-    /**
-     * Entry of cache used for training.
-     */
-    private Map.Entry<GroupTrainerCacheKey<K>, V> entry;
-
-    /**
-     * Context.
-     */
-    private C ctx;
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param entry Entry of cache used for training.
-     * @param ctx Context.
-     */
-    public EntryAndContext(Map.Entry<GroupTrainerCacheKey<K>, V> entry, C ctx) {
-        this.entry = entry;
-        this.ctx = ctx;
-    }
-
-    /**
-     * Get entry of cache used for training.
-     *
-     * @return Entry of cache used for training.
-     */
-    public Map.Entry<GroupTrainerCacheKey<K>, V> entry() {
-        return entry;
-    }
-
-    /**
-     * Get context.
-     *
-     * @return Context.
-     */
-    public C context() {
-        return ctx;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java
deleted file mode 100644
index d855adf..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/HasTrainingUUID.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import java.util.UUID;
-
-/**
- * Interface for classes which contain UUID of training.
- */
-public interface HasTrainingUUID {
-    /**
-     * Get training UUID.
-     *
-     * @return Training UUID.
-     */
-    UUID trainingUUID();
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java
deleted file mode 100644
index ba36e0e..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/group/chain/KeyAndContext.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group.chain;
-
-import org.apache.ignite.ml.trainers.group.GroupTrainerCacheKey;
-
-/**
- * Class containing key and remote context (see explanation of remote context in {@link ComputationsChain}).
- *
- * @param <K> Cache key type.
- * @param <C> Remote context.
- */
-public class KeyAndContext<K, C> {
-    /**
-     * Key of group trainer.
-     */
-    private GroupTrainerCacheKey<K> key;
-
-    /**
-     * Remote context.
-     */
-    private C ctx;
-
-    /**
-     * Construct instance of this class.
-     *
-     * @param key Cache key.
-     * @param ctx Remote context.
-     */
-    public KeyAndContext(GroupTrainerCacheKey<K> key, C ctx) {
-        this.key = key;
-        this.ctx = ctx;
-    }
-
-    /**
-     * Get group trainer cache key.
-     *
-     * @return Group trainer cache key.
-     */
-    public GroupTrainerCacheKey<K> key() {
-        return key;
-    }
-
-    /**
-     * Get remote context.
-     *
-     * @return Remote context.
-     */
-    public C context() {
-        return ctx;
-    }
-}

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
index 9900f85..0c3408e 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/IgniteMLTestSuite.java
@@ -23,11 +23,9 @@ import org.apache.ignite.ml.genetic.GAGridTestSuite;
 import org.apache.ignite.ml.knn.KNNTestSuite;
 import org.apache.ignite.ml.math.MathImplMainTestSuite;
 import org.apache.ignite.ml.nn.MLPTestSuite;
-import org.apache.ignite.ml.optimization.OptimizationTestSuite;
 import org.apache.ignite.ml.preprocessing.PreprocessingTestSuite;
 import org.apache.ignite.ml.regressions.RegressionsTestSuite;
 import org.apache.ignite.ml.svm.SVMTestSuite;
-import org.apache.ignite.ml.trainers.group.TrainersGroupTestSuite;
 import org.apache.ignite.ml.tree.DecisionTreeTestSuite;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
@@ -45,8 +43,6 @@ import org.junit.runners.Suite;
     KNNTestSuite.class,
     LocalModelsTest.class,
     MLPTestSuite.class,
-    TrainersGroupTestSuite.class,
-    OptimizationTestSuite.class,
     DatasetTestSuite.class,
     PreprocessingTestSuite.class,
     GAGridTestSuite.class

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java
index ec9fdaa..bdd1eea 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeapTest.java
@@ -22,7 +22,7 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
-import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap;
+import org.apache.ignite.ml.dataset.primitive.builder.data.SimpleLabeledDatasetDataBuilder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
@@ -64,9 +64,9 @@ public class LSQROnHeapTest {
 
         LSQROnHeap<Integer, double[]> lsqr = new LSQROnHeap<>(
             datasetBuilder,
-            new LinSysPartitionDataBuilderOnHeap<>(
+            new SimpleLabeledDatasetDataBuilder<>(
                 (k, v) -> Arrays.copyOf(v, v.length - 1),
-                (k, v) -> v[3]
+                (k, v) -> new double[]{v[3]}
             )
         );
 
@@ -87,9 +87,9 @@ public class LSQROnHeapTest {
 
         LSQROnHeap<Integer, double[]> lsqr = new LSQROnHeap<>(
             datasetBuilder,
-            new LinSysPartitionDataBuilderOnHeap<>(
+            new SimpleLabeledDatasetDataBuilder<>(
                 (k, v) -> Arrays.copyOf(v, v.length - 1),
-                (k, v) -> v[3]
+                (k, v) -> new double[]{v[3]}
             )
         );
 
@@ -118,9 +118,9 @@ public class LSQROnHeapTest {
 
         try (LSQROnHeap<Integer, double[]> lsqr = new LSQROnHeap<>(
             datasetBuilder,
-            new LinSysPartitionDataBuilderOnHeap<>(
+            new SimpleLabeledDatasetDataBuilder<>(
                 (k, v) -> Arrays.copyOf(v, v.length - 1),
-                (k, v) -> v[4]
+                (k, v) -> new double[]{v[4]}
             )
         )) {
             LSQRResult res = lsqr.solve(0, 1e-12, 1e-12, 1e8, -1, false, null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
index 038b880..654ebe0 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
@@ -31,7 +31,6 @@ import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.optimization.updatecalculators.*;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
index c53f6f1..db14881 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.optimization.updatecalculators.*;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.experimental.runners.Enclosed;

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
index a64af9b..3b65a28 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
@@ -32,7 +32,7 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
 import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
index d966484..4063312 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
@@ -27,7 +27,7 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
 import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.junit.Test;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java
deleted file mode 100644
index f6f4775..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java
+++ /dev/null
@@ -1,64 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization;
-
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.junit.Test;
-
-/**
- * Tests for {@link GradientDescent}.
- */
-public class GradientDescentTest {
-    /** */
-    private static final double PRECISION = 1e-6;
-
-    /**
-     * Test gradient descent optimization on function y = x^2 with gradient function 2 * x.
-     */
-    @Test
-    public void testOptimize() {
-        GradientDescent gradientDescent = new GradientDescent(
-            (inputs, groundTruth, point) -> point.times(2),
-            new SimpleUpdater(0.01)
-        );
-
-        Vector res = gradientDescent.optimize(new DenseLocalOnHeapMatrix(new double[1][1]),
-            new DenseLocalOnHeapVector(new double[]{ 2.0 }));
-
-        TestUtils.assertEquals(0, res.get(0), PRECISION);
-    }
-
-    /**
-     * Test gradient descent optimization on function y = (x - 2)^2 with gradient function 2 * (x - 2).
-     */
-    @Test
-    public void testOptimizeWithOffset() {
-        GradientDescent gradientDescent = new GradientDescent(
-            (inputs, groundTruth, point) -> point.minus(new DenseLocalOnHeapVector(new double[]{ 2.0 })).times(2.0),
-            new SimpleUpdater(0.01)
-        );
-
-        Vector res = gradientDescent.optimize(new DenseLocalOnHeapMatrix(new double[1][1]),
-            new DenseLocalOnHeapVector(new double[]{ 2.0 }));
-
-        TestUtils.assertEquals(2, res.get(0), PRECISION);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java
deleted file mode 100644
index 0ae6e4c..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/OptimizationTestSuite.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization;
-
-import org.apache.ignite.ml.optimization.util.SparseDistributedMatrixMapReducerTest;
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-/**
- * Test suite for group trainer tests.
- */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-    GradientDescentTest.class,
-    SparseDistributedMatrixMapReducerTest.class
-})
-public class OptimizationTestSuite {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java
deleted file mode 100644
index 9017c43..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java
+++ /dev/null
@@ -1,135 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.optimization.util;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Tests for {@link SparseDistributedMatrixMapReducer}.
- */
-public class SparseDistributedMatrixMapReducerTest extends GridCommonAbstractTest {
-    /** Number of nodes in grid */
-    private static final int NODE_COUNT = 2;
-
-    /** */
-    private Ignite ignite;
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() {
-        stopAllGrids();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        /* Grid instance. */
-        ignite = grid(NODE_COUNT);
-        ignite.configuration().setPeerClassLoadingEnabled(true);
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-    }
-
-    /**
-     * Tests that matrix 100x100 filled by "1.0" and distributed across nodes successfully processed (calculate sum of
-     * all elements) via {@link SparseDistributedMatrixMapReducer}.
-     */
-    public void testMapReduce() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(100, 100);
-        for (int i = 0; i < 100; i++)
-            for (int j = 0; j < 100; j++)
-                distributedMatrix.set(i, j, 1);
-        SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix);
-        double total = mapReducer.mapReduce(
-            (matrix, args) -> {
-                double partialSum = 0.0;
-                for (int i = 0; i < matrix.rowSize(); i++)
-                    for (int j = 0; j < matrix.columnSize(); j++)
-                        partialSum += matrix.get(i, j);
-                return partialSum;
-            },
-            sums -> {
-                double totalSum = 0;
-                for (Double partialSum : sums)
-                    if (partialSum != null)
-                        totalSum += partialSum;
-                return totalSum;
-            }, 0.0);
-        assertEquals(100.0 * 100.0, total, 1e-18);
-    }
-
-    /**
-     * Tests that matrix 100x100 filled by "1.0" and distributed across nodes successfully processed via
-     * {@link SparseDistributedMatrixMapReducer} even when mapping function returns {@code null}.
-     */
-    public void testMapReduceWithNullValues() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(100, 100);
-        for (int i = 0; i < 100; i++)
-            for (int j = 0; j < 100; j++)
-                distributedMatrix.set(i, j, 1);
-        SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix);
-        double total = mapReducer.mapReduce(
-            (matrix, args) -> null,
-            sums -> {
-                double totalSum = 0;
-                for (Double partialSum : sums)
-                    if (partialSum != null)
-                        totalSum += partialSum;
-                return totalSum;
-            }, 0.0);
-        assertEquals(0, total, 1e-18);
-    }
-
-    /**
-     * Tests that matrix 1x100 filled by "1.0" and distributed across nodes successfully processed (calculate sum of
-     * all elements) via {@link SparseDistributedMatrixMapReducer} even when not all nodes contains data.
-     */
-    public void testMapReduceWithOneEmptyNode() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(1, 100);
-        for (int j = 0; j < 100; j++)
-            distributedMatrix.set(0, j, 1);
-        SparseDistributedMatrixMapReducer mapReducer = new SparseDistributedMatrixMapReducer(distributedMatrix);
-        double total = mapReducer.mapReduce(
-            (matrix, args) -> {
-                double partialSum = 0.0;
-                for (int i = 0; i < matrix.rowSize(); i++)
-                    for (int j = 0; j < matrix.columnSize(); j++)
-                        partialSum += matrix.get(i, j);
-                return partialSum;
-            },
-            sums -> {
-                double totalSum = 0;
-                for (Double partialSum : sums)
-                    if (partialSum != null)
-                        totalSum += partialSum;
-                return totalSum;
-            }, 0.0);
-        assertEquals(100.0, total, 1e-18);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
index b3c9368..5005ef2 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
@@ -27,9 +27,6 @@ import org.junit.runners.Suite;
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
     LinearRegressionModelTest.class,
-    LocalLinearRegressionQRTrainerTest.class,
-    DistributedLinearRegressionQRTrainerTest.class,
-    BlockDistributedLinearRegressionQRTrainerTest.class,
     LinearRegressionLSQRTrainerTest.class,
     LinearRegressionSGDTrainerTest.class
 })


[36/50] [abbrv] ignite git commit: IGNITE-8256 Fixed simulated node failure in the test

Posted by ag...@apache.org.
IGNITE-8256 Fixed simulated node failure in the test


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

Branch: refs/heads/ignite-7708
Commit: 568c3e79ed474a9870685ab4b01454af1645866c
Parents: 8a42f64
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Fri Apr 13 18:01:00 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Fri Apr 13 18:01:00 2018 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/TxRecoveryStoreEnabledTest.java         | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/568c3e79/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
index 060af21..30ac83d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/TxRecoveryStoreEnabledTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.cache.store.CacheStoreAdapter;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.failure.StopNodeFailureHandler;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.managers.discovery.IgniteDiscoverySpi;
@@ -81,6 +82,8 @@ public class TxRecoveryStoreEnabledTest extends GridCommonAbstractTest {
 
         cfg.setCacheConfiguration(ccfg);
 
+        cfg.setFailureHandler(new StopNodeFailureHandler());
+
         return cfg;
     }
 


[33/50] [abbrv] ignite git commit: IGNITE-8042: .NET: Thin client: authentication support - fix TestAuthenticationEmptyCredentials

Posted by ag...@apache.org.
IGNITE-8042: .NET: Thin client: authentication support - fix TestAuthenticationEmptyCredentials


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

Branch: refs/heads/ignite-7708
Commit: 5fd5968e5e7119571ae315af66f5f23032ff96da
Parents: 8550d61
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Fri Apr 13 13:36:20 2018 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Fri Apr 13 13:36:20 2018 +0300

----------------------------------------------------------------------
 .../dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5fd5968e/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
index 0a6b1a7..67d1c52 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientConnectionTest.cs
@@ -100,7 +100,7 @@ namespace Apache.Ignite.Core.Tests.Client
 
                 cliCfg.UserName = null;
                 ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });
-                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.Username cannot be null"));
+                Assert.IsTrue(ex.Message.StartsWith("IgniteClientConfiguration.UserName cannot be null"));
 
                 cliCfg.UserName = "";
                 ex = Assert.Throws<IgniteClientException>(() => { Ignition.StartClient(cliCfg); });


[37/50] [abbrv] ignite git commit: IGNITE-8233: KNN and SVM algorithms don't work when partition doesn't contain data.

Posted by ag...@apache.org.
IGNITE-8233: KNN and SVM algorithms don't work when partition doesn't contain data.

this closes #3807


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

Branch: refs/heads/ignite-7708
Commit: ee9ca06a8cbec6eec2b963fb1db1b3f383fc1837
Parents: 568c3e7
Author: dmitrievanthony <dm...@gmail.com>
Authored: Fri Apr 13 18:02:37 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Fri Apr 13 18:02:37 2018 +0300

----------------------------------------------------------------------
 .../dataset/impl/cache/CacheBasedDataset.java   |  14 ++-
 .../dataset/impl/cache/util/ComputeUtils.java   |   9 +-
 .../ml/dataset/impl/local/LocalDataset.java     |  16 ++-
 .../dataset/impl/local/LocalDatasetBuilder.java |   8 +-
 .../classification/KNNClassificationModel.java  |  32 +++--
 .../impl/cache/CacheBasedDatasetTest.java       |   9 +-
 .../ignite/ml/knn/KNNClassificationTest.java    | 120 ++++++++++--------
 .../apache/ignite/ml/knn/KNNRegressionTest.java | 122 +++++++++++--------
 .../ignite/ml/knn/LabeledDatasetHelper.java     |  45 ++-----
 .../ignite/ml/knn/LabeledDatasetTest.java       |  54 ++++----
 10 files changed, 226 insertions(+), 203 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
index 463d496..7428faf 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDataset.java
@@ -101,12 +101,16 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
                 partDataBuilder
             );
 
-            R res = map.apply(ctx, data, part);
+            if (data != null) {
+                R res = map.apply(ctx, data, part);
 
-            // Saves partition context after update.
-            ComputeUtils.saveContext(Ignition.localIgnite(), datasetCacheName, part, ctx);
+                // Saves partition context after update.
+                ComputeUtils.saveContext(Ignition.localIgnite(), datasetCacheName, part, ctx);
 
-            return res;
+                return res;
+            }
+
+            return null;
         }, reduce, identity);
     }
 
@@ -125,7 +129,7 @@ public class CacheBasedDataset<K, V, C extends Serializable, D extends AutoClose
                 partDataBuilder
             );
 
-            return map.apply(data, part);
+            return data != null ? map.apply(data, part) : null;
         }, reduce, identity);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
index 0785db2..ce2fcfd 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/cache/util/ComputeUtils.java
@@ -163,9 +163,14 @@ public class ComputeUtils {
             qry.setPartition(part);
 
             long cnt = upstreamCache.localSizeLong(part);
-            try (QueryCursor<Cache.Entry<K, V>> cursor = upstreamCache.query(qry)) {
-                return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx);
+
+            if (cnt > 0) {
+                try (QueryCursor<Cache.Entry<K, V>> cursor = upstreamCache.query(qry)) {
+                    return partDataBuilder.build(new UpstreamCursorAdapter<>(cursor.iterator(), cnt), cnt, ctx);
+                }
             }
+
+            return null;
         });
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
index c08b7de..e312b20 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDataset.java
@@ -55,8 +55,12 @@ public class LocalDataset<C extends Serializable, D extends AutoCloseable> imple
         R identity) {
         R res = identity;
 
-        for (int part = 0; part < ctx.size(); part++)
-            res = reduce.apply(res, map.apply(ctx.get(part), data.get(part), part));
+        for (int part = 0; part < ctx.size(); part++) {
+            D partData = data.get(part);
+
+            if (partData != null)
+                res = reduce.apply(res, map.apply(ctx.get(part), partData, part));
+        }
 
         return res;
     }
@@ -65,8 +69,12 @@ public class LocalDataset<C extends Serializable, D extends AutoCloseable> imple
     @Override public <R> R compute(IgniteBiFunction<D, Integer, R> map, IgniteBinaryOperator<R> reduce, R identity) {
         R res = identity;
 
-        for (int part = 0; part < data.size(); part++)
-            res = reduce.apply(res, map.apply(data.get(part), part));
+        for (int part = 0; part < data.size(); part++) {
+            D partData = data.get(part);
+
+            if (partData != null)
+                res = reduce.apply(res, map.apply(partData, part));
+        }
 
         return res;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
index 0dc1ed6..cfc1801 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/dataset/impl/local/LocalDatasetBuilder.java
@@ -69,16 +69,16 @@ public class LocalDatasetBuilder<K, V> implements DatasetBuilder<K, V> {
         for (int part = 0; part < partitions; part++) {
             int cnt = part == partitions - 1 ? upstreamMap.size() - ptr : Math.min(partSize, upstreamMap.size() - ptr);
 
-            C ctx = partCtxBuilder.build(
+            C ctx = cnt > 0 ? partCtxBuilder.build(
                 new IteratorWindow<>(firstKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt),
                 cnt
-            );
+            ) : null;
 
-            D data = partDataBuilder.build(
+            D data = cnt > 0 ? partDataBuilder.build(
                 new IteratorWindow<>(secondKeysIter, k -> new UpstreamEntry<>(k, upstreamMap.get(k)), cnt),
                 cnt,
                 ctx
-            );
+            ) : null;
 
             ctxList.add(ctx);
             dataList.add(data);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
index 693b81d..0f0cc9f 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/classification/KNNClassificationModel.java
@@ -151,19 +151,29 @@ public class KNNClassificationModel<K, V> implements Model<Vector, Double>, Expo
      */
     @NotNull private LabeledVector[] getKClosestVectors(LabeledDataset<Double, LabeledVector> trainingData,
         TreeMap<Double, Set<Integer>> distanceIdxPairs) {
-        LabeledVector[] res = new LabeledVector[k];
-        int i = 0;
-        final Iterator<Double> iter = distanceIdxPairs.keySet().iterator();
-        while (i < k) {
-            double key = iter.next();
-            Set<Integer> idxs = distanceIdxPairs.get(key);
-            for (Integer idx : idxs) {
-                res[i] = trainingData.getRow(idx);
-                i++;
-                if (i >= k)
-                    break; // go to next while-loop iteration
+        LabeledVector[] res;
+
+        if (trainingData.rowSize() <= k) {
+            res = new LabeledVector[trainingData.rowSize()];
+            for (int i = 0; i < trainingData.rowSize(); i++)
+                res[i] = trainingData.getRow(i);
+        }
+        else {
+            res = new LabeledVector[k];
+            int i = 0;
+            final Iterator<Double> iter = distanceIdxPairs.keySet().iterator();
+            while (i < k) {
+                double key = iter.next();
+                Set<Integer> idxs = distanceIdxPairs.get(key);
+                for (Integer idx : idxs) {
+                    res[i] = trainingData.getRow(idx);
+                    i++;
+                    if (i >= k)
+                        break; // go to next while-loop iteration
+                }
             }
         }
+
         return res;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java
index dc0e160..16ba044 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/dataset/impl/cache/CacheBasedDatasetTest.java
@@ -38,6 +38,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.ml.dataset.primitive.data.SimpleDatasetData;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -81,9 +82,9 @@ public class CacheBasedDatasetTest extends GridCommonAbstractTest {
 
         CacheBasedDatasetBuilder<Integer, String> builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache);
 
-        CacheBasedDataset<Integer, String, Long, AutoCloseable> dataset = builder.build(
+        CacheBasedDataset<Integer, String, Long, SimpleDatasetData> dataset = builder.build(
             (upstream, upstreamSize) -> upstreamSize,
-            (upstream, upstreamSize, ctx) -> null
+            (upstream, upstreamSize, ctx) -> new SimpleDatasetData(new double[0], 0)
         );
 
         assertTrue("Before computation all partitions should not be reserved",
@@ -133,9 +134,9 @@ public class CacheBasedDatasetTest extends GridCommonAbstractTest {
 
         CacheBasedDatasetBuilder<Integer, String> builder = new CacheBasedDatasetBuilder<>(ignite, upstreamCache);
 
-        CacheBasedDataset<Integer, String, Long, AutoCloseable> dataset = builder.build(
+        CacheBasedDataset<Integer, String, Long, SimpleDatasetData> dataset = builder.build(
             (upstream, upstreamSize) -> upstreamSize,
-            (upstream, upstreamSize, ctx) -> null
+            (upstream, upstreamSize, ctx) -> new SimpleDatasetData(new double[0], 0)
         );
 
         assertTrue("Before computation all partitions should not be reserved",

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
index 0877fc0..004718e 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
@@ -17,11 +17,11 @@
 
 package org.apache.ignite.ml.knn;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
-import org.junit.Assert;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
 import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
@@ -29,121 +29,137 @@ import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static junit.framework.TestCase.assertEquals;
 
 /** Tests behaviour of KNNClassificationTest. */
+@RunWith(Parameterized.class)
 public class KNNClassificationTest {
-    /** Precision in test checks. */
-    private static final double PRECISION = 1e-2;
+    /** Number of parts to be tested. */
+    private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7, 100};
+
+    /** Number of partitions. */
+    @Parameterized.Parameter
+    public int parts;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "Data divided on {0} partitions, training with batch size {1}")
+    public static Iterable<Integer[]> data() {
+        List<Integer[]> res = new ArrayList<>();
+
+        for (int part : partsToBeTested)
+            res.add(new Integer[] {part});
+
+        return res;
+    }
 
     /** */
     @Test
-    public void binaryClassificationTest() {
-
+    public void testBinaryClassificationTest() {
         Map<Integer, double[]> data = new HashMap<>();
-        data.put(0, new double[]{1.0, 1.0, 1.0});
-        data.put(1, new double[]{1.0, 2.0, 1.0});
-        data.put(2, new double[]{2.0, 1.0, 1.0});
-        data.put(3, new double[]{-1.0, -1.0, 2.0});
-        data.put(4, new double[]{-1.0, -2.0, 2.0});
-        data.put(5, new double[]{-2.0, -1.0, 2.0});
+        data.put(0, new double[] {1.0, 1.0, 1.0});
+        data.put(1, new double[] {1.0, 2.0, 1.0});
+        data.put(2, new double[] {2.0, 1.0, 1.0});
+        data.put(3, new double[] {-1.0, -1.0, 2.0});
+        data.put(4, new double[] {-1.0, -2.0, 2.0});
+        data.put(5, new double[] {-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
             data,
-            2,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0});
-        Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION);
-        Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0});
-        Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION);
+        Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0});
+        assertEquals(knnMdl.apply(firstVector), 1.0);
+        Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0});
+        assertEquals(knnMdl.apply(secondVector), 2.0);
     }
 
     /** */
     @Test
-    public void binaryClassificationWithSmallestKTest() {
+    public void testBinaryClassificationWithSmallestKTest() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{1.0, 1.0, 1.0});
-        data.put(1, new double[]{1.0, 2.0, 1.0});
-        data.put(2, new double[]{2.0, 1.0, 1.0});
-        data.put(3, new double[]{-1.0, -1.0, 2.0});
-        data.put(4, new double[]{-1.0, -2.0, 2.0});
-        data.put(5, new double[]{-2.0, -1.0, 2.0});
+        data.put(0, new double[] {1.0, 1.0, 1.0});
+        data.put(1, new double[] {1.0, 2.0, 1.0});
+        data.put(2, new double[] {2.0, 1.0, 1.0});
+        data.put(3, new double[] {-1.0, -1.0, 2.0});
+        data.put(4, new double[] {-1.0, -2.0, 2.0});
+        data.put(5, new double[] {-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
             data,
-            2,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(1)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector firstVector = new DenseLocalOnHeapVector(new double[]{2.0, 2.0});
-        Assert.assertEquals(knnMdl.apply(firstVector), 1.0, PRECISION);
-        Vector secondVector = new DenseLocalOnHeapVector(new double[]{-2.0, -2.0});
-        Assert.assertEquals(knnMdl.apply(secondVector), 2.0, PRECISION);
+        Vector firstVector = new DenseLocalOnHeapVector(new double[] {2.0, 2.0});
+        assertEquals(knnMdl.apply(firstVector), 1.0);
+        Vector secondVector = new DenseLocalOnHeapVector(new double[] {-2.0, -2.0});
+        assertEquals(knnMdl.apply(secondVector), 2.0);
     }
 
     /** */
     @Test
-    public void binaryClassificationFarPointsWithSimpleStrategy() {
+    public void testBinaryClassificationFarPointsWithSimpleStrategy() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{10.0, 10.0, 1.0});
-        data.put(1, new double[]{10.0, 20.0, 1.0});
-        data.put(2, new double[]{-1, -1, 1.0});
-        data.put(3, new double[]{-2, -2, 2.0});
-        data.put(4, new double[]{-1.0, -2.0, 2.0});
-        data.put(5, new double[]{-2.0, -1.0, 2.0});
+        data.put(0, new double[] {10.0, 10.0, 1.0});
+        data.put(1, new double[] {10.0, 20.0, 1.0});
+        data.put(2, new double[] {-1, -1, 1.0});
+        data.put(3, new double[] {-2, -2, 2.0});
+        data.put(4, new double[] {-1.0, -2.0, 2.0});
+        data.put(5, new double[] {-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
             data,
-            2,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01});
-        Assert.assertEquals(knnMdl.apply(vector), 2.0, PRECISION);
+        Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01});
+        assertEquals(knnMdl.apply(vector), 2.0);
     }
 
     /** */
     @Test
-    public void binaryClassificationFarPointsWithWeightedStrategy() {
+    public void testBinaryClassificationFarPointsWithWeightedStrategy() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{10.0, 10.0, 1.0});
-        data.put(1, new double[]{10.0, 20.0, 1.0});
-        data.put(2, new double[]{-1, -1, 1.0});
-        data.put(3, new double[]{-2, -2, 2.0});
-        data.put(4, new double[]{-1.0, -2.0, 2.0});
-        data.put(5, new double[]{-2.0, -1.0, 2.0});
+        data.put(0, new double[] {10.0, 10.0, 1.0});
+        data.put(1, new double[] {10.0, 20.0, 1.0});
+        data.put(2, new double[] {-1, -1, 1.0});
+        data.put(3, new double[] {-2, -2, 2.0});
+        data.put(4, new double[] {-1.0, -2.0, 2.0});
+        data.put(5, new double[] {-2.0, -1.0, 2.0});
 
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
             data,
-            2,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.WEIGHTED);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[]{-1.01, -1.01});
-        Assert.assertEquals(knnMdl.apply(vector), 1.0, PRECISION);
+        Vector vector = new DenseLocalOnHeapVector(new double[] {-1.01, -1.01});
+        assertEquals(knnMdl.apply(vector), 1.0);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
index ce9cae5..0c26ba9 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNRegressionTest.java
@@ -17,6 +17,11 @@
 
 package org.apache.ignite.ml.knn;
 
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
 import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
 import org.apache.ignite.ml.knn.regression.KNNRegressionModel;
@@ -25,110 +30,125 @@ import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.junit.Assert;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
 import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
 
 /**
  * Tests for {@link KNNRegressionTrainer}.
  */
+@RunWith(Parameterized.class)
 public class KNNRegressionTest {
+    /** Number of parts to be tested. */
+    private static final int[] partsToBeTested = new int[] {1, 2, 3, 4, 5, 7, 100};
+
+    /** Number of partitions. */
+    @Parameterized.Parameter
+    public int parts;
+
+    /** Parameters. */
+    @Parameterized.Parameters(name = "Data divided on {0} partitions, training with batch size {1}")
+    public static Iterable<Integer[]> data() {
+        List<Integer[]> res = new ArrayList<>();
+
+        for (int part : partsToBeTested)
+            res.add(new Integer[] {part});
+
+        return res;
+    }
+
     /** */
     @Test
-    public void simpleRegressionWithOneNeighbour() {
+    public void testSimpleRegressionWithOneNeighbour() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{11.0, 0, 0, 0, 0, 0});
-        data.put(1, new double[]{12.0, 2.0, 0, 0, 0, 0});
-        data.put(2, new double[]{13.0, 0, 3.0, 0, 0, 0});
-        data.put(3, new double[]{14.0, 0, 0, 4.0, 0, 0});
-        data.put(4, new double[]{15.0, 0, 0, 0, 5.0, 0});
-        data.put(5, new double[]{16.0, 0, 0, 0, 0, 6.0});
+        data.put(0, new double[] {11.0, 0, 0, 0, 0, 0});
+        data.put(1, new double[] {12.0, 2.0, 0, 0, 0, 0});
+        data.put(2, new double[] {13.0, 0, 3.0, 0, 0, 0});
+        data.put(3, new double[] {14.0, 0, 0, 4.0, 0, 0});
+        data.put(4, new double[] {15.0, 0, 0, 0, 5.0, 0});
+        data.put(5, new double[] {16.0, 0, 0, 0, 0, 6.0});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
         KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            new LocalDatasetBuilder<>(data, parts),
             (k, v) -> Arrays.copyOfRange(v, 1, v.length),
             (k, v) -> v[0]
         ).withK(1)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[]{0, 0, 0, 5.0, 0.0});
+        Vector vector = new DenseLocalOnHeapVector(new double[] {0, 0, 0, 5.0, 0.0});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(15, knnMdl.apply(vector), 1E-12);
     }
 
     /** */
     @Test
-    public void longly() {
+    public void testLongly() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947});
-        data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948});
-        data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949});
-        data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950});
-        data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951});
-        data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952});
-        data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953});
-        data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954});
-        data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955});
-        data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957});
-        data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958});
-        data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959});
-        data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960});
-        data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961});
-        data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
         KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            new LocalDatasetBuilder<>(data, parts),
             (k, v) -> Arrays.copyOfRange(v, 1, v.length),
             (k, v) -> v[0]
         ).withK(3)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956});
+        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
     }
 
     /** */
+    @Test
     public void testLonglyWithWeightedStrategy() {
         Map<Integer, double[]> data = new HashMap<>();
-
-        data.put(0, new double[]{60323, 83.0, 234289, 2356, 1590, 107608, 1947});
-        data.put(1, new double[]{61122, 88.5, 259426, 2325, 1456, 108632, 1948});
-        data.put(2, new double[]{60171, 88.2, 258054, 3682, 1616, 109773, 1949});
-        data.put(3, new double[]{61187, 89.5, 284599, 3351, 1650, 110929, 1950});
-        data.put(4, new double[]{63221, 96.2, 328975, 2099, 3099, 112075, 1951});
-        data.put(5, new double[]{63639, 98.1, 346999, 1932, 3594, 113270, 1952});
-        data.put(6, new double[]{64989, 99.0, 365385, 1870, 3547, 115094, 1953});
-        data.put(7, new double[]{63761, 100.0, 363112, 3578, 3350, 116219, 1954});
-        data.put(8, new double[]{66019, 101.2, 397469, 2904, 3048, 117388, 1955});
-        data.put(9, new double[]{68169, 108.4, 442769, 2936, 2798, 120445, 1957});
-        data.put(10, new double[]{66513, 110.8, 444546, 4681, 2637, 121950, 1958});
-        data.put(11, new double[]{68655, 112.6, 482704, 3813, 2552, 123366, 1959});
-        data.put(12, new double[]{69564, 114.2, 502601, 3931, 2514, 125368, 1960});
-        data.put(13, new double[]{69331, 115.7, 518173, 4806, 2572, 127852, 1961});
-        data.put(14, new double[]{70551, 116.9, 554894, 4007, 2827, 130081, 1962});
+        data.put(0, new double[] {60323, 83.0, 234289, 2356, 1590, 107608, 1947});
+        data.put(1, new double[] {61122, 88.5, 259426, 2325, 1456, 108632, 1948});
+        data.put(2, new double[] {60171, 88.2, 258054, 3682, 1616, 109773, 1949});
+        data.put(3, new double[] {61187, 89.5, 284599, 3351, 1650, 110929, 1950});
+        data.put(4, new double[] {63221, 96.2, 328975, 2099, 3099, 112075, 1951});
+        data.put(5, new double[] {63639, 98.1, 346999, 1932, 3594, 113270, 1952});
+        data.put(6, new double[] {64989, 99.0, 365385, 1870, 3547, 115094, 1953});
+        data.put(7, new double[] {63761, 100.0, 363112, 3578, 3350, 116219, 1954});
+        data.put(8, new double[] {66019, 101.2, 397469, 2904, 3048, 117388, 1955});
+        data.put(9, new double[] {68169, 108.4, 442769, 2936, 2798, 120445, 1957});
+        data.put(10, new double[] {66513, 110.8, 444546, 4681, 2637, 121950, 1958});
+        data.put(11, new double[] {68655, 112.6, 482704, 3813, 2552, 123366, 1959});
+        data.put(12, new double[] {69564, 114.2, 502601, 3931, 2514, 125368, 1960});
+        data.put(13, new double[] {69331, 115.7, 518173, 4806, 2572, 127852, 1961});
+        data.put(14, new double[] {70551, 116.9, 554894, 4007, 2827, 130081, 1962});
 
         KNNRegressionTrainer trainer = new KNNRegressionTrainer();
 
         KNNRegressionModel knnMdl = (KNNRegressionModel) trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            new LocalDatasetBuilder<>(data, parts),
             (k, v) -> Arrays.copyOfRange(v, 1, v.length),
             (k, v) -> v[0]
         ).withK(3)
             .withDistanceMeasure(new EuclideanDistance())
             .withStrategy(KNNStrategy.SIMPLE);
 
-        Vector vector = new DenseLocalOnHeapVector(new double[]{104.6, 419180, 2822, 2857, 118734, 1956});
+        Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
         System.out.println(knnMdl.apply(vector));
         Assert.assertEquals(67857, knnMdl.apply(vector), 2000);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
index a25b303..dbcdb99 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetHelper.java
@@ -21,64 +21,33 @@ import java.io.IOException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import org.apache.ignite.Ignite;
 import org.apache.ignite.ml.structures.LabeledDataset;
 import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
  * Base class for decision trees test.
  */
-public class LabeledDatasetHelper extends GridCommonAbstractTest {
-    /** Count of nodes. */
-    private static final int NODE_COUNT = 4;
-
+public class LabeledDatasetHelper {
     /** Separator. */
     private static final String SEPARATOR = "\t";
 
-    /** Grid instance. */
-    protected Ignite ignite;
-
-    /**
-     * Default constructor.
-     */
-    public LabeledDatasetHelper() {
-        super(false);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
     /**
      * Loads labeled dataset from file with .txt extension.
      *
      * @param rsrcPath path to dataset.
      * @return null if path is incorrect.
      */
-    LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) {
+    public static LabeledDataset loadDatasetFromTxt(String rsrcPath, boolean isFallOnBadData) {
         try {
-            Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI());
+            Path path = Paths.get(LabeledDatasetHelper.class.getClassLoader().getResource(rsrcPath).toURI());
             try {
                 return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData);
-            } catch (IOException e) {
+            }
+            catch (IOException e) {
                 e.printStackTrace();
             }
-        } catch (URISyntaxException e) {
+        }
+        catch (URISyntaxException e) {
             e.printStackTrace();
             return null;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee9ca06a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
index 77d40a6..e986740 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/LabeledDatasetTest.java
@@ -21,7 +21,6 @@ import java.io.IOException;
 import java.net.URISyntaxException;
 import java.nio.file.Path;
 import java.nio.file.Paths;
-import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.math.ExternalizableTest;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.exceptions.CardinalityException;
@@ -32,9 +31,13 @@ import org.apache.ignite.ml.structures.LabeledDataset;
 import org.apache.ignite.ml.structures.LabeledDatasetTestTrainPair;
 import org.apache.ignite.ml.structures.LabeledVector;
 import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
+import org.junit.Test;
+
+import static junit.framework.TestCase.assertEquals;
+import static junit.framework.TestCase.fail;
 
 /** Tests behaviour of KNNClassificationTest. */
-public class LabeledDatasetTest extends LabeledDatasetHelper implements ExternalizableTest<LabeledDataset> {
+public class LabeledDatasetTest implements ExternalizableTest<LabeledDataset> {
     /** */
     private static final String KNN_IRIS_TXT = "datasets/knn/iris.txt";
 
@@ -51,9 +54,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     private static final String IRIS_MISSED_DATA = "datasets/knn/missed_data.txt";
 
     /** */
+    @Test
     public void testFeatureNames() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},
@@ -71,9 +73,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testAccessMethods() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},
@@ -98,9 +99,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testFailOnYNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},
@@ -122,9 +122,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testFailOnXNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {};
         double[] lbs = new double[] {1.0, 1.0, 1.0, 2.0, 2.0, 2.0};
@@ -140,18 +139,17 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testLoadingCorrectTxtFile() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        LabeledDataset training = loadDatasetFromTxt(KNN_IRIS_TXT, false);
+        LabeledDataset training = LabeledDatasetHelper.loadDatasetFromTxt(KNN_IRIS_TXT, false);
         assertEquals(training.rowSize(), 150);
     }
 
     /** */
+    @Test
     public void testLoadingEmptyFile() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         try {
-            loadDatasetFromTxt(EMPTY_TXT, false);
+            LabeledDatasetHelper.loadDatasetFromTxt(EMPTY_TXT, false);
             fail("EmptyFileException");
         }
         catch (EmptyFileException e) {
@@ -161,11 +159,10 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testLoadingFileWithFirstEmptyRow() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         try {
-            loadDatasetFromTxt(NO_DATA_TXT, false);
+            LabeledDatasetHelper.loadDatasetFromTxt(NO_DATA_TXT, false);
             fail("NoDataException");
         }
         catch (NoDataException e) {
@@ -175,19 +172,17 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testLoadingFileWithIncorrectData() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        LabeledDataset training = loadDatasetFromTxt(IRIS_INCORRECT_TXT, false);
+        LabeledDataset training = LabeledDatasetHelper.loadDatasetFromTxt(IRIS_INCORRECT_TXT, false);
         assertEquals(149, training.rowSize());
     }
 
     /** */
+    @Test
     public void testFailOnLoadingFileWithIncorrectData() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         try {
-            loadDatasetFromTxt(IRIS_INCORRECT_TXT, true);
+            LabeledDatasetHelper.loadDatasetFromTxt(IRIS_INCORRECT_TXT, true);
             fail("FileParsingException");
         }
         catch (FileParsingException e) {
@@ -198,9 +193,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testLoadingFileWithMissedData() throws URISyntaxException, IOException {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         Path path = Paths.get(this.getClass().getClassLoader().getResource(IRIS_MISSED_DATA).toURI());
 
         LabeledDataset training = LabeledDatasetLoader.loadFromTxtFile(path, ",", false, false);
@@ -209,9 +203,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testSplitting() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},
@@ -246,9 +239,8 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
     }
 
     /** */
+    @Test
     public void testLabels() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},
@@ -267,8 +259,6 @@ public class LabeledDatasetTest extends LabeledDatasetHelper implements External
 
     /** */
     @Override public void testExternalization() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
         double[][] mtx =
             new double[][] {
                 {1.0, 1.0},


[13/50] [abbrv] ignite git commit: IGNITE-8204: SQL: fixed hangs when lazy flag is enabled. This closes #3785.

Posted by ag...@apache.org.
IGNITE-8204: SQL: fixed hangs when lazy flag is enabled. This closes #3785.


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

Branch: refs/heads/ignite-7708
Commit: 747e6c5f9c635a5b9c6856efd2b94b05297b7f25
Parents: 98ef925
Author: Alexander Paschenko <al...@gmail.com>
Authored: Wed Apr 11 16:20:16 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Apr 11 16:20:16 2018 +0300

----------------------------------------------------------------------
 .../query/h2/twostep/GridMapQueryExecutor.java         |  7 +++++++
 .../query/h2/twostep/MapQueryLazyWorker.java           | 13 +++++++++++--
 2 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/747e6c5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index 9b1e4fa..930ada2 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -234,6 +234,13 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * @return Busy lock for lazy workers to guard their operations with.
+     */
+    GridSpinBusyLock busyLock() {
+        return busyLock;
+    }
+
+    /**
      * @param node Node.
      * @param msg Message.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/747e6c5f/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
index 59c050f..98f3df9 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/MapQueryLazyWorker.java
@@ -80,8 +80,17 @@ public class MapQueryLazyWorker extends GridWorker {
             while (!isCancelled()) {
                 Runnable task = tasks.take();
 
-                if (task != null)
-                    task.run();
+                if (task != null) {
+                    if (!exec.busyLock().enterBusy())
+                        return;
+
+                    try {
+                        task.run();
+                    }
+                    finally {
+                        exec.busyLock().leaveBusy();
+                    }
+                }
             }
         }
         finally {


[42/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
IGNITE-8232: ML package cleanup for 2.5 release

this closes #3806


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

Branch: refs/heads/ignite-7708
Commit: 47cfdc27e5079ee0ec91de1539bff498ffc1edc2
Parents: ee9ca06
Author: dmitrievanthony <dm...@gmail.com>
Authored: Fri Apr 13 18:08:08 2018 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Fri Apr 13 18:08:08 2018 +0300

----------------------------------------------------------------------
 .../examples/ml/nn/MLPTrainerExample.java       |   2 +-
 ...nWithLSQRTrainerAndNormalizationExample.java | 180 ---------
 ...dLinearRegressionWithLSQRTrainerExample.java | 169 --------
 ...tedLinearRegressionWithQRTrainerExample.java | 137 -------
 ...edLinearRegressionWithSGDTrainerExample.java | 177 --------
 .../LinearRegressionLSQRTrainerExample.java     | 169 ++++++++
 ...sionLSQRTrainerWithNormalizationExample.java | 180 +++++++++
 .../LinearRegressionSGDTrainerExample.java      | 176 ++++++++
 .../main/java/org/apache/ignite/ml/Trainer.java |  36 --
 .../apache/ignite/ml/estimators/Estimators.java |  50 ---
 .../ignite/ml/estimators/package-info.java      |  22 -
 .../ml/math/functions/IgniteBiFunction.java     |   8 +-
 .../LinSysPartitionDataBuilderOnHeap.java       |  86 ----
 .../math/isolve/LinSysPartitionDataOnHeap.java  |  65 ---
 .../ml/math/isolve/lsqr/AbstractLSQR.java       |   3 +-
 .../ignite/ml/math/isolve/lsqr/LSQROnHeap.java  |  27 +-
 .../org/apache/ignite/ml/nn/MLPTrainer.java     |   1 -
 .../apache/ignite/ml/nn/UpdatesStrategy.java    |  95 +++++
 .../ignite/ml/optimization/GradientDescent.java | 202 ----------
 .../ml/optimization/GradientFunction.java       |  31 --
 .../LeastSquaresGradientFunction.java           |  33 --
 .../util/SparseDistributedMatrixMapReducer.java |  84 ----
 .../ml/optimization/util/package-info.java      |  22 -
 .../linear/LinearRegressionLSQRTrainer.java     |  10 +-
 .../linear/LinearRegressionQRTrainer.java       |  72 ----
 .../linear/LinearRegressionSGDTrainer.java      |   7 +-
 .../org/apache/ignite/ml/trainers/Trainer.java  |  33 --
 .../trainers/group/BaseLocalProcessorJob.java   | 146 -------
 .../ignite/ml/trainers/group/ConstModel.java    |  46 ---
 .../ignite/ml/trainers/group/GroupTrainer.java  | 208 ----------
 .../group/GroupTrainerBaseProcessorTask.java    | 144 -------
 .../ml/trainers/group/GroupTrainerCacheKey.java | 125 ------
 .../group/GroupTrainerEntriesProcessorTask.java |  64 ---
 .../ml/trainers/group/GroupTrainerInput.java    |  37 --
 .../group/GroupTrainerKeysProcessorTask.java    |  62 ---
 .../ml/trainers/group/GroupTrainingContext.java |  98 -----
 .../group/LocalEntriesProcessorJob.java         |  85 ----
 .../trainers/group/LocalKeysProcessorJob.java   |  78 ----
 .../ignite/ml/trainers/group/Metaoptimizer.java |  93 -----
 .../group/MetaoptimizerDistributedStep.java     |  97 -----
 .../group/MetaoptimizerGroupTrainer.java        | 132 ------
 .../ml/trainers/group/ResultAndUpdates.java     | 178 --------
 .../ml/trainers/group/UpdateStrategies.java     |  47 ---
 .../ml/trainers/group/UpdatesStrategy.java      |  95 -----
 .../ignite/ml/trainers/group/chain/Chains.java  |  56 ---
 .../trainers/group/chain/ComputationsChain.java | 246 -----------
 .../chain/DistributedEntryProcessingStep.java   |  34 --
 .../chain/DistributedKeyProcessingStep.java     |  33 --
 .../trainers/group/chain/DistributedStep.java   |  70 ----
 .../trainers/group/chain/EntryAndContext.java   |  70 ----
 .../trainers/group/chain/HasTrainingUUID.java   |  32 --
 .../ml/trainers/group/chain/KeyAndContext.java  |  67 ---
 .../ml/trainers/group/chain/package-info.java   |  22 -
 .../ignite/ml/trainers/group/package-info.java  |  22 -
 .../org/apache/ignite/ml/IgniteMLTestSuite.java |   4 -
 .../ml/math/isolve/lsqr/LSQROnHeapTest.java     |  14 +-
 .../ignite/ml/nn/MLPTrainerIntegrationTest.java |   1 -
 .../org/apache/ignite/ml/nn/MLPTrainerTest.java |   1 -
 .../MLPTrainerMnistIntegrationTest.java         |   2 +-
 .../ml/nn/performance/MLPTrainerMnistTest.java  |   2 +-
 .../ml/optimization/GradientDescentTest.java    |  64 ---
 .../ml/optimization/OptimizationTestSuite.java  |  33 --
 .../SparseDistributedMatrixMapReducerTest.java  | 135 -------
 .../ml/regressions/RegressionsTestSuite.java    |   3 -
 .../linear/ArtificialRegressionDatasets.java    | 404 -------------------
 ...istributedLinearRegressionQRTrainerTest.java |  36 --
 ...istributedLinearRegressionQRTrainerTest.java |  36 --
 .../GenericLinearRegressionTrainerTest.java     | 206 ----------
 ...wareAbstractLinearRegressionTrainerTest.java | 127 ------
 .../linear/LinearRegressionSGDTrainerTest.java  |   2 +-
 .../LocalLinearRegressionQRTrainerTest.java     |  36 --
 .../group/DistributedWorkersChainTest.java      | 189 ---------
 .../ml/trainers/group/GroupTrainerTest.java     |  90 -----
 .../trainers/group/SimpleGroupTrainerInput.java |  63 ---
 .../ml/trainers/group/TestGroupTrainer.java     | 144 -------
 .../group/TestGroupTrainerLocalContext.java     |  85 ----
 .../trainers/group/TestGroupTrainingCache.java  |  70 ----
 .../group/TestGroupTrainingSecondCache.java     |  56 ---
 .../ml/trainers/group/TestLocalContext.java     |  51 ---
 .../ml/trainers/group/TestTrainingLoopStep.java |  65 ---
 .../trainers/group/TrainersGroupTestSuite.java  |  32 --
 ...iteOLSMultipleLinearRegressionBenchmark.java |  69 ----
 .../yardstick/ml/regression/package-info.java   |  22 -
 83 files changed, 666 insertions(+), 5840 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
index ce44cc6..5d1ac38 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
@@ -32,7 +32,7 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
 import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
 import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
 import org.apache.ignite.thread.IgniteThread;
 
 /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
deleted file mode 100644
index 99e6577..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
+++ /dev/null
@@ -1,180 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.ml.regression.linear;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.ScanQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor;
-import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
-import org.apache.ignite.thread.IgniteThread;
-
-import javax.cache.Cache;
-import java.util.Arrays;
-import java.util.UUID;
-
-/**
- * Run linear regression model over distributed matrix.
- *
- * @see LinearRegressionLSQRTrainer
- * @see NormalizationTrainer
- * @see NormalizationPreprocessor
- */
-public class DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample {
-    /** */
-    private static final double[][] data = {
-        {8, 78, 284, 9.100000381, 109},
-        {9.300000191, 68, 433, 8.699999809, 144},
-        {7.5, 70, 739, 7.199999809, 113},
-        {8.899999619, 96, 1792, 8.899999619, 97},
-        {10.19999981, 74, 477, 8.300000191, 206},
-        {8.300000191, 111, 362, 10.89999962, 124},
-        {8.800000191, 77, 671, 10, 152},
-        {8.800000191, 168, 636, 9.100000381, 162},
-        {10.69999981, 82, 329, 8.699999809, 150},
-        {11.69999981, 89, 634, 7.599999905, 134},
-        {8.5, 149, 631, 10.80000019, 292},
-        {8.300000191, 60, 257, 9.5, 108},
-        {8.199999809, 96, 284, 8.800000191, 111},
-        {7.900000095, 83, 603, 9.5, 182},
-        {10.30000019, 130, 686, 8.699999809, 129},
-        {7.400000095, 145, 345, 11.19999981, 158},
-        {9.600000381, 112, 1357, 9.699999809, 186},
-        {9.300000191, 131, 544, 9.600000381, 177},
-        {10.60000038, 80, 205, 9.100000381, 127},
-        {9.699999809, 130, 1264, 9.199999809, 179},
-        {11.60000038, 140, 688, 8.300000191, 80},
-        {8.100000381, 154, 354, 8.399999619, 103},
-        {9.800000191, 118, 1632, 9.399999619, 101},
-        {7.400000095, 94, 348, 9.800000191, 117},
-        {9.399999619, 119, 370, 10.39999962, 88},
-        {11.19999981, 153, 648, 9.899999619, 78},
-        {9.100000381, 116, 366, 9.199999809, 102},
-        {10.5, 97, 540, 10.30000019, 95},
-        {11.89999962, 176, 680, 8.899999619, 80},
-        {8.399999619, 75, 345, 9.600000381, 92},
-        {5, 134, 525, 10.30000019, 126},
-        {9.800000191, 161, 870, 10.39999962, 108},
-        {9.800000191, 111, 669, 9.699999809, 77},
-        {10.80000019, 114, 452, 9.600000381, 60},
-        {10.10000038, 142, 430, 10.69999981, 71},
-        {10.89999962, 238, 822, 10.30000019, 86},
-        {9.199999809, 78, 190, 10.69999981, 93},
-        {8.300000191, 196, 867, 9.600000381, 106},
-        {7.300000191, 125, 969, 10.5, 162},
-        {9.399999619, 82, 499, 7.699999809, 95},
-        {9.399999619, 125, 925, 10.19999981, 91},
-        {9.800000191, 129, 353, 9.899999619, 52},
-        {3.599999905, 84, 288, 8.399999619, 110},
-        {8.399999619, 183, 718, 10.39999962, 69},
-        {10.80000019, 119, 540, 9.199999809, 57},
-        {10.10000038, 180, 668, 13, 106},
-        {9, 82, 347, 8.800000191, 40},
-        {10, 71, 345, 9.199999809, 50},
-        {11.30000019, 118, 463, 7.800000191, 35},
-        {11.30000019, 121, 728, 8.199999809, 86},
-        {12.80000019, 68, 383, 7.400000095, 57},
-        {10, 112, 316, 10.39999962, 57},
-        {6.699999809, 109, 388, 8.899999619, 94}
-    };
-
-    /** Run example. */
-    public static void main(String[] args) throws InterruptedException {
-        System.out.println();
-        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
-        // Start ignite grid.
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println(">>> Ignite grid started.");
-
-            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
-            // because we create ignite cache internally.
-            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                SparseDistributedMatrixExample.class.getSimpleName(), () -> {
-                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
-
-                System.out.println(">>> Create new normalization trainer object.");
-                NormalizationTrainer<Integer, double[]> normalizationTrainer = new NormalizationTrainer<>();
-
-                System.out.println(">>> Perform the training to get the normalization preprocessor.");
-                IgniteBiFunction<Integer, double[], double[]> preprocessor = normalizationTrainer.fit(
-                    ignite,
-                    dataCache,
-                    (k, v) -> Arrays.copyOfRange(v, 1, v.length)
-                );
-
-                System.out.println(">>> Create new linear regression trainer object.");
-                LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
-
-                System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel mdl = trainer.fit(ignite, dataCache, preprocessor, (k, v) -> v[0]);
-
-                System.out.println(">>> Linear regression model: " + mdl);
-
-                System.out.println(">>> ---------------------------------");
-                System.out.println(">>> | Prediction\t| Ground Truth\t|");
-                System.out.println(">>> ---------------------------------");
-
-                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
-                    for (Cache.Entry<Integer, double[]> observation : observations) {
-                        Integer key = observation.getKey();
-                        double[] val = observation.getValue();
-                        double groundTruth = val[0];
-
-                        double prediction = mdl.apply(new DenseLocalOnHeapVector(preprocessor.apply(key, val)));
-
-                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
-                    }
-                }
-
-                System.out.println(">>> ---------------------------------");
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-
-    /**
-     * Fills cache with data and returns it.
-     *
-     * @param ignite Ignite instance.
-     * @return Filled Ignite Cache.
-     */
-    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
-        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
-        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
-        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
-
-        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
-
-        for (int i = 0; i < data.length; i++)
-            cache.put(i, data[i]);
-
-        return cache;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
deleted file mode 100644
index 25aec0c..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.ml.regression.linear;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.ScanQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
-import org.apache.ignite.thread.IgniteThread;
-
-import javax.cache.Cache;
-import java.util.Arrays;
-import java.util.UUID;
-
-/**
- * Run linear regression model over distributed matrix.
- *
- * @see LinearRegressionLSQRTrainer
- */
-public class DistributedLinearRegressionWithLSQRTrainerExample {
-    /** */
-    private static final double[][] data = {
-        {8, 78, 284, 9.100000381, 109},
-        {9.300000191, 68, 433, 8.699999809, 144},
-        {7.5, 70, 739, 7.199999809, 113},
-        {8.899999619, 96, 1792, 8.899999619, 97},
-        {10.19999981, 74, 477, 8.300000191, 206},
-        {8.300000191, 111, 362, 10.89999962, 124},
-        {8.800000191, 77, 671, 10, 152},
-        {8.800000191, 168, 636, 9.100000381, 162},
-        {10.69999981, 82, 329, 8.699999809, 150},
-        {11.69999981, 89, 634, 7.599999905, 134},
-        {8.5, 149, 631, 10.80000019, 292},
-        {8.300000191, 60, 257, 9.5, 108},
-        {8.199999809, 96, 284, 8.800000191, 111},
-        {7.900000095, 83, 603, 9.5, 182},
-        {10.30000019, 130, 686, 8.699999809, 129},
-        {7.400000095, 145, 345, 11.19999981, 158},
-        {9.600000381, 112, 1357, 9.699999809, 186},
-        {9.300000191, 131, 544, 9.600000381, 177},
-        {10.60000038, 80, 205, 9.100000381, 127},
-        {9.699999809, 130, 1264, 9.199999809, 179},
-        {11.60000038, 140, 688, 8.300000191, 80},
-        {8.100000381, 154, 354, 8.399999619, 103},
-        {9.800000191, 118, 1632, 9.399999619, 101},
-        {7.400000095, 94, 348, 9.800000191, 117},
-        {9.399999619, 119, 370, 10.39999962, 88},
-        {11.19999981, 153, 648, 9.899999619, 78},
-        {9.100000381, 116, 366, 9.199999809, 102},
-        {10.5, 97, 540, 10.30000019, 95},
-        {11.89999962, 176, 680, 8.899999619, 80},
-        {8.399999619, 75, 345, 9.600000381, 92},
-        {5, 134, 525, 10.30000019, 126},
-        {9.800000191, 161, 870, 10.39999962, 108},
-        {9.800000191, 111, 669, 9.699999809, 77},
-        {10.80000019, 114, 452, 9.600000381, 60},
-        {10.10000038, 142, 430, 10.69999981, 71},
-        {10.89999962, 238, 822, 10.30000019, 86},
-        {9.199999809, 78, 190, 10.69999981, 93},
-        {8.300000191, 196, 867, 9.600000381, 106},
-        {7.300000191, 125, 969, 10.5, 162},
-        {9.399999619, 82, 499, 7.699999809, 95},
-        {9.399999619, 125, 925, 10.19999981, 91},
-        {9.800000191, 129, 353, 9.899999619, 52},
-        {3.599999905, 84, 288, 8.399999619, 110},
-        {8.399999619, 183, 718, 10.39999962, 69},
-        {10.80000019, 119, 540, 9.199999809, 57},
-        {10.10000038, 180, 668, 13, 106},
-        {9, 82, 347, 8.800000191, 40},
-        {10, 71, 345, 9.199999809, 50},
-        {11.30000019, 118, 463, 7.800000191, 35},
-        {11.30000019, 121, 728, 8.199999809, 86},
-        {12.80000019, 68, 383, 7.400000095, 57},
-        {10, 112, 316, 10.39999962, 57},
-        {6.699999809, 109, 388, 8.899999619, 94}
-    };
-
-    /** Run example. */
-    public static void main(String[] args) throws InterruptedException {
-        System.out.println();
-        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
-        // Start ignite grid.
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println(">>> Ignite grid started.");
-
-            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
-            // because we create ignite cache internally.
-            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                DistributedLinearRegressionWithLSQRTrainerExample.class.getSimpleName(), () -> {
-                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
-
-                System.out.println(">>> Create new linear regression trainer object.");
-                LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
-
-                System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel mdl = trainer.fit(
-                    ignite,
-                    dataCache,
-                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
-                    (k, v) -> v[0]
-                );
-
-                System.out.println(">>> Linear regression model: " + mdl);
-
-                System.out.println(">>> ---------------------------------");
-                System.out.println(">>> | Prediction\t| Ground Truth\t|");
-                System.out.println(">>> ---------------------------------");
-
-                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
-                    for (Cache.Entry<Integer, double[]> observation : observations) {
-                        double[] val = observation.getValue();
-                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
-                        double groundTruth = val[0];
-
-                        double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs));
-
-                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
-                    }
-                }
-
-                System.out.println(">>> ---------------------------------");
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-
-    /**
-     * Fills cache with data and returns it.
-     *
-     * @param ignite Ignite instance.
-     * @return Filled Ignite Cache.
-     */
-    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
-        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
-        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
-        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
-
-        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
-
-        for (int i = 0; i < data.length; i++)
-            cache.put(i, data[i]);
-
-        return cache;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
deleted file mode 100644
index 98d5e4e..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
+++ /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.
- */
-
-package org.apache.ignite.examples.ml.regression.linear;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
-import org.apache.ignite.thread.IgniteThread;
-
-import java.util.Arrays;
-
-/**
- * Run linear regression model over distributed matrix.
- *
- * @see LinearRegressionQRTrainer
- */
-public class DistributedLinearRegressionWithQRTrainerExample {
-    /** */
-    private static final double[][] data = {
-        {8, 78, 284, 9.100000381, 109},
-        {9.300000191, 68, 433, 8.699999809, 144},
-        {7.5, 70, 739, 7.199999809, 113},
-        {8.899999619, 96, 1792, 8.899999619, 97},
-        {10.19999981, 74, 477, 8.300000191, 206},
-        {8.300000191, 111, 362, 10.89999962, 124},
-        {8.800000191, 77, 671, 10, 152},
-        {8.800000191, 168, 636, 9.100000381, 162},
-        {10.69999981, 82, 329, 8.699999809, 150},
-        {11.69999981, 89, 634, 7.599999905, 134},
-        {8.5, 149, 631, 10.80000019, 292},
-        {8.300000191, 60, 257, 9.5, 108},
-        {8.199999809, 96, 284, 8.800000191, 111},
-        {7.900000095, 83, 603, 9.5, 182},
-        {10.30000019, 130, 686, 8.699999809, 129},
-        {7.400000095, 145, 345, 11.19999981, 158},
-        {9.600000381, 112, 1357, 9.699999809, 186},
-        {9.300000191, 131, 544, 9.600000381, 177},
-        {10.60000038, 80, 205, 9.100000381, 127},
-        {9.699999809, 130, 1264, 9.199999809, 179},
-        {11.60000038, 140, 688, 8.300000191, 80},
-        {8.100000381, 154, 354, 8.399999619, 103},
-        {9.800000191, 118, 1632, 9.399999619, 101},
-        {7.400000095, 94, 348, 9.800000191, 117},
-        {9.399999619, 119, 370, 10.39999962, 88},
-        {11.19999981, 153, 648, 9.899999619, 78},
-        {9.100000381, 116, 366, 9.199999809, 102},
-        {10.5, 97, 540, 10.30000019, 95},
-        {11.89999962, 176, 680, 8.899999619, 80},
-        {8.399999619, 75, 345, 9.600000381, 92},
-        {5, 134, 525, 10.30000019, 126},
-        {9.800000191, 161, 870, 10.39999962, 108},
-        {9.800000191, 111, 669, 9.699999809, 77},
-        {10.80000019, 114, 452, 9.600000381, 60},
-        {10.10000038, 142, 430, 10.69999981, 71},
-        {10.89999962, 238, 822, 10.30000019, 86},
-        {9.199999809, 78, 190, 10.69999981, 93},
-        {8.300000191, 196, 867, 9.600000381, 106},
-        {7.300000191, 125, 969, 10.5, 162},
-        {9.399999619, 82, 499, 7.699999809, 95},
-        {9.399999619, 125, 925, 10.19999981, 91},
-        {9.800000191, 129, 353, 9.899999619, 52},
-        {3.599999905, 84, 288, 8.399999619, 110},
-        {8.399999619, 183, 718, 10.39999962, 69},
-        {10.80000019, 119, 540, 9.199999809, 57},
-        {10.10000038, 180, 668, 13, 106},
-        {9, 82, 347, 8.800000191, 40},
-        {10, 71, 345, 9.199999809, 50},
-        {11.30000019, 118, 463, 7.800000191, 35},
-        {11.30000019, 121, 728, 8.199999809, 86},
-        {12.80000019, 68, 383, 7.400000095, 57},
-        {10, 112, 316, 10.39999962, 57},
-        {6.699999809, 109, 388, 8.899999619, 94}
-    };
-
-    /** Run example. */
-    public static void main(String[] args) throws InterruptedException {
-        System.out.println();
-        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
-        // Start ignite grid.
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println(">>> Ignite grid started.");
-            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
-            // because we create ignite cache internally.
-            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                SparseDistributedMatrixExample.class.getSimpleName(), () -> {
-
-                // Create SparseDistributedMatrix, new cache will be created automagically.
-                System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread.");
-                SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(data);
-
-                System.out.println(">>> Create new linear regression trainer object.");
-                Trainer<LinearRegressionModel, Matrix> trainer = new LinearRegressionQRTrainer();
-
-                System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel mdl = trainer.train(distributedMatrix);
-                System.out.println(">>> Linear regression model: " + mdl);
-
-                System.out.println(">>> ---------------------------------");
-                System.out.println(">>> | Prediction\t| Ground Truth\t|");
-                System.out.println(">>> ---------------------------------");
-                for (double[] observation : data) {
-                    Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length));
-                    double prediction = mdl.apply(inputs);
-                    double groundTruth = observation[0];
-                    System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
-                }
-                System.out.println(">>> ---------------------------------");
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
deleted file mode 100644
index 44366e1..0000000
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.examples.ml.regression.linear;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
-import org.apache.ignite.cache.query.QueryCursor;
-import org.apache.ignite.cache.query.ScanQuery;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
-import org.apache.ignite.thread.IgniteThread;
-
-import javax.cache.Cache;
-import java.util.Arrays;
-import java.util.UUID;
-
-/**
- * Run linear regression model over distributed matrix.
- *
- * @see LinearRegressionQRTrainer
- */
-public class DistributedLinearRegressionWithSGDTrainerExample {
-    /** */
-    private static final double[][] data = {
-        {8, 78, 284, 9.100000381, 109},
-        {9.300000191, 68, 433, 8.699999809, 144},
-        {7.5, 70, 739, 7.199999809, 113},
-        {8.899999619, 96, 1792, 8.899999619, 97},
-        {10.19999981, 74, 477, 8.300000191, 206},
-        {8.300000191, 111, 362, 10.89999962, 124},
-        {8.800000191, 77, 671, 10, 152},
-        {8.800000191, 168, 636, 9.100000381, 162},
-        {10.69999981, 82, 329, 8.699999809, 150},
-        {11.69999981, 89, 634, 7.599999905, 134},
-        {8.5, 149, 631, 10.80000019, 292},
-        {8.300000191, 60, 257, 9.5, 108},
-        {8.199999809, 96, 284, 8.800000191, 111},
-        {7.900000095, 83, 603, 9.5, 182},
-        {10.30000019, 130, 686, 8.699999809, 129},
-        {7.400000095, 145, 345, 11.19999981, 158},
-        {9.600000381, 112, 1357, 9.699999809, 186},
-        {9.300000191, 131, 544, 9.600000381, 177},
-        {10.60000038, 80, 205, 9.100000381, 127},
-        {9.699999809, 130, 1264, 9.199999809, 179},
-        {11.60000038, 140, 688, 8.300000191, 80},
-        {8.100000381, 154, 354, 8.399999619, 103},
-        {9.800000191, 118, 1632, 9.399999619, 101},
-        {7.400000095, 94, 348, 9.800000191, 117},
-        {9.399999619, 119, 370, 10.39999962, 88},
-        {11.19999981, 153, 648, 9.899999619, 78},
-        {9.100000381, 116, 366, 9.199999809, 102},
-        {10.5, 97, 540, 10.30000019, 95},
-        {11.89999962, 176, 680, 8.899999619, 80},
-        {8.399999619, 75, 345, 9.600000381, 92},
-        {5, 134, 525, 10.30000019, 126},
-        {9.800000191, 161, 870, 10.39999962, 108},
-        {9.800000191, 111, 669, 9.699999809, 77},
-        {10.80000019, 114, 452, 9.600000381, 60},
-        {10.10000038, 142, 430, 10.69999981, 71},
-        {10.89999962, 238, 822, 10.30000019, 86},
-        {9.199999809, 78, 190, 10.69999981, 93},
-        {8.300000191, 196, 867, 9.600000381, 106},
-        {7.300000191, 125, 969, 10.5, 162},
-        {9.399999619, 82, 499, 7.699999809, 95},
-        {9.399999619, 125, 925, 10.19999981, 91},
-        {9.800000191, 129, 353, 9.899999619, 52},
-        {3.599999905, 84, 288, 8.399999619, 110},
-        {8.399999619, 183, 718, 10.39999962, 69},
-        {10.80000019, 119, 540, 9.199999809, 57},
-        {10.10000038, 180, 668, 13, 106},
-        {9, 82, 347, 8.800000191, 40},
-        {10, 71, 345, 9.199999809, 50},
-        {11.30000019, 118, 463, 7.800000191, 35},
-        {11.30000019, 121, 728, 8.199999809, 86},
-        {12.80000019, 68, 383, 7.400000095, 57},
-        {10, 112, 316, 10.39999962, 57},
-        {6.699999809, 109, 388, 8.899999619, 94}
-    };
-
-    /** Run example. */
-    public static void main(String[] args) throws InterruptedException {
-        System.out.println();
-        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
-        // Start ignite grid.
-        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
-            System.out.println(">>> Ignite grid started.");
-            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
-            // because we create ignite cache internally.
-            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                DistributedLinearRegressionWithSGDTrainerExample.class.getSimpleName(), () -> {
-
-                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
-
-                System.out.println(">>> Create new linear regression trainer object.");
-                LinearRegressionSGDTrainer<?> trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>(
-                    new RPropUpdateCalculator(),
-                    RPropParameterUpdate::sumLocal,
-                    RPropParameterUpdate::avg
-                ), 100000,  10, 100, 123L);
-
-                System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel mdl = trainer.fit(
-                    ignite,
-                    dataCache,
-                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
-                    (k, v) -> v[0]
-                );
-
-                System.out.println(">>> Linear regression model: " + mdl);
-
-                System.out.println(">>> ---------------------------------");
-                System.out.println(">>> | Prediction\t| Ground Truth\t|");
-                System.out.println(">>> ---------------------------------");
-
-                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
-                    for (Cache.Entry<Integer, double[]> observation : observations) {
-                        double[] val = observation.getValue();
-                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
-                        double groundTruth = val[0];
-
-                        double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs));
-
-                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
-                    }
-                }
-
-                System.out.println(">>> ---------------------------------");
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-
-    /**
-     * Fills cache with data and returns it.
-     *
-     * @param ignite Ignite instance.
-     * @return Filled Ignite Cache.
-     */
-    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
-        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
-        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
-        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
-
-        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
-
-        for (int i = 0; i < data.length; i++)
-            cache.put(i, data[i]);
-
-        return cache;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java
new file mode 100644
index 0000000..276d43f
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerExample.java
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.ml.regression.linear;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
+import org.apache.ignite.thread.IgniteThread;
+
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
+/**
+ * Run linear regression model over distributed matrix.
+ *
+ * @see LinearRegressionLSQRTrainer
+ */
+public class LinearRegressionLSQRTrainerExample {
+    /** */
+    private static final double[][] data = {
+        {8, 78, 284, 9.100000381, 109},
+        {9.300000191, 68, 433, 8.699999809, 144},
+        {7.5, 70, 739, 7.199999809, 113},
+        {8.899999619, 96, 1792, 8.899999619, 97},
+        {10.19999981, 74, 477, 8.300000191, 206},
+        {8.300000191, 111, 362, 10.89999962, 124},
+        {8.800000191, 77, 671, 10, 152},
+        {8.800000191, 168, 636, 9.100000381, 162},
+        {10.69999981, 82, 329, 8.699999809, 150},
+        {11.69999981, 89, 634, 7.599999905, 134},
+        {8.5, 149, 631, 10.80000019, 292},
+        {8.300000191, 60, 257, 9.5, 108},
+        {8.199999809, 96, 284, 8.800000191, 111},
+        {7.900000095, 83, 603, 9.5, 182},
+        {10.30000019, 130, 686, 8.699999809, 129},
+        {7.400000095, 145, 345, 11.19999981, 158},
+        {9.600000381, 112, 1357, 9.699999809, 186},
+        {9.300000191, 131, 544, 9.600000381, 177},
+        {10.60000038, 80, 205, 9.100000381, 127},
+        {9.699999809, 130, 1264, 9.199999809, 179},
+        {11.60000038, 140, 688, 8.300000191, 80},
+        {8.100000381, 154, 354, 8.399999619, 103},
+        {9.800000191, 118, 1632, 9.399999619, 101},
+        {7.400000095, 94, 348, 9.800000191, 117},
+        {9.399999619, 119, 370, 10.39999962, 88},
+        {11.19999981, 153, 648, 9.899999619, 78},
+        {9.100000381, 116, 366, 9.199999809, 102},
+        {10.5, 97, 540, 10.30000019, 95},
+        {11.89999962, 176, 680, 8.899999619, 80},
+        {8.399999619, 75, 345, 9.600000381, 92},
+        {5, 134, 525, 10.30000019, 126},
+        {9.800000191, 161, 870, 10.39999962, 108},
+        {9.800000191, 111, 669, 9.699999809, 77},
+        {10.80000019, 114, 452, 9.600000381, 60},
+        {10.10000038, 142, 430, 10.69999981, 71},
+        {10.89999962, 238, 822, 10.30000019, 86},
+        {9.199999809, 78, 190, 10.69999981, 93},
+        {8.300000191, 196, 867, 9.600000381, 106},
+        {7.300000191, 125, 969, 10.5, 162},
+        {9.399999619, 82, 499, 7.699999809, 95},
+        {9.399999619, 125, 925, 10.19999981, 91},
+        {9.800000191, 129, 353, 9.899999619, 52},
+        {3.599999905, 84, 288, 8.399999619, 110},
+        {8.399999619, 183, 718, 10.39999962, 69},
+        {10.80000019, 119, 540, 9.199999809, 57},
+        {10.10000038, 180, 668, 13, 106},
+        {9, 82, 347, 8.800000191, 40},
+        {10, 71, 345, 9.199999809, 50},
+        {11.30000019, 118, 463, 7.800000191, 35},
+        {11.30000019, 121, 728, 8.199999809, 86},
+        {12.80000019, 68, 383, 7.400000095, 57},
+        {10, 112, 316, 10.39999962, 57},
+        {6.699999809, 109, 388, 8.899999619, 94}
+    };
+
+    /** Run example. */
+    public static void main(String[] args) throws InterruptedException {
+        System.out.println();
+        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
+        // Start ignite grid.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Ignite grid started.");
+
+            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
+            // because we create ignite cache internally.
+            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
+                LinearRegressionLSQRTrainerExample.class.getSimpleName(), () -> {
+                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
+
+                System.out.println(">>> Create new linear regression trainer object.");
+                LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
+
+                System.out.println(">>> Perform the training to get the model.");
+                LinearRegressionModel mdl = trainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+                    (k, v) -> v[0]
+                );
+
+                System.out.println(">>> Linear regression model: " + mdl);
+
+                System.out.println(">>> ---------------------------------");
+                System.out.println(">>> | Prediction\t| Ground Truth\t|");
+                System.out.println(">>> ---------------------------------");
+
+                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
+                    for (Cache.Entry<Integer, double[]> observation : observations) {
+                        double[] val = observation.getValue();
+                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
+                        double groundTruth = val[0];
+
+                        double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs));
+
+                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
+                    }
+                }
+
+                System.out.println(">>> ---------------------------------");
+            });
+
+            igniteThread.start();
+
+            igniteThread.join();
+        }
+    }
+
+    /**
+     * Fills cache with data and returns it.
+     *
+     * @param ignite Ignite instance.
+     * @return Filled Ignite Cache.
+     */
+    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
+        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
+        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
+
+        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
+
+        for (int i = 0; i < data.length; i++)
+            cache.put(i, data[i]);
+
+        return cache;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java
new file mode 100644
index 0000000..0358f44
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionLSQRTrainerWithNormalizationExample.java
@@ -0,0 +1,180 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.ml.regression.linear;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor;
+import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
+import org.apache.ignite.thread.IgniteThread;
+
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
+/**
+ * Run linear regression model over distributed matrix.
+ *
+ * @see LinearRegressionLSQRTrainer
+ * @see NormalizationTrainer
+ * @see NormalizationPreprocessor
+ */
+public class LinearRegressionLSQRTrainerWithNormalizationExample {
+    /** */
+    private static final double[][] data = {
+        {8, 78, 284, 9.100000381, 109},
+        {9.300000191, 68, 433, 8.699999809, 144},
+        {7.5, 70, 739, 7.199999809, 113},
+        {8.899999619, 96, 1792, 8.899999619, 97},
+        {10.19999981, 74, 477, 8.300000191, 206},
+        {8.300000191, 111, 362, 10.89999962, 124},
+        {8.800000191, 77, 671, 10, 152},
+        {8.800000191, 168, 636, 9.100000381, 162},
+        {10.69999981, 82, 329, 8.699999809, 150},
+        {11.69999981, 89, 634, 7.599999905, 134},
+        {8.5, 149, 631, 10.80000019, 292},
+        {8.300000191, 60, 257, 9.5, 108},
+        {8.199999809, 96, 284, 8.800000191, 111},
+        {7.900000095, 83, 603, 9.5, 182},
+        {10.30000019, 130, 686, 8.699999809, 129},
+        {7.400000095, 145, 345, 11.19999981, 158},
+        {9.600000381, 112, 1357, 9.699999809, 186},
+        {9.300000191, 131, 544, 9.600000381, 177},
+        {10.60000038, 80, 205, 9.100000381, 127},
+        {9.699999809, 130, 1264, 9.199999809, 179},
+        {11.60000038, 140, 688, 8.300000191, 80},
+        {8.100000381, 154, 354, 8.399999619, 103},
+        {9.800000191, 118, 1632, 9.399999619, 101},
+        {7.400000095, 94, 348, 9.800000191, 117},
+        {9.399999619, 119, 370, 10.39999962, 88},
+        {11.19999981, 153, 648, 9.899999619, 78},
+        {9.100000381, 116, 366, 9.199999809, 102},
+        {10.5, 97, 540, 10.30000019, 95},
+        {11.89999962, 176, 680, 8.899999619, 80},
+        {8.399999619, 75, 345, 9.600000381, 92},
+        {5, 134, 525, 10.30000019, 126},
+        {9.800000191, 161, 870, 10.39999962, 108},
+        {9.800000191, 111, 669, 9.699999809, 77},
+        {10.80000019, 114, 452, 9.600000381, 60},
+        {10.10000038, 142, 430, 10.69999981, 71},
+        {10.89999962, 238, 822, 10.30000019, 86},
+        {9.199999809, 78, 190, 10.69999981, 93},
+        {8.300000191, 196, 867, 9.600000381, 106},
+        {7.300000191, 125, 969, 10.5, 162},
+        {9.399999619, 82, 499, 7.699999809, 95},
+        {9.399999619, 125, 925, 10.19999981, 91},
+        {9.800000191, 129, 353, 9.899999619, 52},
+        {3.599999905, 84, 288, 8.399999619, 110},
+        {8.399999619, 183, 718, 10.39999962, 69},
+        {10.80000019, 119, 540, 9.199999809, 57},
+        {10.10000038, 180, 668, 13, 106},
+        {9, 82, 347, 8.800000191, 40},
+        {10, 71, 345, 9.199999809, 50},
+        {11.30000019, 118, 463, 7.800000191, 35},
+        {11.30000019, 121, 728, 8.199999809, 86},
+        {12.80000019, 68, 383, 7.400000095, 57},
+        {10, 112, 316, 10.39999962, 57},
+        {6.699999809, 109, 388, 8.899999619, 94}
+    };
+
+    /** Run example. */
+    public static void main(String[] args) throws InterruptedException {
+        System.out.println();
+        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
+        // Start ignite grid.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Ignite grid started.");
+
+            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
+            // because we create ignite cache internally.
+            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
+                SparseDistributedMatrixExample.class.getSimpleName(), () -> {
+                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
+
+                System.out.println(">>> Create new normalization trainer object.");
+                NormalizationTrainer<Integer, double[]> normalizationTrainer = new NormalizationTrainer<>();
+
+                System.out.println(">>> Perform the training to get the normalization preprocessor.");
+                IgniteBiFunction<Integer, double[], double[]> preprocessor = normalizationTrainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length)
+                );
+
+                System.out.println(">>> Create new linear regression trainer object.");
+                LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
+
+                System.out.println(">>> Perform the training to get the model.");
+                LinearRegressionModel mdl = trainer.fit(ignite, dataCache, preprocessor, (k, v) -> v[0]);
+
+                System.out.println(">>> Linear regression model: " + mdl);
+
+                System.out.println(">>> ---------------------------------");
+                System.out.println(">>> | Prediction\t| Ground Truth\t|");
+                System.out.println(">>> ---------------------------------");
+
+                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
+                    for (Cache.Entry<Integer, double[]> observation : observations) {
+                        Integer key = observation.getKey();
+                        double[] val = observation.getValue();
+                        double groundTruth = val[0];
+
+                        double prediction = mdl.apply(new DenseLocalOnHeapVector(preprocessor.apply(key, val)));
+
+                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
+                    }
+                }
+
+                System.out.println(">>> ---------------------------------");
+            });
+
+            igniteThread.start();
+
+            igniteThread.join();
+        }
+    }
+
+    /**
+     * Fills cache with data and returns it.
+     *
+     * @param ignite Ignite instance.
+     * @return Filled Ignite Cache.
+     */
+    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
+        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
+        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
+
+        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
+
+        for (int i = 0; i < data.length; i++)
+            cache.put(i, data[i]);
+
+        return cache;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java
new file mode 100644
index 0000000..ce6ad3b
--- /dev/null
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/LinearRegressionSGDTrainerExample.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.examples.ml.regression.linear;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
+import org.apache.ignite.thread.IgniteThread;
+
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
+/**
+ * Run linear regression model over distributed matrix.
+ *
+ * @see LinearRegressionSGDTrainer
+ */
+public class LinearRegressionSGDTrainerExample {
+    /** */
+    private static final double[][] data = {
+        {8, 78, 284, 9.100000381, 109},
+        {9.300000191, 68, 433, 8.699999809, 144},
+        {7.5, 70, 739, 7.199999809, 113},
+        {8.899999619, 96, 1792, 8.899999619, 97},
+        {10.19999981, 74, 477, 8.300000191, 206},
+        {8.300000191, 111, 362, 10.89999962, 124},
+        {8.800000191, 77, 671, 10, 152},
+        {8.800000191, 168, 636, 9.100000381, 162},
+        {10.69999981, 82, 329, 8.699999809, 150},
+        {11.69999981, 89, 634, 7.599999905, 134},
+        {8.5, 149, 631, 10.80000019, 292},
+        {8.300000191, 60, 257, 9.5, 108},
+        {8.199999809, 96, 284, 8.800000191, 111},
+        {7.900000095, 83, 603, 9.5, 182},
+        {10.30000019, 130, 686, 8.699999809, 129},
+        {7.400000095, 145, 345, 11.19999981, 158},
+        {9.600000381, 112, 1357, 9.699999809, 186},
+        {9.300000191, 131, 544, 9.600000381, 177},
+        {10.60000038, 80, 205, 9.100000381, 127},
+        {9.699999809, 130, 1264, 9.199999809, 179},
+        {11.60000038, 140, 688, 8.300000191, 80},
+        {8.100000381, 154, 354, 8.399999619, 103},
+        {9.800000191, 118, 1632, 9.399999619, 101},
+        {7.400000095, 94, 348, 9.800000191, 117},
+        {9.399999619, 119, 370, 10.39999962, 88},
+        {11.19999981, 153, 648, 9.899999619, 78},
+        {9.100000381, 116, 366, 9.199999809, 102},
+        {10.5, 97, 540, 10.30000019, 95},
+        {11.89999962, 176, 680, 8.899999619, 80},
+        {8.399999619, 75, 345, 9.600000381, 92},
+        {5, 134, 525, 10.30000019, 126},
+        {9.800000191, 161, 870, 10.39999962, 108},
+        {9.800000191, 111, 669, 9.699999809, 77},
+        {10.80000019, 114, 452, 9.600000381, 60},
+        {10.10000038, 142, 430, 10.69999981, 71},
+        {10.89999962, 238, 822, 10.30000019, 86},
+        {9.199999809, 78, 190, 10.69999981, 93},
+        {8.300000191, 196, 867, 9.600000381, 106},
+        {7.300000191, 125, 969, 10.5, 162},
+        {9.399999619, 82, 499, 7.699999809, 95},
+        {9.399999619, 125, 925, 10.19999981, 91},
+        {9.800000191, 129, 353, 9.899999619, 52},
+        {3.599999905, 84, 288, 8.399999619, 110},
+        {8.399999619, 183, 718, 10.39999962, 69},
+        {10.80000019, 119, 540, 9.199999809, 57},
+        {10.10000038, 180, 668, 13, 106},
+        {9, 82, 347, 8.800000191, 40},
+        {10, 71, 345, 9.199999809, 50},
+        {11.30000019, 118, 463, 7.800000191, 35},
+        {11.30000019, 121, 728, 8.199999809, 86},
+        {12.80000019, 68, 383, 7.400000095, 57},
+        {10, 112, 316, 10.39999962, 57},
+        {6.699999809, 109, 388, 8.899999619, 94}
+    };
+
+    /** Run example. */
+    public static void main(String[] args) throws InterruptedException {
+        System.out.println();
+        System.out.println(">>> Linear regression model over sparse distributed matrix API usage example started.");
+        // Start ignite grid.
+        try (Ignite ignite = Ignition.start("examples/config/example-ignite.xml")) {
+            System.out.println(">>> Ignite grid started.");
+            // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
+            // because we create ignite cache internally.
+            IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
+                LinearRegressionSGDTrainerExample.class.getSimpleName(), () -> {
+
+                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
+
+                System.out.println(">>> Create new linear regression trainer object.");
+                LinearRegressionSGDTrainer<?> trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>(
+                    new RPropUpdateCalculator(),
+                    RPropParameterUpdate::sumLocal,
+                    RPropParameterUpdate::avg
+                ), 100000,  10, 100, 123L);
+
+                System.out.println(">>> Perform the training to get the model.");
+                LinearRegressionModel mdl = trainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+                    (k, v) -> v[0]
+                );
+
+                System.out.println(">>> Linear regression model: " + mdl);
+
+                System.out.println(">>> ---------------------------------");
+                System.out.println(">>> | Prediction\t| Ground Truth\t|");
+                System.out.println(">>> ---------------------------------");
+
+                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
+                    for (Cache.Entry<Integer, double[]> observation : observations) {
+                        double[] val = observation.getValue();
+                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
+                        double groundTruth = val[0];
+
+                        double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs));
+
+                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
+                    }
+                }
+
+                System.out.println(">>> ---------------------------------");
+            });
+
+            igniteThread.start();
+
+            igniteThread.join();
+        }
+    }
+
+    /**
+     * Fills cache with data and returns it.
+     *
+     * @param ignite Ignite instance.
+     * @return Filled Ignite Cache.
+     */
+    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
+        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
+        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
+
+        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
+
+        for (int i = 0; i < data.length; i++)
+            cache.put(i, data[i]);
+
+        return cache;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java
deleted file mode 100644
index f53b801..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/Trainer.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml;
-
-/**
- * Interface for Trainers. Trainer is just a function which produces model from the data.
- *
- * @param <M> Type of produced model.
- * @param <T> Type of data needed for model producing.
- */
-// TODO: IGNITE-7659: Reduce multiple Trainer interfaces to one
-@Deprecated
-public interface Trainer<M extends Model, T> {
-    /**
-     * Returns model based on data
-     *
-     * @param data data to build model
-     * @return model
-     */
-    M train(T data);
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java b/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java
deleted file mode 100644
index b2731ff..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/estimators/Estimators.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.estimators;
-
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Function;
-import java.util.stream.Stream;
-import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.ml.Model;
-import org.apache.ignite.ml.math.functions.IgniteTriFunction;
-
-/** Estimators. */
-public class Estimators {
-    /** Simple implementation of mean squared error estimator. */
-    public static <T, V> IgniteTriFunction<Model<T, V>, Stream<IgniteBiTuple<T, V>>, Function<V, Double>, Double> MSE() {
-        return (model, stream, f) -> stream.mapToDouble(dp -> {
-            double diff = f.apply(dp.get2()) - f.apply(model.apply(dp.get1()));
-            return diff * diff;
-        }).average().orElse(0);
-    }
-
-    /** Simple implementation of errors percentage estimator. */
-    public static <T, V> IgniteTriFunction<Model<T, V>, Stream<IgniteBiTuple<T, V>>, Function<V, Double>, Double> errorsPercentage() {
-        return (model, stream, f) -> {
-            AtomicLong total = new AtomicLong(0);
-
-            long cnt = stream.
-                peek((ib) -> total.incrementAndGet()).
-                filter(dp -> !model.apply(dp.get1()).equals(dp.get2())).
-                count();
-
-            return (double)cnt / total.get();
-        };
-    }
-}

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
index dc49739..45fd035 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/functions/IgniteBiFunction.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.ml.math.functions;
 
 import java.io.Serializable;
+import java.util.Objects;
 import java.util.function.BiFunction;
 
 /**
@@ -25,5 +26,10 @@ import java.util.function.BiFunction;
  *
  * @see java.util.function.BiFunction
  */
-public interface IgniteBiFunction<A, B, T> extends BiFunction<A, B, T>, Serializable {
+public interface IgniteBiFunction<T, U, R> extends BiFunction<T, U, R>, Serializable {
+    /** {@inheritDoc} */
+    default <V> IgniteBiFunction<T, U, V> andThen(IgniteFunction<? super R, ? extends V> after) {
+        Objects.requireNonNull(after);
+        return (T t, U u) -> after.apply(apply(t, u));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java
deleted file mode 100644
index e80b935..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataBuilderOnHeap.java
+++ /dev/null
@@ -1,86 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.math.isolve;
-
-import java.io.Serializable;
-import java.util.Iterator;
-import org.apache.ignite.ml.dataset.PartitionDataBuilder;
-import org.apache.ignite.ml.dataset.UpstreamEntry;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-
-/**
- * Linear system partition data builder that builds {@link LinSysPartitionDataOnHeap}.
- *
- * @param <K> Type of a key in <tt>upstream</tt> data.
- * @param <V> Type of a value in <tt>upstream</tt> data.
- * @param <C> Type of a partition <tt>context</tt>.
- */
-public class LinSysPartitionDataBuilderOnHeap<K, V, C extends Serializable>
-    implements PartitionDataBuilder<K, V, C, LinSysPartitionDataOnHeap> {
-    /** */
-    private static final long serialVersionUID = -7820760153954269227L;
-
-    /** Extractor of X matrix row. */
-    private final IgniteBiFunction<K, V, double[]> xExtractor;
-
-    /** Extractor of Y vector value. */
-    private final IgniteBiFunction<K, V, Double> yExtractor;
-
-    /**
-     * Constructs a new instance of linear system partition data builder.
-     *
-     * @param xExtractor Extractor of X matrix row.
-     * @param yExtractor Extractor of Y vector value.
-     */
-    public LinSysPartitionDataBuilderOnHeap(IgniteBiFunction<K, V, double[]> xExtractor,
-        IgniteBiFunction<K, V, Double> yExtractor) {
-        this.xExtractor = xExtractor;
-        this.yExtractor = yExtractor;
-    }
-
-    /** {@inheritDoc} */
-    @Override public LinSysPartitionDataOnHeap build(Iterator<UpstreamEntry<K, V>> upstreamData, long upstreamDataSize,
-        C ctx) {
-        // Prepares the matrix of features in flat column-major format.
-        int xCols = -1;
-        double[] x = null;//new double[Math.toIntExact(upstreamDataSize * cols)];
-        double[] y = new double[Math.toIntExact(upstreamDataSize)];
-
-        int ptr = 0;
-        while (upstreamData.hasNext()) {
-            UpstreamEntry<K, V> entry = upstreamData.next();
-            double[] row = xExtractor.apply(entry.getKey(), entry.getValue());
-
-            if (xCols < 0) {
-                xCols = row.length;
-                x = new double[Math.toIntExact(upstreamDataSize * xCols)];
-            }
-            else
-                assert row.length == xCols : "X extractor must return exactly " + xCols + " columns";
-
-            for (int i = 0; i < xCols; i++)
-                x[Math.toIntExact(i * upstreamDataSize) + ptr] = row[i];
-
-            y[ptr] = yExtractor.apply(entry.getKey(), entry.getValue());
-
-            ptr++;
-        }
-
-        return new LinSysPartitionDataOnHeap(x, y, Math.toIntExact(upstreamDataSize));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java
deleted file mode 100644
index 89c8e44..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/LinSysPartitionDataOnHeap.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.math.isolve;
-
-/**
- * On Heap partition data that keeps part of a linear system.
- */
-public class LinSysPartitionDataOnHeap implements AutoCloseable {
-    /** Part of X matrix. */
-    private final double[] x;
-
-    /** Part of Y vector. */
-    private final double[] y;
-
-    /** Number of rows. */
-    private final int rows;
-
-    /**
-     * Constructs a new instance of linear system partition data.
-     *
-     * @param x Part of X matrix.
-     * @param y Part of Y vector.
-     * @param rows Number of rows.
-     */
-    public LinSysPartitionDataOnHeap(double[] x, double[] y, int rows) {
-        this.x = x;
-        this.rows = rows;
-        this.y = y;
-    }
-
-    /** */
-    public double[] getX() {
-        return x;
-    }
-
-    /** */
-    public int getRows() {
-        return rows;
-    }
-
-    /** */
-    public double[] getY() {
-        return y;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        // Do nothing, GC will clean up.
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java
index 8d190cd..d1d3219 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/AbstractLSQR.java
@@ -19,6 +19,7 @@ package org.apache.ignite.ml.math.isolve.lsqr;
 
 import com.github.fommil.netlib.BLAS;
 import java.util.Arrays;
+import org.apache.ignite.ml.math.Precision;
 
 /**
  * Basic implementation of the LSQR algorithm without assumptions about dataset storage format or data processing
@@ -30,7 +31,7 @@ import java.util.Arrays;
 // TODO: IGNITE-7660: Refactor LSQR algorithm
 public abstract class AbstractLSQR {
     /** The smallest representable positive number such that 1.0 + eps != 1.0. */
-    private static final double eps = Double.longBitsToDouble(Double.doubleToLongBits(1.0) | 1) - 1.0;
+    private static final double eps = Precision.EPSILON;
 
     /** BLAS (Basic Linear Algebra Subprograms) instance. */
     private static BLAS blas = BLAS.getInstance();

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java
index b1cc4c9..e138cf3 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/isolve/lsqr/LSQROnHeap.java
@@ -22,14 +22,14 @@ import java.util.Arrays;
 import org.apache.ignite.ml.dataset.Dataset;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.PartitionDataBuilder;
-import org.apache.ignite.ml.math.isolve.LinSysPartitionDataOnHeap;
+import org.apache.ignite.ml.dataset.primitive.data.SimpleLabeledDatasetData;
 
 /**
  * Distributed implementation of LSQR algorithm based on {@link AbstractLSQR} and {@link Dataset}.
  */
 public class LSQROnHeap<K, V> extends AbstractLSQR implements AutoCloseable {
     /** Dataset. */
-    private final Dataset<LSQRPartitionContext, LinSysPartitionDataOnHeap> dataset;
+    private final Dataset<LSQRPartitionContext, SimpleLabeledDatasetData> dataset;
 
     /**
      * Constructs a new instance of OnHeap LSQR algorithm implementation.
@@ -38,7 +38,7 @@ public class LSQROnHeap<K, V> extends AbstractLSQR implements AutoCloseable {
      * @param partDataBuilder Partition data builder.
      */
     public LSQROnHeap(DatasetBuilder<K, V> datasetBuilder,
-        PartitionDataBuilder<K, V, LSQRPartitionContext, LinSysPartitionDataOnHeap> partDataBuilder) {
+        PartitionDataBuilder<K, V, LSQRPartitionContext, SimpleLabeledDatasetData> partDataBuilder) {
         this.dataset = datasetBuilder.build(
             (upstream, upstreamSize) -> new LSQRPartitionContext(),
             partDataBuilder
@@ -48,20 +48,20 @@ public class LSQROnHeap<K, V> extends AbstractLSQR implements AutoCloseable {
     /** {@inheritDoc} */
     @Override protected double bnorm() {
         return dataset.computeWithCtx((ctx, data) -> {
-            ctx.setU(Arrays.copyOf(data.getY(), data.getY().length));
+            ctx.setU(Arrays.copyOf(data.getLabels(), data.getLabels().length));
 
-            return BLAS.getInstance().dnrm2(data.getY().length, data.getY(), 1);
+            return BLAS.getInstance().dnrm2(data.getLabels().length, data.getLabels(), 1);
         }, (a, b) -> a == null ? b : b == null ? a : Math.sqrt(a * a + b * b));
     }
 
     /** {@inheritDoc} */
     @Override protected double beta(double[] x, double alfa, double beta) {
         return dataset.computeWithCtx((ctx, data) -> {
-            if (data.getX() == null)
+            if (data.getFeatures() == null)
                 return null;
 
-            int cols = data.getX().length / data.getRows();
-            BLAS.getInstance().dgemv("N", data.getRows(), cols, alfa, data.getX(),
+            int cols = data.getFeatures().length / data.getRows();
+            BLAS.getInstance().dgemv("N", data.getRows(), cols, alfa, data.getFeatures(),
                 Math.max(1, data.getRows()), x, 1, beta, ctx.getU(), 1);
 
             return BLAS.getInstance().dnrm2(ctx.getU().length, ctx.getU(), 1);
@@ -71,13 +71,13 @@ public class LSQROnHeap<K, V> extends AbstractLSQR implements AutoCloseable {
     /** {@inheritDoc} */
     @Override protected double[] iter(double bnorm, double[] target) {
         double[] res = dataset.computeWithCtx((ctx, data) -> {
-            if (data.getX() == null)
+            if (data.getFeatures() == null)
                 return null;
 
-            int cols =  data.getX().length / data.getRows();
+            int cols =  data.getFeatures().length / data.getRows();
             BLAS.getInstance().dscal(ctx.getU().length, 1 / bnorm, ctx.getU(), 1);
             double[] v = new double[cols];
-            BLAS.getInstance().dgemv("T", data.getRows(), cols, 1.0, data.getX(),
+            BLAS.getInstance().dgemv("T", data.getRows(), cols, 1.0, data.getFeatures(),
                 Math.max(1, data.getRows()), ctx.getU(), 1, 0, v, 1);
 
             return v;
@@ -101,7 +101,10 @@ public class LSQROnHeap<K, V> extends AbstractLSQR implements AutoCloseable {
      * @return number of columns
      */
     @Override protected int getColumns() {
-        return dataset.compute(data -> data.getX() == null ? null :  data.getX().length / data.getRows(), (a, b) -> a == null ? b : a);
+        return dataset.compute(
+            data -> data.getFeatures() == null ? null : data.getFeatures().length / data.getRows(),
+            (a, b) -> a == null ? b : a
+        );
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
index fe955cb..d12a276 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
@@ -33,7 +33,6 @@ import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.nn.initializers.RandomInitializer;
 import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator;
 import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.ml.util.Utils;
 
 import java.io.Serializable;


[39/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java
deleted file mode 100644
index ed6bf36..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java
+++ /dev/null
@@ -1,404 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-/**
- * Artificial regression datasets to be used in regression trainers tests. These datasets were generated by scikit-learn
- * tools, {@code sklearn.datasets.make_regression} procedure.
- */
-public class ArtificialRegressionDatasets {
-    /**
-     * Artificial dataset with 10 observations described by 1 feature.
-     */
-    public static final TestDataset regression10x1 = new TestDataset(new double[][] {
-        {1.97657990214, 0.197725444973},
-        {-5.0835948878, -0.279921224228},
-        {-5.09032600779, -0.352291245969},
-        {9.67660993007, 0.755464872441},
-        {4.95927629958, 0.451981771462},
-        {29.2635107429, 2.2277440173},
-        {-18.3122588459, -1.25363275369},
-        {-3.61729307199, -0.273362913982},
-        {-7.19042139249, -0.473846634967},
-        {3.68008403347, 0.353883097536}
-    }, new double[] {13.554054703}, -0.808655936776);
-
-    /**
-     * Artificial dataset with 10 observations described by 5 features.
-     */
-    public static final TestDataset regression10x5 = new TestDataset(new double[][] {
-        {118.635647237, 0.687593385888, -1.18956185502, -0.305420702986, 1.98794097418, -0.776629036361},
-        {-18.2808432286, -0.165921853684, -0.156162539573, 1.56284391134, -0.198876782109, -0.0921618505605},
-        {22.6110523992, 0.0268106268606, 0.702141470035, -0.41503615392, -1.09726502337, 1.30830482813},
-        {209.820435262, 0.379809113402, -0.192097238579, -1.27460497119, 2.48052002019, -0.574430888865},
-        {-253.750024054, -1.48044570917, -0.331747484523, 0.387993627712, 0.372583756237, -2.27404065923},
-        {-24.6467766166, -0.66991474156, 0.269042238935, -0.271412703096, -0.561166818525, 1.37067541854},
-        {-311.903650717, 0.268274438122, -1.10491275353, -1.06738703543, -2.24387799735, -0.207431467989},
-        {74.2055323536, -0.329489531894, -0.493350762533, -0.644851462227, 0.661220945573, 1.65950140864},
-        {57.0312289904, -1.07266578457, 0.80375035572, -0.45207210139, 1.69314420969, -1.10526080856},
-        {12.149399645, 1.46504629281, -1.05843246079, 0.266225365277, -0.0113100353869, -0.983495425471}
-    }, new double[] {99.8393653561, 82.4948224094, 20.2087724072, 97.3306384162, 55.7502297387}, 3.98444039189);
-
-    /**
-     * Artificial dataset with 100 observations described by 5 features.
-     */
-    public static final TestDataset regression100x5 = new TestDataset(new double[][] {
-        {-44.2310642946, -0.0331360137605, -0.5290800706, -0.634340342338, -0.428433927151, 0.830582347183},
-        {76.2539139721, -0.216200869652, 0.513212019048, -0.693404511747, 0.132995973133, 1.28470259833},
-        {293.369799914, 2.90735870802, 0.457740818846, -0.490470696097, -0.442343455187, 0.584038258781},
-        {124.258807314, 1.64158129148, 0.0616936820145, 1.24082841519, -1.20126518593, -0.542298907742},
-        {13.6610807249, -1.10834821778, 0.545508208111, 1.81361288715, -0.786543112444, 0.250772626496},
-        {101.924582305, -0.433526394969, 0.257594734335, 1.22333193911, 0.76626554927, -0.0400734567005},
-        {25.5963186303, -0.202003301507, 0.717101151637, -0.486881225605, 1.15215024807, -0.921615554612},
-        {75.7959681263, -0.604173187402, 0.0364386836472, 1.67544714536, 0.394743148877, 0.0237966550759},
-        {-97.539357166, -0.774517689169, -0.0966902473883, -0.152250704254, -0.325472625458, 0.0720711851256},
-        {0.394748999236, -0.559303402754, -0.0493339259273, -1.10840277768, -0.0800969523557, 1.80939282066},
-        {-62.0138166431, 0.062614716778, -0.844143618016, 0.55269949861, -2.32580899335, 1.58020577369},
-        {584.427692931, 2.13184767906, 1.22222461994, 1.71894070494, 2.69512281718, 0.294123497874},
-        {-59.8323709765, 1.00006112818, -1.54481230765, -0.781282316493, 0.0255925284853, -0.0821173744608},
-        {101.565711925, -0.38699836725, 1.06934591441, -0.260429311097, 1.02628949564, 0.0431473245174},
-        {-141.592607814, 0.993279116267, -0.371768203378, -0.851483217286, -1.96241293548, -0.612279404296},
-        {34.8038723379, -0.0182719243972, 0.306367604506, -0.650526589206, 1.30693112283, -0.587465952557},
-        {-16.9554534069, -0.703006786668, -0.770718401931, 0.748423272307, 0.502544067819, 0.346625621533},
-        {-76.2896177709, -0.16440174812, -1.77431555198, 0.195326723837, 2.01240994405, -1.19559207119},
-        {-3.23827624818, -0.674138419631, -1.62238580284, 2.02235607862, 0.679194838679, 0.150203732584},
-        {-21.962456854, -0.766271014206, 0.958599712131, -0.313045794728, 0.232655576106, -0.360950549871},
-        {349.583669646, 1.75976166947, 1.47271612346, 0.0346005603489, 0.474907228495, 0.61379496381},
-        {-418.397356757, -1.83395936566, -0.911702678716, -0.532478094882, -2.03835348133, -0.423005552518},
-        {55.0298153952, -0.0301384716096, -0.0137929430966, -0.348583692759, 0.986486580719, 0.154436524434},
-        {127.150063206, 1.92682560465, -0.434844790414, 0.1082898967, -0.00723338222402, -0.513199251824},
-        {89.6172507626, 1.02463790902, 0.744369837717, 1.250323683, -1.58252612128, -0.588242778808},
-        {92.5124829355, -0.403298547743, 0.0422774545428, -0.175000467434, 1.61110066857, 0.422330077287},
-        {-303.040366788, 0.611569308879, -1.21926246291, -2.49250330276, -0.789166929605, -1.30166501196},
-        {-17.4020602839, 1.72337202371, -1.83540537288, 0.731588761841, -0.338642535062, -1.11053518125},
-        {114.918701324, 0.437385758628, 0.975885170381, 0.439444038872, 1.51666514156, -1.93095020264},
-        {-8.43548064928, -0.799507968686, -0.00842968328782, -0.154994093964, 1.09169753491, -0.0114818657732},
-        {109.209286025, 2.56472965015, -2.07047248035, -0.46764001177, 0.845267147375, -0.236767841427},
-        {61.5259982971, -0.379391870148, -0.131017762354, -0.220275015864, 1.82097825699, -0.0568354876403},
-        {-71.3872099588, 0.642138455414, -1.00242489879, 0.536780074488, 0.350977275771, -1.8204862883},
-        {-21.2768078629, -0.454268998895, 0.0992324274219, 0.0363496803224, 0.281940751723, -0.198435570828},
-        {-8.07838891387, -0.331642089041, -0.494067341253, 0.386035842816, -0.738221128298, 1.18236299649},
-        {30.4818041751, 0.099206096537, 0.150688905006, 0.332932621949, 0.194845631964, -0.446717875795},
-        {237.209150991, 1.12560447042, 0.448488431264, -0.724623711259, 0.401868257097, 1.67129001163},
-        {185.172816475, 0.36594142556, -0.0796476435741, 0.473836257, 1.30890722633, 0.592415068693},
-        {19.8830237044, 1.52497319332, 0.466906090264, -0.716635613964, -1.19532276745, -0.697663531684},
-        {209.396793626, 0.368478789658, 0.699162303982, 1.96702434462, -0.815379139879, 0.863369634396},
-        {-215.100514168, -1.83902416164, -1.14966820385, -1.01044860587, 1.76881340629, -0.32165916241},
-        {-33.4687353426, -0.0451102002703, 0.642212950033, 0.580822065219, -1.02341504063, -0.781229325942},
-        {150.251474823, 0.220170650298, 0.224858901011, 0.541299425328, 1.15151550963, 0.0329044069571},
-        {92.2160506097, 1.86450932451, -0.991150940533, -1.49137866968, 1.02113774105, 0.0544762857136},
-        {41.2138467595, -0.778892265105, 0.714957464344, 1.79833618993, -0.335322825621, -0.397548301803},
-        {13.151262759, 0.301745607362, 0.129778280739, 0.260094818273, -0.10587841585, -0.599330307629},
-        {-367.864703951, -1.68695981263, -0.611957677512, -0.0362971579679, -1.2169760515, -1.43224375134},
-        {-57.218869838, 0.428806849751, 0.654302177028, -1.31651788496, 0.363857431276, -1.49953703016},
-        {53.0877462955, -0.411907760185, -0.192634094071, -0.275879375023, 0.603562526571, 1.16508196734},
-        {-8.11860742896, 1.00263982158, -0.157031169267, -1.11795623393, 0.35711440521, -0.851124640982},
-        {-49.1878248403, -0.0253797866589, -0.574767070714, 0.200339045636, -0.0107042446803, -0.351288977927},
-        {-73.8835407053, -2.07980276724, 1.12235566491, -0.917150593536, 0.741384768556, 0.56229424235},
-        {143.163604045, 0.33627769945, 1.07948757447, 0.894869929963, 1.18688316974, -1.54722487849},
-        {92.7045830908, 0.944091525689, 0.693296229491, 0.700097596814, -1.23666276942, -0.203890113084},
-        {79.1878852355, -0.221973023853, -0.566066329011, 1.57683748648, 0.52854717911, 0.147924782476},
-        {30.6547392801, -1.03466213359, 0.606784904328, -0.298096511956, 0.83332987683, 0.636339018254},
-        {-329.128386019, -1.41363866598, -1.34966434823, -0.989010564149, 0.46889477248, -1.20493210784},
-        {121.190205512, 0.0393914245697, 1.98392444232, -0.65310705226, -0.385899987099, 0.444982471471},
-        {-97.0333075649, 0.264325871992, -0.43074811924, -1.14737761316, -0.453134140655, -0.038507405311},
-        {158.273624516, 0.302255432981, -0.292046617818, 1.0704087606, 0.815965268115, 0.470631083546},
-        {8.24795061818, -1.15155524496, 1.29538707184, -0.4650881541, 0.805123486308, -0.134706887329},
-        {87.1140049059, -0.103540823781, -0.192259440773, 1.79648860085, -1.07525447993, 1.06985127941},
-        {-25.1300772481, -0.97140742052, 0.033393948794, -0.698311192672, 0.74417168942, 0.752776770225},
-        {-285.477057638, -0.480612406803, -1.46081500036, -1.92518386336, -0.426454066275, -0.0539099489597},
-        {-65.1269988498, -1.22733468764, 0.121538452336, 0.752958777557, -0.40643211762, 0.257674949803},
-        {-17.1813504942, 0.823753836891, 0.445142465255, 0.185644700144, -1.99733367514, -0.247899323048},
-        {-46.7543447303, 0.183482778928, -0.934858705943, -1.21961947396, 0.460921844744, 0.571388077177},
-        {-1.7536190499, -0.107517908181, 0.0334282610968, -0.556676121428, -0.485957577159, 0.943570398164},
-        {-42.8460452689, 0.944999215632, 0.00530052154909, -0.348526283976, -1.724125354, -0.122649339813},
-        {62.6291497267, 0.249619894002, 1.3139125969, -1.5644227783, 0.117605482783, 0.304844650662},
-        {97.4552176343, 1.59332799639, -1.17868305562, 1.02998378902, -0.31959491258, -0.183038322076},
-        {-6.19358885758, 0.437951016253, 0.373339269494, -0.204072768495, 0.477969349931, -1.52176449389},
-        {34.0350630099, 0.839319087287, -0.610157662489, 1.73881448393, -1.89200107709, 0.204946415522},
-        {54.9790822536, -0.191792583114, 0.989791127554, -0.502154080064, 0.469939512389, -0.102304071079},
-        {58.8272402843, 0.0769623906454, 0.501297284297, -0.410054999243, 0.595712387781, -0.0968329050729},
-        {95.3620983209, 0.0661481959314, 0.0935137309086, 1.11823292347, -0.612960777903, 0.767865072757},
-        {62.4278196648, 0.78350610065, -1.09977017652, 0.526824784479, 1.41310104196, -0.887902707319},
-        {57.6298676729, 0.60084172954, -0.785932027202, 0.0271301584637, -0.134109499719, 0.877256170191},
-        {5.14112905382, -0.738359365006, 1.40242539359, -0.852833010305, -0.68365080837, 0.88561193696},
-        {11.6057244034, -0.958911227571, 1.15715937023, 1.20108425431, 0.882980929338, -1.77404120156},
-        {-265.758185272, -1.2092434823, -0.0550151798639, 0.00703735243613, -1.01767244359, -1.40616581707},
-        {180.625928828, -0.139091127126, 0.243250756129, 2.17509702585, -0.541735827898, 1.2109459934},
-        {-183.604103216, -0.324555097769, -1.71317286749, 1.03645005723, 0.497569347608, -1.96688185911},
-        {9.93237328848, 0.825483591345, 0.910287997312, -1.64938108528, 0.98964075968, -1.65748940528},
-        {-88.6846949813, -0.0759295112746, -0.593311990101, -0.578711915019, 0.256298822361, -0.429322890198},
-        {175.367391479, 0.9361754906, -0.0172852897292, 1.04078658833, 0.919566407184, -0.554923019093},
-        {-175.538247146, -1.43498590417, 0.37233438556, -0.897205352198, -0.339309952316, -0.0321624527843},
-        {-126.331680318, 0.160446617623, 0.816642363249, -1.39863371652, 0.199747744327, -2.13493607457},
-        {116.677107593, 1.19300905847, -0.404409346893, 0.646338976096, -0.534204093869, 0.36692724765},
-        {-181.675962893, -1.57613169533, -0.41549571451, -0.956673746013, 0.35723782515, 0.318317395128},
-        {-55.1457877823, 0.63723030991, -0.324480386466, 0.296028333894, -1.68117515658, -0.131945601375},
-        {25.2534791013, 0.594818219911, -0.0247380403547, -0.101492246071, -0.0745619242015, -0.370837128867},
-        {63.6006283756, -1.53493473818, 0.946464097439, 0.637741397831, 0.938866921166, 0.54405291856},
-        {-69.6245547661, 0.328482934094, -0.776881060846, -0.285133098443, -1.06107824512, 0.49952182341},
-        {233.425957233, 3.10582399189, -0.0854710508706, 0.455873479133, -0.0974589364949, -1.18914783551},
-        {-86.5564290626, -0.819839276484, 0.584745927593, -0.544737106102, -1.21927675581, 0.758502626434},
-        {425.357285631, 1.70712253847, 1.19892647853, 1.60619661301, 0.36832665241, 0.880791322709},
-        {111.797225426, 0.558940594145, -0.746492420236, 1.90172101792, 0.853590062366, -0.867970723941},
-        {-253.616801014, -0.426513440051, 0.0388582291888, -1.18576061365, -2.70895868242, 0.26982210287},
-        {-394.801501024, -1.65087241498, 0.735525201393, -2.02413077052, -0.96492749037, -1.89014065613}
-    }, new double[] {93.3843533037, 72.3610889215, 57.5295295915, 63.7287541653, 65.2263084024}, 6.85683020686);
-
-    /**
-     * Artificial dataset with 100 observations described by 10 features.
-     */
-    public static final TestDataset regression100x10 = new TestDataset(new double[][] {
-        {69.5794204114, -0.684238565877, 0.175665643732, 0.882115894035, 0.612844187624,
-            -0.685301720572, -0.8266500007, -0.0383407025118, 1.7105205222, 0.457436379836, -0.291563926494},
-        {80.1390102826, -1.80708821811, 0.811271788195, 0.30248512861, 0.910658009566,
-            -1.61869762501, -0.148325085362, -0.0714164596509, 0.671646742271, 2.15160094956, -0.0495754979721},
-        {-156.975447515, 0.170702943934, -0.973403372054, -0.093974528453, 1.54577255871,
-            -0.0969022857972, -1.10639617368, 1.51752480948, -2.86016865032, 1.24063030602, -0.521785751026},
-        {-158.134931891, 0.0890071395055, -0.0811824442353, -0.737354274843, -1.7575255492,
-            0.265777246641, 0.0745347238144, -0.457603542683, -1.37034043839, 1.86011799875, 0.651214189491},
-        {-131.465820263, 0.0767565260375, 0.651724194978, 0.142113799753, 0.244367469855,
-            -0.334395162837, -0.069092305876, -0.691806779713, -1.28386786177, -1.43647491141, 0.00721053414234},
-        {-125.468890054, 0.43361925912, -0.800231440065, -0.576001094593, 0.0783664516431,
-            -1.33613252233, -0.968385062126, -1.22077801286, 0.193456109638, -3.09372314386, 0.817979620215},
-        {-44.1113403874, -0.595796803171, 1.29482131972, -0.784513985654, 0.364702038003,
-            -3.2452492093, -0.451605560847, 0.988546607514, 0.492096628873, -0.343018842342, -0.519231306954},
-        {61.2269707872, -0.0289059337716, -1.00409238976, 0.329908621635, 1.41965097539,
-            0.0395065997587, -0.477939549336, 0.842336765911, -0.808790019648, 1.70241718768, -0.117194118865},
-        {301.434286126, 0.430005308515, 1.01290089725, -0.228221561554, 0.463405921629,
-            -0.602413489517, 1.13832440088, 0.930949226185, -0.196440161506, 1.46304624346, 1.23831509056},
-        {-270.454814681, -1.43805412632, -0.256309572507, -0.358047601174, 0.265151660237,
-            1.07087986377, -1.93784654681, -0.854440691754, 0.665691996289, -1.87508012738, -0.387092423365},
-        {-97.6198688184, -1.67658167161, -0.170246709551, -2.26863722189, 0.280289356338,
-            -0.690038347855, -1.69282684019, 0.978606053022, 1.28237852256, -1.2941998486, 0.766405365374},
-        {-29.5630902399, -1.75615633921, 0.633927486329, -1.24117311555, -0.15884687004,
-            0.31296863712, -1.29513272039, 0.344090683606, 1.19598425093, -1.96195019104, 1.81415061059},
-        {-130.896377427, 0.577719366939, -0.087267771748, -0.060088767013, 0.469803880788,
-            -1.03078212088, -1.41547398887, 1.38980586981, -0.37118000595, -1.81689513712, -0.3099432567},
-        {79.6300698059, 1.23408625633, 1.06464588017, 1.23403332691, -1.10993859098,
-            0.874825200577, 0.589337796957, -1.10266185141, 0.842960469618, -0.89231962021, 0.284074900504},
-        {-154.712112815, -1.64474237898, -0.328581696933, 0.38834343178, 0.02682160335,
-            -0.251167527796, -0.199330632103, -0.0405837345525, -0.908200250794, -1.3283756975, 0.540894408264},
-        {233.447381562, 0.395156450609, 0.156412599781, 0.126453148554, 2.40829068933,
-            1.01623530754, -0.0856520211145, -0.874970377099, 0.280617145254, -0.307070438514, 0.4599616054},
-        {209.012380432, -0.848646647675, 0.558383548084, -0.259628264419, 1.1624126549,
-            -0.0755949979572, -0.373930759448, 0.985903312667, 0.435839508011, -0.760916312668, 1.89847574116},
-        {-39.8987262091, 0.176656582642, 0.508538223618, 0.995038391204, -2.08809409812,
-            0.743926580134, 0.246007971514, -0.458288599906, -0.579976479473, 0.0591577146017, 1.64321662761},
-        {222.078510236, -0.24031989218, -0.168104260522, -0.727838425954, 0.557181757624,
-            -0.164906646307, 2.01559331734, 0.897263594222, 0.0921535309562, 0.351910490325, -0.018228500121},
-        {-250.916272061, -2.71504637339, 0.498966191294, -3.16410707344, -0.842488891776,
-            1.27425275951, 0.0141733666756, 0.695942743199, 0.0917995810179, -0.501447196978, -0.355738068451},
-        {134.07259088, 0.0845637591619, 0.237410106679, -0.291458113729, 1.39418566986,
-            -1.18813057956, -0.683117067763, -0.518910379335, 1.35998426879, -1.28404562245, 0.489131754943},
-        {104.988440209, 0.00770925058526, 0.47113239214, -0.606231247854, 0.310679840217,
-            0.146297599928, 0.732013998647, -0.284544010865, 0.402622530153, -0.0217367745613, 0.0742970687987},
-        {155.558071031, 1.11171654653, 0.726629222799, -0.195820863177, 0.801333855535,
-            0.744034755544, 1.11377275513, -0.75673532139, -0.114117607244, -0.158966474923, -0.29701120385},
-        {90.7600194013, -0.104364079622, -0.0165109945217, 0.933002972987, -1.80652594466,
-            -1.34760892883, -0.304511906801, 0.0584734540581, 1.5332169392, 0.478835797824, 1.71534051065},
-        {-313.910553214, 0.149908925551, 0.232806828559, -0.0708920471592, -0.0649553559745,
-            0.377753357707, -0.957292311668, 0.545360522582, -1.37905464371, -0.940702110994, -1.53620430047},
-        {-80.9380113754, 0.135586606896, 0.95759558815, -1.36879020479, 0.735413996144,
-            0.637984100201, -1.79563152885, 1.55025691631, 0.634702068786, -0.203690334141, -0.83954824721},
-        {-244.336816695, -0.179127343947, -2.12396005014, -0.431179356484, -0.860562153749,
-            -1.10270688639, -0.986886012982, -0.945091656162, -0.445428453767, 1.32269756209, -0.223712672168},
-        {123.069612745, 0.703857129626, 0.291605144784, 1.40233051946, 0.278603787802,
-            -0.693567967466, -0.15587953395, 2.10213915684, 0.130663329174, -0.393184478882, 0.0874812844555},
-        {-148.274944223, 1.66294967732, 0.0830002694123, 0.32492930502, 1.11864359687,
-            -0.381901627785, -1.06367037132, -0.392583620174, -1.16283326187, 0.104931461025, -1.64719611405},
-        {-82.0018788235, 0.497118817453, 0.731125358012, -0.00976413646786, -0.0178930713492,
-            -0.814978582886, 0.0602834712523, -0.661940479055, -0.957902899386, -1.34489251111, 0.22166518707},
-        {-35.742996986, 0.0661349516701, -0.204314495629, 1.17101314753, -2.53846825562,
-            -0.560282479298, -0.393442894828, 0.988953809491, -0.911281277704, 0.86862242698, 2.59576940486},
-        {-109.588885664, -0.0793151346628, -0.408962434518, -0.598817776528, 0.0277205469561,
-            0.116291018958, 0.0280416838086, -0.72544170676, -0.669302814774, 0.0751898759816, -0.311002356179},
-        {57.8285173441, 0.53753903532, 0.676340503752, -2.10608342721, 0.477714987751,
-            0.465695114442, 0.245966562421, -1.05230350808, -0.309794163113, -1.12067331828, 1.07841453304},
-        {204.660622582, -0.717565166685, 0.295179660279, -0.377579912697, 1.88425526905,
-            0.251875238436, -0.900214103232, -1.02877401105, 0.291693915093, 1.24889067987, 1.78506220081},
-        {350.949109103, 2.82276814452, -0.429358342127, 1.12140362367, 1.18120725208,
-            -1.63913834939, 1.61441562446, -0.364003766916, -0.258752942225, -0.808124680189, 0.556463488303},
-        {170.960252153, 0.147245922081, 0.3257117575, 0.211749283649, -0.0150701808404,
-            -0.888523132148, 0.777862088798, 0.296729270892, -0.332927550718, 0.888968144245, 1.20913118467},
-        {112.192270383, 0.129846138824, -0.934371449036, -0.595825303214, 1.74749214629,
-            -0.0500069421443, -0.161976298602, -2.54100791613, 1.99632530735, -0.0691582773758, -0.863939367415},
-        {-56.7847711121, 0.0950532853751, -0.467349228201, -0.26457152362, -0.422134692317,
-            -0.0734763062127, 0.90128235602, -1.68470856275, -0.0699692697335, -0.463335845504, -0.301754321169},
-        {-37.9223252258, -1.40835827778, 0.566142056244, -3.22393318933, 0.228823495106,
-            -1.8480727782, 0.129468321643, -1.77392686536, 0.0112549619662, 0.146433267822, 1.29379901303},
-        {-59.7303066136, 0.835675535576, -0.552173157548, 1.90730898966, -0.520145317195,
-            1.55174485912, -1.37531768692, -0.408165743742, 0.0939675842223, 0.318004128812, 0.324378038446},
-        {-0.916090786983, 0.425763794043, -0.295541268984, -0.066619586336, 2.03494974978,
-            -0.197109278058, -0.823307883209, 0.895531446352, -0.276435938737, -1.54580056755, -0.820051830246},
-        {-20.3601082842, 0.56420556369, 0.741234589387, -0.565853617392, -0.311399905686,
-            2.24066463251, -0.071704904286, -1.22796531596, 0.186020404046, -0.786874824874, 0.23140277151},
-        {-22.9342855182, -0.0682789648279, -1.30680909143, 0.0486490588348, 0.890275695028,
-            -0.257961411112, -0.381531755985, 1.56251482581, -2.11808219232, 0.741828675202, 0.696388901165},
-        {-157.251026807, -2.3120966502, 0.183734662375, 1.02192264962, 0.591272941061,
-            -0.0132855098339, -1.02016546348, 1.19642432892, 0.867653154846, -1.37600041722, -1.08542822792},
-        {-68.6110752055, -1.2429968179, -0.950064269349, -0.332379873336, 0.25793632341,
-            0.145780713577, -0.512109283074, -0.477887632032, 0.448960776324, -0.190215737958, 0.219578347563},
-        {-56.1204152481, -0.811729480846, -0.647410362207, 0.934547463984, -0.390943346216,
-            -0.409981308474, 0.0923465893049, 1.9281242912, -0.624713581674, -0.0599353282306, -0.0188591746808},
-        {348.530651658, 2.51721790231, 0.7560998114, -2.69620396681, 0.5174276585,
-            0.403570816695, 0.901648571306, 0.269313230294, 1.07811463589, 0.986649559679, 0.514710327657},
-        {-105.719065924, 0.679016972998, 0.341319363316, -0.515209647377, 0.800000866847,
-            -0.795474442628, -0.866849274801, -1.32927961486, 0.17679343917, -1.93744422464, -0.476447619273},
-        {-197.389429553, -1.98585668879, -0.962610549884, -2.48860863254, -0.545990524642,
-            -0.13005685654, -1.23413782366, 1.17443427507, 1.4785554038, -0.193717671824, -0.466403609229},
-        {-23.9625285402, -0.392164367603, 1.07583388583, -0.412686712477, -0.89339030785,
-            -0.774862334739, -0.186491999529, -0.300162444329, 0.177377235999, 0.134038296039, 0.957945226616},
-        {-91.145725943, -0.154640540119, 0.732911957939, -0.206326119636, -0.569816760116,
-            0.249393336416, -1.02762332953, 0.25096708081, 0.386927162941, -0.346382299592, 0.243099162109},
-        {-80.7295722208, -1.72670707303, 0.138139045677, 0.0648055728598, 0.186182854422,
-            1.07226527747, -1.26133459043, 0.213883744163, 1.47115466163, -1.54791582859, 0.170924664865},
-        {-317.060323531, -0.349785690206, -0.740759426066, -0.407970845617, -0.689282767277,
-            -1.25608665316, -0.772546119412, -2.02925712813, 0.132949072522, -0.191465137244, -1.29079690284},
-        {-252.491508279, -1.24643122869, 1.55335609203, 0.356613424877, 0.817434495353,
-            -1.74503747683, -0.818046363088, -1.58284235058, 0.357919389759, -1.18942962791, -1.91728745247},
-        {-66.8121363157, -0.584246455697, -0.104254351782, 1.17911687508, -0.29288167882,
-            0.891836132692, 0.232853863255, 0.423294355343, -0.669493690103, -1.15783890498, 0.188213983735},
-        {140.681464689, 1.33156046873, -1.8847915949, -0.666528837988, -0.513356191443,
-            0.281290031669, -1.07815005006, 1.22384196227, 1.39093631269, 0.527644817197, 1.21595221509},
-        {-174.22326767, 0.475428766034, 0.856847216768, -0.734282773151, -0.923514989791,
-            0.917510828772, 0.674878068543, 0.0644776431114, -0.607796192908, 0.867740011912, -1.97799769281},
-        {74.3899799579, 0.00915743526294, 0.553578683413, 1.66930486354, 0.15562803404,
-            1.8455840688, -0.371704942927, 1.11228894843, -0.37464389118, -0.48789151589, 0.79553866342},
-        {70.1167175897, 0.154877045187, 1.47803572976, -0.0355743163524, -2.47914644675,
-            0.672384381837, 1.63160379529, 1.81874583854, 1.22797339421, -0.0131258061634, -0.390265963676},
-        {-11.0364788877, 0.173049156249, -1.78140521797, -1.29982707214, -0.48025663179,
-            -0.469112922302, -1.98718063269, 0.585086542043, 0.264611327837, 1.48855512579, 2.00672263496},
-        {-112.711292736, -1.59239636827, -0.600613018822, -0.0209667499746, -1.81872893331,
-            -0.739893084955, 0.140261888569, -0.498107678308, 2.53664045504, -0.536385019089, -0.608755809378},
-        {-198.064468217, 0.737175509877, -2.01835515547, -2.18045950065, 0.428584922529,
-            -1.01848835019, -0.470645361539, -0.00703630153547, -2.2341302754, 1.51483167022, -0.410184418418},
-        {70.2747963991, 1.49474111532, -0.19517712503, 0.7392852909, -0.326060871666,
-            -0.566710349675, 0.14053094122, -0.562830341306, 0.22931613446, -0.0344439061448, 0.175150510551},
-        {207.909021337, 0.839887009159, 0.268826583246, -0.313047158862, 1.12009996015,
-            0.214209976971, -0.396147338251, 2.16039704403, 0.699141312749, 0.756192350992, -0.145368196901},
-        {169.428609429, -1.13702350819, 1.23964530597, -0.864443556622, -0.885630795949,
-            -0.523872327352, 0.467159824748, 0.476596383923, 0.4343735578, 1.4075417896, 2.22939328991},
-        {-176.909833405, 0.0875512760866, -0.455542269288, 0.539742307764, -0.762003092788,
-            0.41829123457, -0.818116139644, -2.01761645956, 0.557395073218, 1.5823271814, -1.0168826293},
-        {-27.734298611, -0.841257541979, 0.348961259301, 1.36935991472, -0.0694528057586,
-            -1.27303784913, 0.152155656569, 1.9279466651, 0.9589415766, -1.76634370106, -1.08831026428},
-        {-55.8416853588, 0.927711536927, 0.157856746063, -0.295628714893, 0.0296602829783,
-            1.75198587897, -0.38285446366, -0.253287154535, -1.64032395229, -0.842089054965, 1.00493779183},
-        {56.0899797005, 0.326117761734, -1.93514762146, 1.0229172721, 0.125568968732,
-            2.37760000658, -0.498532972011, -0.733375842271, -0.757445726993, -0.49515057432, 2.01559891524},
-        {-176.220234909, 1.571129843, -0.867707605929, -0.709690799512, -1.51535538937,
-            1.27424225477, -0.109513704468, -1.46822183, 0.281077088939, -1.97084024232, -0.322309524179},
-        {37.7155152941, 0.363383774219, -0.0240881298641, -1.60692745228, -1.26961656439,
-            -0.41299134216, 1.2890099968, -1.34101694629, -0.455387485256, -0.14055003482, 1.5407059956},
-        {-102.163416997, -2.05927378316, -0.470182865756, -0.875528863204, 0.0361720859253,
-            -1.03713912263, 0.417362606334, 0.707587625276, -0.0591627772581, -2.58905252006, 0.516573345216},
-        {-206.47095321, 0.270030584651, 1.85544202116, -0.144189208964, -0.696400687327,
-            0.0226388634283, -0.490952489106, -1.69209527849, 0.00973614309272, -0.484105876992, -0.991474668217},
-        {201.50637416, 0.513659215697, -0.335630132208, -0.140006500483, 0.149679720127,
-            -1.89526167503, -0.0614973894156, 0.0813221153552, 0.630952530848, 2.40201011339, 0.997708264073},
-        {-72.0667371571, 0.0841570292899, -0.216125859013, -1.77155215764, 2.15081767322,
-            0.00953341785443, -1.0826077946, -0.791135571106, -0.989393577892, -0.791485083644, -0.063560999686},
-        {-162.903837815, -0.273764637097, 0.282387854873, -1.39881596931, 0.554941097854,
-            -0.88790718926, -0.693189960902, 0.398762630571, -1.61878562893, -0.345976341096, 0.138298909959},
-        {-34.3291926715, -0.499883755911, -0.847296893019, -0.323673126437, 0.531205373462,
-            -0.0204345595983, 0.284954510306, 0.565031773028, -0.272049818708, -0.130369799738, -0.617572026201},
-        {76.1272883187, -0.908810282403, -1.04139421904, 0.890678872055, 1.32990256154,
-            -0.0150445428835, 0.593918101047, 0.356897732999, 0.824651162423, -1.54544256217, -0.795703905296},
-        {171.833705285, -0.0425219657568, -0.884042952325, 1.91202504537, 0.381908223898,
-            -0.205693527739, 1.53656598237, 0.534880398015, 0.291950716831, -1.1258051056, -0.0612803476297},
-        {-235.445792009, 0.261252102941, -0.170931758001, 1.67878144235, 0.0278283741792,
-            -1.23194408479, -0.190931886594, 1.0000157972, -2.18792142659, -0.230654984288, -1.36626493512},
-        {348.968834231, 1.35713154434, 0.950377770072, 0.0700577471848, 0.96907140156,
-            2.00890422081, 0.0896405239806, 0.614309607351, 1.07723409067, 2.58506968136, 0.202889806148},
-        {-61.0128039201, 0.465438505031, -1.31448530533, 0.374781933416, -0.0118298606041,
-            -0.477338357738, -0.587656108109, 1.66449545077, 0.435836048385, -0.287027953004, -1.06613472784},
-        {-50.687090469, 0.382331825989, -0.597140322197, 1.1276065465, -1.35593777887,
-            1.14949964423, -0.858742432885, -0.563211485633, -0.57167161928, 0.0294891749132, 1.9571639493},
-        {-186.653649045, -0.00981380006029, 1.0371088941, -1.25319048981, -0.694043021068,
-            1.7280802541, -0.191210409232, -0.866039238001, -0.0791927416078, -0.232228656558, -0.93723545053},
-        {34.5395591744, 0.680943971029, -0.075875481801, -0.144408300848, -0.869070791528,
-            0.496870904214, 1.0940401388, -0.510489750436, -0.47562728601, 0.951406841944, 0.12983846382},
-        {-23.7618645627, 0.527032820313, -0.58295129357, -0.3894567306, -0.0547905472556,
-            -1.86103603537, 0.0506988360667, 1.02778539291, -0.0613720063422, 0.411280841442, -0.665810811374},
-        {116.007776415, 0.441750249008, 0.549342185228, 0.731558201455, -0.903624700864,
-            -2.13208328824, 0.381223328983, 0.283479210749, 1.17705098922, -2.38800904207, 1.32108350152},
-        {-148.479593311, -0.814604260049, -0.821204361946, -1.08768677334, -0.0659445766599,
-            0.583741297405, 0.669345853296, -0.0935352010726, -0.254906787938, -0.394599725657, -1.26305927257},
-        {244.865845084, 0.776784257443, 0.267205388558, 2.37746488031, -0.379275360853,
-            -0.157454754411, -0.359580726073, 0.886887721861, 1.53707627973, 0.634390546684, 0.984864824122},
-        {-81.9954096721, 0.594841146008, -1.22273253129, 0.532466794358, 1.69864239257,
-            -0.12293671327, -2.06645974171, 0.611808231703, -1.32291985291, 0.722066660478, -0.0021343848511},
-        {-245.715046329, -1.77850303496, -0.176518810079, 1.20463434525, -0.597826204963,
-            -1.45842350123, -0.765730251727, -2.17764204443, 0.12996635702, -0.705509516482, 0.170639846082},
-        {123.011946043, -0.909707162714, 0.92357208515, 0.373251929121, 1.24629576577,
-            0.0662688299998, -0.372240547929, -0.739353735168, 0.323495756066, 0.954154005738, 0.69606859977},
-        {-70.4564963177, 0.650682297051, 0.378131376232, 1.37860253614, -0.924042783872,
-            0.802851073842, -0.450299927542, 0.235646185302, -0.148779896161, 1.01308126122, -0.48206889502},
-        {21.5288687935, 0.290876355386, 0.0765702960599, 0.905225489744, 0.252841861521,
-            1.26729272819, 0.315397441908, -2.00317261368, -0.250990653758, 0.425615332405, 0.0875320802483},
-        {231.370169905, 0.535138021352, -1.07151617232, 0.824383756287, 1.84428896701,
-            -0.890892034494, 0.0480296332924, -0.59251208055, 0.267564961845, -0.230698441998, 0.857077278291},
-        {38.8318274023, 2.63547217711, -0.585553060394, 0.430550920323, -0.532619160993,
-            1.25335488136, -1.65265278435, 0.0433880112291, -0.166143379872, 0.534066441314, 1.18929937797},
-        {116.362219013, -0.275949982433, 0.468069787645, -0.879814121059, 0.862799331322,
-            1.18464846725, 0.747084253268, 1.39202500691, -1.23374181275, 0.0949815110503, 0.696546907194},
-        {260.540154731, 1.13798788241, -0.0991903174656, 0.1241636043, -0.201415073037,
-            1.57683389508, 1.81535629587, 1.07873616646, -0.355800782882, 2.18333193195, 0.0711071144615},
-        {-165.835194521, -2.76613178307, 0.805314338858, 0.81526046683, -0.710489036197,
-            -1.20189542317, -0.692110074722, -0.117239516622, 1.0431459458, -0.111898596299, -0.0775811519297},
-        {-341.189958588, 0.668555635008, -1.0940034941, -0.497881262778, -0.603682823779,
-            -0.396875163796, -0.849144848521, 0.403936807183, -1.82076277475, -0.137500972546, -1.22769896568}
-    }, new double[] {45.8685095528, 11.9400336005, 16.3984976652, 79.9069814034, 5.65486853464,
-        83.6427296424, 27.4571268153, 73.5881193584, 27.1465364511, 79.4095449062}, -5.14077007134);
-
-    /** */
-    public static class TestDataset {
-
-        /** */
-        private final double[][] data;
-
-        /** */
-        private final double[] expWeights;
-
-        /** */
-        private final double expIntercept;
-
-        /** */
-        TestDataset(double[][] data, double[] expWeights, double expIntercept) {
-            this.data = data;
-            this.expWeights = expWeights;
-            this.expIntercept = expIntercept;
-        }
-
-        /** */
-        public double[][] getData() {
-            return data;
-        }
-
-        /** */
-        public double[] getExpWeights() {
-            return expWeights;
-        }
-
-        /** */
-        public double getExpIntercept() {
-            return expIntercept;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java
deleted file mode 100644
index 0c09d75..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.SparseBlockDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
-
-/**
- * Tests for {@link LinearRegressionQRTrainer} on {@link SparseBlockDistributedMatrix}.
- */
-public class BlockDistributedLinearRegressionQRTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest {
-    /** */
-    public BlockDistributedLinearRegressionQRTrainerTest() {
-        super(
-            new LinearRegressionQRTrainer(),
-            SparseBlockDistributedMatrix::new,
-            SparseBlockDistributedVector::new,
-            1e-6
-        );
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java
deleted file mode 100644
index 2a506d9..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
-
-/**
- * Tests for {@link LinearRegressionQRTrainer} on {@link SparseDistributedMatrix}.
- */
-public class DistributedLinearRegressionQRTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest {
-    /** */
-    public DistributedLinearRegressionQRTrainerTest() {
-        super(
-            new LinearRegressionQRTrainer(),
-            SparseDistributedMatrix::new,
-            SparseDistributedVector::new,
-            1e-6
-        );
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java
deleted file mode 100644
index a55623c..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java
+++ /dev/null
@@ -1,206 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import java.util.Scanner;
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.junit.Test;
-
-/**
- * Base class for all linear regression trainers.
- */
-public class GenericLinearRegressionTrainerTest {
-    /** */
-    private final Trainer<LinearRegressionModel, Matrix> trainer;
-
-    /** */
-    private final IgniteFunction<double[][], Matrix> matrixCreator;
-
-    /** */
-    private final IgniteFunction<double[], Vector> vectorCreator;
-
-    /** */
-    private final double precision;
-
-    /** */
-    public GenericLinearRegressionTrainerTest(
-        Trainer<LinearRegressionModel, Matrix> trainer,
-        IgniteFunction<double[][], Matrix> matrixCreator,
-        IgniteFunction<double[], Vector> vectorCreator,
-        double precision) {
-        this.trainer = trainer;
-        this.matrixCreator = matrixCreator;
-        this.vectorCreator = vectorCreator;
-        this.precision = precision;
-    }
-
-    /**
-     * Test trainer on regression model y = 2 * x.
-     */
-    @Test
-    public void testTrainWithoutIntercept() {
-        Matrix data = matrixCreator.apply(new double[][] {
-            {2.0, 1.0},
-            {4.0, 2.0}
-        });
-
-        LinearRegressionModel mdl = trainer.train(data);
-
-        TestUtils.assertEquals(4, mdl.apply(vectorCreator.apply(new double[] {2})), precision);
-        TestUtils.assertEquals(6, mdl.apply(vectorCreator.apply(new double[] {3})), precision);
-        TestUtils.assertEquals(8, mdl.apply(vectorCreator.apply(new double[] {4})), precision);
-    }
-
-    /**
-     * Test trainer on regression model y = -1 * x + 1.
-     */
-    @Test
-    public void testTrainWithIntercept() {
-        Matrix data = matrixCreator.apply(new double[][] {
-            {1.0, 0.0},
-            {0.0, 1.0}
-        });
-
-        LinearRegressionModel mdl = trainer.train(data);
-
-        TestUtils.assertEquals(0.5, mdl.apply(vectorCreator.apply(new double[] {0.5})), precision);
-        TestUtils.assertEquals(2, mdl.apply(vectorCreator.apply(new double[] {-1})), precision);
-        TestUtils.assertEquals(-1, mdl.apply(vectorCreator.apply(new double[] {2})), precision);
-    }
-
-    /**
-     * Test trainer on diabetes dataset.
-     */
-    @Test
-    public void testTrainOnDiabetesDataset() {
-        Matrix data = loadDataset("datasets/regression/diabetes.csv", 442, 10);
-
-        LinearRegressionModel mdl = trainer.train(data);
-
-        Vector expWeights = vectorCreator.apply(new double[] {
-            -10.01219782, -239.81908937, 519.83978679, 324.39042769, -792.18416163,
-            476.74583782, 101.04457032, 177.06417623, 751.27932109, 67.62538639
-        });
-
-        double expIntercept = 152.13348416;
-
-        TestUtils.assertEquals("Wrong weights", expWeights, mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", expIntercept, mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Test trainer on boston dataset.
-     */
-    @Test
-    public void testTrainOnBostonDataset() {
-        Matrix data = loadDataset("datasets/regression/boston.csv", 506, 13);
-
-        LinearRegressionModel mdl = trainer.train(data);
-
-        Vector expWeights = vectorCreator.apply(new double[] {
-            -1.07170557e-01, 4.63952195e-02, 2.08602395e-02, 2.68856140e+00, -1.77957587e+01, 3.80475246e+00,
-            7.51061703e-04, -1.47575880e+00, 3.05655038e-01, -1.23293463e-02, -9.53463555e-01, 9.39251272e-03,
-            -5.25466633e-01
-        });
-
-        double expIntercept = 36.4911032804;
-
-        TestUtils.assertEquals("Wrong weights", expWeights, mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", expIntercept, mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 10 observations described by 1 feature.
-     */
-    @Test
-    public void testTrainOnArtificialDataset10x1() {
-        ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression10x1;
-
-        LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData()));
-
-        TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 10 observations described by 5 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset10x5() {
-        ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression10x5;
-
-        LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData()));
-
-        TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 100 observations described by 5 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset100x5() {
-        ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression100x5;
-
-        LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData()));
-
-        TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 100 observations described by 10 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset100x10() {
-        ArtificialRegressionDatasets.TestDataset dataset = ArtificialRegressionDatasets.regression100x10;
-
-        LinearRegressionModel mdl = trainer.train(matrixCreator.apply(dataset.getData()));
-
-        TestUtils.assertEquals("Wrong weights", dataset.getExpWeights(), mdl.getWeights(), precision);
-        TestUtils.assertEquals("Wrong intercept", dataset.getExpIntercept(), mdl.getIntercept(), precision);
-    }
-
-    /**
-     * Loads dataset file and returns corresponding matrix.
-     *
-     * @param fileName Dataset file name
-     * @param nobs Number of observations
-     * @param nvars Number of features
-     * @return Data matrix
-     */
-    private Matrix loadDataset(String fileName, int nobs, int nvars) {
-        double[][] matrix = new double[nobs][nvars + 1];
-        Scanner scanner = new Scanner(this.getClass().getClassLoader().getResourceAsStream(fileName));
-        int i = 0;
-        while (scanner.hasNextLine()) {
-            String row = scanner.nextLine();
-            int j = 0;
-            for (String feature : row.split(",")) {
-                matrix[i][j] = Double.parseDouble(feature);
-                j++;
-            }
-            i++;
-        }
-        return matrixCreator.apply(matrix);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
deleted file mode 100644
index 9b75bd4..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
+++ /dev/null
@@ -1,127 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Test;
-
-/**
- * Grid aware abstract linear regression trainer test.
- */
-public abstract class GridAwareAbstractLinearRegressionTrainerTest extends GridCommonAbstractTest {
-    /** Number of nodes in grid */
-    private static final int NODE_COUNT = 3;
-
-    /**
-     * Delegate actually performs tests.
-     */
-    private final GenericLinearRegressionTrainerTest delegate;
-
-    /** */
-    private Ignite ignite;
-
-    /** */
-    public GridAwareAbstractLinearRegressionTrainerTest(
-        Trainer<LinearRegressionModel, Matrix> trainer,
-        IgniteFunction<double[][], Matrix> matrixCreator,
-        IgniteFunction<double[], Vector> vectorCreator,
-        double precision) {
-        delegate = new GenericLinearRegressionTrainerTest(trainer, matrixCreator, vectorCreator, precision);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() {
-        stopAllGrids();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        /* Grid instance. */
-        ignite = grid(NODE_COUNT);
-        ignite.configuration().setPeerClassLoadingEnabled(true);
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-    }
-
-    /**
-     * Test trainer on regression model y = 2 * x.
-     */
-    @Test
-    public void testTrainWithoutIntercept() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainWithoutIntercept();
-    }
-
-    /**
-     * Test trainer on regression model y = -1 * x + 1.
-     */
-    @Test
-    public void testTrainWithIntercept() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainWithIntercept();
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 10 observations described by 1 feature.
-     */
-    @Test
-    public void testTrainOnArtificialDataset10x1() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainOnArtificialDataset10x1();
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 10 observations described by 5 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset10x5() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainOnArtificialDataset10x5();
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 100 observations described by 5 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset100x5() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainOnArtificialDataset100x5();
-    }
-
-    /**
-     * Tests trainer on artificial dataset with 100 observations described by 10 features.
-     */
-    @Test
-    public void testTrainOnArtificialDataset100x10() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        delegate.testTrainOnArtificialDataset100x10();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
index fa8fac4..c62cca5 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainerTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.ml.regressions.linear;
 
 import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
 import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+import org.apache.ignite.ml.nn.UpdatesStrategy;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java
deleted file mode 100644
index f37d71d..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-
-/**
- * Tests for {@link LinearRegressionQRTrainer} on {@link DenseLocalOnHeapMatrix}.
- */
-public class LocalLinearRegressionQRTrainerTest extends GenericLinearRegressionTrainerTest {
-    /** */
-    public LocalLinearRegressionQRTrainerTest() {
-        super(
-            new LinearRegressionQRTrainer(),
-            DenseLocalOnHeapMatrix::new,
-            DenseLocalOnHeapVector::new,
-            1e-6
-        );
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java
deleted file mode 100644
index 7ad59d1..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/DistributedWorkersChainTest.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.Chains;
-import org.apache.ignite.ml.trainers.group.chain.ComputationsChain;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.junit.Assert;
-
-/** */
-public class DistributedWorkersChainTest extends GridCommonAbstractTest {
-    /** Count of nodes. */
-    private static final int NODE_COUNT = 3;
-
-    /** Grid instance. */
-    protected Ignite ignite;
-
-    /**
-     * Default constructor.
-     */
-    public DistributedWorkersChainTest() {
-        super(false);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-        TestGroupTrainingCache.getOrCreate(ignite).removeAll();
-        TestGroupTrainingSecondCache.getOrCreate(ignite).removeAll();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** */
-    public void testId() {
-        ComputationsChain<TestLocalContext, Double, Integer, Integer, Integer> chain = Chains.create();
-
-        UUID trainingUUID = UUID.randomUUID();
-        Integer res = chain.process(1, new GroupTrainingContext<>(new TestLocalContext(0, trainingUUID), TestGroupTrainingCache.getOrCreate(ignite), ignite));
-
-        Assert.assertEquals(1L, (long)res);
-    }
-
-    /** */
-    public void testSimpleLocal() {
-        ComputationsChain<TestLocalContext, Double, Integer, Integer, Integer> chain = Chains.create();
-
-        IgniteCache<GroupTrainerCacheKey<Double>, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite);
-        int init = 1;
-        int initLocCtxData = 0;
-        UUID trainingUUID = UUID.randomUUID();
-        TestLocalContext locCtx = new TestLocalContext(initLocCtxData, trainingUUID);
-
-        Integer res = chain.
-            thenLocally((prev, lc) -> prev + 1).
-            process(init, new GroupTrainingContext<>(locCtx, cache, ignite));
-
-        Assert.assertEquals(init + 1, (long)res);
-        Assert.assertEquals(initLocCtxData, locCtx.data());
-    }
-
-    /** */
-    public void testChainLocal() {
-        ComputationsChain<TestLocalContext, Double, Integer, Integer, Integer> chain = Chains.create();
-
-        IgniteCache<GroupTrainerCacheKey<Double>, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite);
-        int init = 1;
-        int initLocCtxData = 0;
-        UUID trainingUUID = UUID.randomUUID();
-        TestLocalContext locCtx = new TestLocalContext(initLocCtxData, trainingUUID);
-
-        Integer res = chain.
-            thenLocally((prev, lc) -> prev + 1).
-            thenLocally((prev, lc) -> prev * 5).
-            process(init, new GroupTrainingContext<>(locCtx, cache, ignite));
-
-        Assert.assertEquals((init + 1) * 5, (long)res);
-        Assert.assertEquals(initLocCtxData, locCtx.data());
-    }
-
-    /** */
-    public void testChangeLocalContext() {
-        ComputationsChain<TestLocalContext, Double, Integer, Integer, Integer> chain = Chains.create();
-        IgniteCache<GroupTrainerCacheKey<Double>, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite);
-        int init = 1;
-        int newData = 10;
-        UUID trainingUUID = UUID.randomUUID();
-        TestLocalContext locCtx = new TestLocalContext(0, trainingUUID);
-
-        Integer res = chain.
-            thenLocally((prev, lc) -> { lc.setData(newData); return prev;}).
-            process(init, new GroupTrainingContext<>(locCtx, cache, ignite));
-
-        Assert.assertEquals(newData, locCtx.data());
-        Assert.assertEquals(init, res.intValue());
-    }
-
-    /** */
-    public void testDistributed() {
-        ComputationsChain<TestLocalContext, Double, Integer, Integer, Integer> chain = Chains.create();
-        IgniteCache<GroupTrainerCacheKey<Double>, Integer> cache = TestGroupTrainingCache.getOrCreate(ignite);
-        int init = 1;
-        UUID trainingUUID = UUID.randomUUID();
-        TestLocalContext locCtx = new TestLocalContext(0, trainingUUID);
-
-        Map<GroupTrainerCacheKey<Double>, Integer> m = new HashMap<>();
-        m.put(new GroupTrainerCacheKey<>(0L, 1.0, trainingUUID), 1);
-        m.put(new GroupTrainerCacheKey<>(1L, 2.0, trainingUUID), 2);
-        m.put(new GroupTrainerCacheKey<>(2L, 3.0, trainingUUID), 3);
-        m.put(new GroupTrainerCacheKey<>(3L, 4.0, trainingUUID), 4);
-
-        Stream<GroupTrainerCacheKey<Double>> keys = m.keySet().stream();
-
-        cache.putAll(m);
-
-        IgniteBiFunction<Integer, TestLocalContext, IgniteSupplier<Stream<GroupTrainerCacheKey<Double>>>> function = (o, l) -> () -> keys;
-        IgniteFunction<List<Integer>, Integer> max = ints -> ints.stream().mapToInt(x -> x).max().orElse(Integer.MIN_VALUE);
-
-        Integer res = chain.
-            thenDistributedForEntries((integer, context) -> () -> null, this::readAndIncrement, function, max).
-            process(init, new GroupTrainingContext<>(locCtx, cache, ignite));
-
-        int localMax = m.values().stream().max(Comparator.comparingInt(i -> i)).orElse(Integer.MIN_VALUE);
-
-        assertEquals((long)localMax, (long)res);
-
-        for (GroupTrainerCacheKey<Double> key : m.keySet())
-            m.compute(key, (k, v) -> v + 1);
-
-        assertMapEqualsCache(m, cache);
-    }
-
-    /** */
-    private ResultAndUpdates<Integer> readAndIncrement(EntryAndContext<Double, Integer, Void> ec) {
-        Integer val = ec.entry().getValue();
-
-        ResultAndUpdates<Integer> res = ResultAndUpdates.of(val);
-        res.updateCache(TestGroupTrainingCache.getOrCreate(Ignition.localIgnite()), ec.entry().getKey(), val + 1);
-
-        return res;
-    }
-
-    /** */
-    private <K, V> void assertMapEqualsCache(Map<K, V> m, IgniteCache<K, V> cache) {
-        assertEquals(m.size(), cache.size());
-
-        for (K k : m.keySet())
-            assertEquals(m.get(k), cache.get(k));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java
deleted file mode 100644
index 5bb9a47..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/GroupTrainerTest.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-/**
- * Test of {@link GroupTrainer}.
- */
-public class GroupTrainerTest extends GridCommonAbstractTest {
-    /** Count of nodes. */
-    private static final int NODE_COUNT = 3;
-
-    /** Grid instance. */
-    private Ignite ignite;
-
-    /**
-     * Default constructor.
-     */
-    public GroupTrainerTest() {
-        super(false);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-        TestGroupTrainingCache.getOrCreate(ignite).removeAll();
-        TestGroupTrainingSecondCache.getOrCreate(ignite).removeAll();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        for (int i = 1; i <= NODE_COUNT; i++)
-            startGrid(i);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /** */
-    public void testGroupTrainer() {
-        TestGroupTrainer trainer = new TestGroupTrainer(ignite);
-
-        int limit = 5;
-        int eachNumCnt = 3;
-        int iterCnt = 2;
-
-        ConstModel<Integer> mdl = trainer.train(new SimpleGroupTrainerInput(limit, eachNumCnt, iterCnt));
-        int locRes = computeLocally(limit, eachNumCnt, iterCnt);
-        assertEquals(locRes, (int)mdl.apply(10));
-    }
-
-    /** */
-    private int computeLocally(int limit, int eachNumCnt, int iterCnt) {
-        Map<GroupTrainerCacheKey<Double>, Integer> m = new HashMap<>();
-
-        for (int i = 0; i < limit; i++) {
-            for (int j = 0; j < eachNumCnt; j++)
-                m.put(new GroupTrainerCacheKey<>(i, (double)j, null), i);
-        }
-
-        for (int i = 0; i < iterCnt; i++)
-            for (GroupTrainerCacheKey<Double> key : m.keySet())
-                m.compute(key, (key1, integer) -> integer * integer);
-
-        return m.values().stream().filter(x -> x % 2 == 0).mapToInt(i -> i).sum();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java
deleted file mode 100644
index db1adc7..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/SimpleGroupTrainerInput.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.UUID;
-import java.util.stream.IntStream;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-
-public class SimpleGroupTrainerInput implements GroupTrainerInput<Double> {
-    /** */
-    private int limit;
-
-    /** */
-    private int eachNumberCount;
-
-    /** */
-    private int iterCnt;
-
-    /** */
-    public SimpleGroupTrainerInput(int limit, int eachNumCnt, int iterCnt) {
-        this.limit = limit;
-        this.eachNumberCount = eachNumCnt;
-        this.iterCnt = iterCnt;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSupplier<Stream<GroupTrainerCacheKey<Double>>> initialKeys(UUID trainingUUID) {
-        int lim = limit;
-        UUID uuid = trainingUUID;
-        return () -> IntStream.range(0, lim).mapToObj(i -> new GroupTrainerCacheKey<>(i, 0.0, uuid));
-    }
-
-    /** */
-    public int limit() {
-        return limit;
-    }
-
-    /** */
-    public int iterCnt() {
-        return iterCnt;
-    }
-
-    /** */
-    public int eachNumberCount() {
-        return eachNumberCount;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java
deleted file mode 100644
index 0a49fe0..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainer.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.Chains;
-import org.apache.ignite.ml.trainers.group.chain.ComputationsChain;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-
-/**
- * Test group trainer.
- */
-public class TestGroupTrainer extends GroupTrainer<TestGroupTrainerLocalContext, Double, Integer, Integer, Integer,
-    Double, ConstModel<Integer>, SimpleGroupTrainerInput, Void> {
-    /**
-     * Construct instance of this class with given parameters.
-     *
-     * @param ignite Ignite instance.
-     */
-    public TestGroupTrainer(Ignite ignite) {
-        super(TestGroupTrainingCache.getOrCreate(ignite), ignite);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected TestGroupTrainerLocalContext initialLocalContext(SimpleGroupTrainerInput data,
-        UUID trainingUUID) {
-        return new TestGroupTrainerLocalContext(data.iterCnt(), data.eachNumberCount(), data.limit(), trainingUUID);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteFunction<GroupTrainerCacheKey<Double>, ResultAndUpdates<Integer>> distributedInitializer(
-        SimpleGroupTrainerInput data) {
-        return key -> {
-            long i = key.nodeLocalEntityIndex();
-            UUID trainingUUID = key.trainingUUID();
-            IgniteCache<GroupTrainerCacheKey<Double>, Integer> cache
-                = TestGroupTrainingCache.getOrCreate(Ignition.localIgnite());
-
-            long sum = i * data.eachNumberCount();
-
-            ResultAndUpdates<Integer> res = ResultAndUpdates.of((int)sum);
-
-            for (int j = 0; j < data.eachNumberCount(); j++)
-                res.updateCache(cache, new GroupTrainerCacheKey<>(i, (double)j, trainingUUID), (int)i);
-
-            return res;
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteFunction<List<Integer>, Integer> reduceDistributedInitData() {
-        return id -> id.stream().mapToInt(x -> x).sum();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected Double locallyProcessInitData(Integer data, TestGroupTrainerLocalContext locCtx) {
-        return data.doubleValue();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected ComputationsChain<TestGroupTrainerLocalContext,
-        Double, Integer, Double, Double> trainingLoopStep() {
-        // TODO:IGNITE-7405 here we should explicitly create variable because we cannot infer context type, think about it.
-        ComputationsChain<TestGroupTrainerLocalContext, Double, Integer, Double, Double> chain = Chains.
-            create(new TestTrainingLoopStep());
-        return chain.
-            thenLocally((aDouble, context) -> {
-                context.incCnt();
-                return aDouble;
-            });
-    }
-
-    /** {@inheritDoc} */
-    @Override protected boolean shouldContinue(Double data, TestGroupTrainerLocalContext locCtx) {
-        return locCtx.cnt() < locCtx.maxCnt();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteSupplier<Void> extractContextForFinalResultCreation(Double data,
-        TestGroupTrainerLocalContext locCtx) {
-        // No context is needed.
-        return () -> null;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteSupplier<Stream<GroupTrainerCacheKey<Double>>> finalResultKeys(Double data,
-        TestGroupTrainerLocalContext locCtx) {
-        int limit = locCtx.limit();
-        int cnt = locCtx.eachNumberCnt();
-        UUID uuid = locCtx.trainingUUID();
-
-        return () -> TestGroupTrainingCache.allKeys(limit, cnt, uuid);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteFunction<EntryAndContext<Double, Integer, Void>,
-        ResultAndUpdates<Integer>> finalResultsExtractor() {
-        return entryAndCtx -> {
-            Integer val = entryAndCtx.entry().getValue();
-            return ResultAndUpdates.of(val % 2 == 0 ? val : 0);
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteFunction<List<Integer>, Integer> finalResultsReducer() {
-        return id -> id.stream().mapToInt(x -> x).sum();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected ConstModel<Integer> mapFinalResult(Integer res, TestGroupTrainerLocalContext locCtx) {
-        return new ConstModel<>(res);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void cleanup(TestGroupTrainerLocalContext locCtx) {
-        Stream<GroupTrainerCacheKey<Double>> toRemote = TestGroupTrainingCache.allKeys(locCtx.limit(),
-            locCtx.eachNumberCnt(), locCtx.trainingUUID());
-
-        TestGroupTrainingCache.getOrCreate(ignite).removeAll(toRemote.collect(Collectors.toSet()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java
deleted file mode 100644
index e1a533b..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainerLocalContext.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.UUID;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/** */
-public class TestGroupTrainerLocalContext implements HasTrainingUUID {
-    /** */
-    private int cnt = 0;
-
-    /** */
-    private int maxCnt;
-
-    /** */
-    private int eachNumberCnt;
-
-    /** */
-    private int limit;
-
-    /** */
-    private UUID trainingUUID;
-
-    /** */
-    public TestGroupTrainerLocalContext(int maxCnt, int eachNumberCnt, int limit, UUID trainingUUID) {
-        this.maxCnt = maxCnt;
-        this.eachNumberCnt = eachNumberCnt;
-        this.limit = limit;
-        this.trainingUUID = trainingUUID;
-        this.cnt = 0;
-    }
-
-    /** */
-    public int cnt() {
-        return cnt;
-    }
-
-    /** */
-    public void setCnt(int cnt) {
-        this.cnt = cnt;
-    }
-
-    /** */
-    public TestGroupTrainerLocalContext incCnt() {
-        this.cnt++;
-
-        return this;
-    }
-
-    /** */
-    public int maxCnt() {
-        return maxCnt;
-    }
-
-    /** */
-    public int eachNumberCnt() {
-        return eachNumberCnt;
-    }
-
-    /** */
-    public int limit() {
-        return limit;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID trainingUUID() {
-        return trainingUUID;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java
deleted file mode 100644
index afee674..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingCache.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.Arrays;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/** */
-public class TestGroupTrainingCache {
-    /** */
-    public static String CACHE_NAME = "TEST_GROUP_TRAINING_CACHE";
-
-    /** */
-    public static IgniteCache<GroupTrainerCacheKey<Double>, Integer> getOrCreate(Ignite ignite) {
-        CacheConfiguration<GroupTrainerCacheKey<Double>, Integer> cfg = new CacheConfiguration<>();
-
-        // Write to primary.
-        cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
-
-        // Atomic transactions only.
-        cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-
-        // No copying of values.
-        cfg.setCopyOnRead(false);
-
-        // Cache is partitioned.
-        cfg.setCacheMode(CacheMode.PARTITIONED);
-
-        cfg.setBackups(0);
-
-        cfg.setOnheapCacheEnabled(true);
-
-        cfg.setName(CACHE_NAME);
-
-        return ignite.getOrCreateCache(cfg);
-    }
-
-    /** */
-    public static Stream<GroupTrainerCacheKey<Double>> allKeys(int limit, int eachNumberCnt, UUID trainingUUID) {
-        GroupTrainerCacheKey<Double>[] a =new GroupTrainerCacheKey[limit * eachNumberCnt];
-
-        for (int num = 0; num < limit; num++)
-            for (int i = 0; i < eachNumberCnt; i++)
-                a[num * eachNumberCnt + i] = new GroupTrainerCacheKey<>(num, (double)i, trainingUUID);
-
-        return Arrays.stream(a);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java
deleted file mode 100644
index e16ed7c..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestGroupTrainingSecondCache.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CacheAtomicityMode;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.cache.CacheWriteSynchronizationMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-
-/** */
-public class TestGroupTrainingSecondCache {
-    /** */
-    public static String CACHE_NAME = "TEST_GROUP_TRAINING_SECOND_CACHE";
-
-    /** */
-    public static IgniteCache<GroupTrainerCacheKey<Character>, Integer> getOrCreate(Ignite ignite) {
-        CacheConfiguration<GroupTrainerCacheKey<Character>, Integer> cfg = new CacheConfiguration<>();
-
-        // Write to primary.
-        cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
-
-        // Atomic transactions only.
-        cfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
-
-        // No copying of values.
-        cfg.setCopyOnRead(false);
-
-        // Cache is partitioned.
-        cfg.setCacheMode(CacheMode.PARTITIONED);
-
-        cfg.setBackups(0);
-
-        cfg.setOnheapCacheEnabled(true);
-
-        cfg.setName(CACHE_NAME);
-
-        return ignite.getOrCreateCache(cfg);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java
deleted file mode 100644
index 3f0237f..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestLocalContext.java
+++ /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.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.UUID;
-import org.apache.ignite.ml.trainers.group.chain.HasTrainingUUID;
-
-/** */
-public class TestLocalContext implements HasTrainingUUID {
-    /** */
-    private final UUID trainingUUID;
-
-    /** */
-    private int data;
-
-    /** */
-    public TestLocalContext(int data, UUID trainingUUID) {
-        this.data = data;
-        this.trainingUUID = trainingUUID;
-    }
-
-    /** */
-    public int data() {
-        return data;
-    }
-
-    /** */
-    public void setData(int data) {
-        this.data = data;
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID trainingUUID() {
-        return trainingUUID;
-    }
-}


[23/50] [abbrv] ignite git commit: IGNITE-7871 Fixed condition for cache partitions validation. - Fixes #3804.

Posted by ag...@apache.org.
IGNITE-7871 Fixed condition for cache partitions validation. - Fixes #3804.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 7a1d0ea70127f73409071d7f9581a8207be97491
Parents: 706e7f8
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Thu Apr 12 14:29:43 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Thu Apr 12 14:29:43 2018 +0300

----------------------------------------------------------------------
 .../distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7a1d0ea7/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 dd4a571..af5acd6 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
@@ -2760,7 +2760,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                     || grpCtx.config().isReadThrough()
                     || grpCtx.config().isWriteThrough()
                     || grpCtx.config().getCacheStoreFactory() != null
-                    || grpCtx.config().getRebalanceDelay() != -1
+                    || grpCtx.config().getRebalanceDelay() == -1
                     || grpCtx.config().getRebalanceMode() == CacheRebalanceMode.NONE)
                 continue;
 


[38/50] [abbrv] ignite git commit: IGNITE-8232: ML package cleanup for 2.5 release

Posted by ag...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java
deleted file mode 100644
index caf92f6..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TestTrainingLoopStep.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import java.util.List;
-import java.util.UUID;
-import java.util.stream.Stream;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.ml.math.functions.IgniteFunction;
-import org.apache.ignite.ml.math.functions.IgniteSupplier;
-import org.apache.ignite.ml.trainers.group.chain.DistributedEntryProcessingStep;
-import org.apache.ignite.ml.trainers.group.chain.EntryAndContext;
-
-/** */
-public class TestTrainingLoopStep implements DistributedEntryProcessingStep<TestGroupTrainerLocalContext,
-    Double, Integer, Void, Double, Double> {
-    /** {@inheritDoc} */
-    @Override public IgniteSupplier<Void> remoteContextSupplier(Double input, TestGroupTrainerLocalContext locCtx) {
-        // No context is needed.
-        return () -> null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFunction<EntryAndContext<Double, Integer, Void>, ResultAndUpdates<Double>> worker() {
-        return entryAndContext -> {
-            Integer oldVal = entryAndContext.entry().getValue();
-            double v = oldVal * oldVal;
-            ResultAndUpdates<Double> res = ResultAndUpdates.of(v);
-            res.updateCache(TestGroupTrainingCache.getOrCreate(Ignition.localIgnite()),
-                entryAndContext.entry().getKey(), (int)v);
-            return res;
-        };
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteSupplier<Stream<GroupTrainerCacheKey<Double>>> keys(Double input,
-        TestGroupTrainerLocalContext locCtx) {
-        // Copying here because otherwise locCtx will be serialized with supplier returned in result.
-        int limit = locCtx.limit();
-        int cnt = locCtx.eachNumberCnt();
-        UUID uuid = locCtx.trainingUUID();
-
-        return () -> TestGroupTrainingCache.allKeys(limit, cnt, uuid);
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteFunction<List<Double>, Double> reducer() {
-        return doubles -> doubles.stream().mapToDouble(x -> x).sum();
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java
deleted file mode 100644
index 0ec5afb..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/trainers/group/TrainersGroupTestSuite.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.trainers.group;
-
-import org.junit.runner.RunWith;
-import org.junit.runners.Suite;
-
-/**
- * Test suite for group trainer tests.
- */
-@RunWith(Suite.class)
-@Suite.SuiteClasses({
-    DistributedWorkersChainTest.class,
-    GroupTrainerTest.class
-})
-public class TrainersGroupTestSuite {
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/47cfdc27/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
deleted file mode 100644
index 89b5471..0000000
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.yardstick.ml.regression;
-
-import java.util.Map;
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
-import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-
-/**
- * Ignite benchmark that performs ML Grid operations.
- */
-@SuppressWarnings("unused")
-public class IgniteOLSMultipleLinearRegressionBenchmark extends IgniteAbstractBenchmark {
-    /** {@inheritDoc} */
-    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
-        runLongly();
-
-        return true;
-    }
-
-    /**
-     * Based on OLSMultipleLinearRegressionTest#testLongly.
-     */
-    private void runLongly() {
-        // Y values are first, then independent vars
-        // Each row is one observation
-        double[][] data = new double[][] {
-            { 60323, 83.0, 234289, 2356, 1590, 107608, 1947 },
-            { 61122, 88.5, 259426, 2325, 1456, 108632, 1948 },
-            { 60171, 88.2, 258054, 3682, 1616, 109773, 1949 },
-            { 61187, 89.5, 284599, 3351, 1650, 110929, 1950 },
-            { 63221, 96.2, 328975, 2099, 3099, 112075, 1951 },
-            { 63639, 98.1, 346999, 1932, 3594, 113270, 1952 },
-            { 64989, 99.0, 365385, 1870, 3547, 115094, 1953 },
-            { 63761, 100.0, 363112, 3578, 3350, 116219, 1954 },
-            { 66019, 101.2, 397469, 2904, 3048, 117388, 1955 },
-            { 67857, 104.6, 419180, 2822, 2857, 118734, 1956 },
-            { 68169, 108.4, 442769, 2936, 2798, 120445, 1957 },
-            { 66513, 110.8, 444546, 4681, 2637, 121950, 1958 },
-            { 68655, 112.6, 482704, 3813, 2552, 123366, 1959 },
-            { 69564, 114.2, 502601, 3931, 2514, 125368, 1960 },
-            { 69331, 115.7, 518173, 4806, 2572, 127852, 1961 },
-            { 70551, 116.9, 554894, 4007, 2827, 130081, 1962 }
-        };
-
-        final int nobs = 16;
-        final int nvars = 6;
-
-        LinearRegressionQRTrainer trainer = new LinearRegressionQRTrainer();
-        LinearRegressionModel model = trainer.train(new DenseLocalOnHeapMatrix(data));
-    }
-}

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


[35/50] [abbrv] ignite git commit: IGNITE-8248 Fixed npe in Web Console agent int case of self-signed certificate.

Posted by ag...@apache.org.
IGNITE-8248 Fixed npe in Web Console agent int case of self-signed certificate.


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

Branch: refs/heads/ignite-7708
Commit: 8a42f64a38422304f62f263623c4517f469f8ddb
Parents: 56e3f43
Author: Andrey Novikov <an...@gridgain.com>
Authored: Fri Apr 13 18:08:35 2018 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Apr 13 18:08:35 2018 +0700

----------------------------------------------------------------------
 .../org/apache/ignite/console/agent/AgentLauncher.java   | 11 +++++------
 1 file changed, 5 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/8a42f64a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
----------------------------------------------------------------------
diff --git a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
index 4db26ba..385ce08 100644
--- a/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
+++ b/modules/web-console/web-agent/src/main/java/org/apache/ignite/console/agent/AgentLauncher.java
@@ -32,6 +32,7 @@ import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
 import java.net.UnknownHostException;
+import java.security.cert.X509Certificate;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
@@ -122,18 +123,16 @@ public class AgentLauncher {
         return new TrustManager[] {
             new X509TrustManager() {
                 /** {@inheritDoc} */
-                @Override public java.security.cert.X509Certificate[] getAcceptedIssuers() {
-                    return null;
+                @Override public X509Certificate[] getAcceptedIssuers() {
+                    return new X509Certificate[0];
                 }
 
                 /** {@inheritDoc} */
-                @Override public void checkClientTrusted(
-                    java.security.cert.X509Certificate[] certs, String authType) {
+                @Override public void checkClientTrusted(X509Certificate[] certs, String authType) {
                 }
 
                 /** {@inheritDoc} */
-                @Override public void checkServerTrusted(
-                    java.security.cert.X509Certificate[] certs, String authType) {
+                @Override public void checkServerTrusted(X509Certificate[] certs, String authType) {
                 }
             }};
     }


[34/50] [abbrv] ignite git commit: IGNITE-8245 Fixed input appearance position with error.

Posted by ag...@apache.org.
IGNITE-8245 Fixed input appearance position with error.


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

Branch: refs/heads/ignite-7708
Commit: 56e3f43e8b31d13e3a7cc590208ffe92187ad00d
Parents: 5fd5968
Author: Dmitriy Shabalin <ds...@gridgain.com>
Authored: Fri Apr 13 17:55:02 2018 +0700
Committer: Andrey Novikov <an...@gridgain.com>
Committed: Fri Apr 13 17:55:02 2018 +0700

----------------------------------------------------------------------
 .../frontend/app/primitives/form-field/index.scss       | 12 ++++++++++++
 1 file changed, 12 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/56e3f43e/modules/web-console/frontend/app/primitives/form-field/index.scss
----------------------------------------------------------------------
diff --git a/modules/web-console/frontend/app/primitives/form-field/index.scss b/modules/web-console/frontend/app/primitives/form-field/index.scss
index 7d9ea1f..1035ade 100644
--- a/modules/web-console/frontend/app/primitives/form-field/index.scss
+++ b/modules/web-console/frontend/app/primitives/form-field/index.scss
@@ -240,6 +240,18 @@
                 box-shadow: none;
             }
         }
+
+        // Added right offset to appearance of input for invalid password
+        & > input[type='password'].ng-invalid.ng-touched {
+            padding-right: 36px;
+        }
+
+        // Added right offset to appearance of dropdown for invalid data
+        & > button.select-toggle.ng-invalid.ng-touched {
+            &:after {
+                right: 36px;
+            }
+        }
     }
 
     &__errors {


[47/50] [abbrv] ignite git commit: IGNITE-8097: Java thin client: throw handshake exception eagerly on connect phase in case of failure. This closes #3822.

Posted by ag...@apache.org.
IGNITE-8097: Java thin client: throw handshake exception eagerly on connect phase in case of failure. This closes #3822.


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

Branch: refs/heads/ignite-7708
Commit: 7586cfd50e01fd7d781ad652df23734a7ebeb54d
Parents: af2b742
Author: Alexey Kukushkin <al...@yahoo.com>
Authored: Mon Apr 16 11:47:19 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Mon Apr 16 11:47:19 2018 +0300

----------------------------------------------------------------------
 .../internal/client/thin/ReliableChannel.java   |  2 ++
 .../apache/ignite/client/FunctionalTest.java    | 37 ++++++++++++++++----
 .../org/apache/ignite/client/SecurityTest.java  | 22 ++++++------
 3 files changed, 42 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7586cfd5/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
index 392b8f8..dac4320 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ReliableChannel.java
@@ -81,6 +81,8 @@ final class ReliableChannel implements AutoCloseable {
 
         primary = addrs.get(new Random().nextInt(addrs.size())); // we already verified there is at least one address
 
+        ch = chFactory.apply(new ClientChannelConfiguration(clientCfg).setAddress(primary)).get();
+
         for (InetSocketAddress a : addrs)
             if (a != primary)
                 this.backups.add(a);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7586cfd5/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
index d69ac4d..b49f7e3 100644
--- a/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/client/FunctionalTest.java
@@ -39,21 +39,16 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.PartitionLossPolicy;
 import org.apache.ignite.cache.QueryEntity;
 import org.apache.ignite.cache.QueryIndex;
-import org.apache.ignite.client.ClientCache;
-import org.apache.ignite.client.ClientCacheConfiguration;
-import org.apache.ignite.client.Comparers;
-import org.apache.ignite.client.Config;
-import org.apache.ignite.client.IgniteClient;
 import org.apache.ignite.configuration.ClientConfiguration;
-import org.apache.ignite.client.LocalIgniteCluster;
-import org.apache.ignite.client.Person;
 import org.junit.Test;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 /**
  * Thin client functional tests.
@@ -380,6 +375,34 @@ public class FunctionalTest {
         }
     }
 
+    /**
+     * Test client fails on start if server is unavailable
+     */
+    @Test
+    public void testClientFailsOnStart() {
+        ClientConnectionException expEx = null;
+
+        try (IgniteClient ignored = Ignition.startClient(getClientConfiguration())) {
+            // No-op.
+        }
+        catch (ClientConnectionException connEx) {
+            expEx = connEx;
+        }
+        catch (Exception ex) {
+            fail(String.format(
+                "%s expected but %s was received: %s",
+                ClientConnectionException.class.getName(),
+                ex.getClass().getName(),
+                ex
+            ));
+        }
+
+        assertNotNull(
+            String.format("%s expected but no exception was received", ClientConnectionException.class.getName()),
+            expEx
+        );
+    }
+
     /** */
     private static ClientConfiguration getClientConfiguration() {
         return new ClientConfiguration()

http://git-wip-us.apache.org/repos/asf/ignite/blob/7586cfd5/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java b/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java
index dc57f0c..e2b11db 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/client/SecurityTest.java
@@ -127,25 +127,23 @@ public class SecurityTest {
 
     /** Test valid user authentication. */
     @Test
-    public void testInvalidUserAuthentication() throws Exception {
+    public void testInvalidUserAuthentication() {
+        Exception authError = null;
+
         try (Ignite ignored = igniteWithAuthentication();
              IgniteClient client = Ignition.startClient(new ClientConfiguration().setAddresses(Config.SERVER)
                  .setUserName("JOE")
                  .setUserPassword("password")
              )
         ) {
-            Exception authError = null;
-
-            try {
-                client.getOrCreateCache("testAuthentication");
-            }
-            catch (Exception e) {
-                authError = e;
-            }
-
-            assertNotNull("Authentication with invalid credentials succeeded", authError);
-            assertTrue("Invalid type of authentication error", authError instanceof ClientAuthenticationException);
+            client.getOrCreateCache("testAuthentication");
+        }
+        catch (Exception e) {
+            authError = e;
         }
+
+        assertNotNull("Authentication with invalid credentials succeeded", authError);
+        assertTrue("Invalid type of authentication error", authError instanceof ClientAuthenticationException);
     }
 
     /** Test valid user authentication. */


[16/50] [abbrv] ignite git commit: IGNITE-6679 Clean up some deprecated cache metrics

Posted by ag...@apache.org.
IGNITE-6679 Clean up some deprecated cache metrics

Signed-off-by: Anton Vinogradov <av...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 32fc6c3c1b013c8477acb9cf51d8326a13307c64
Parents: 14402e4
Author: NSAmelchev <ns...@gmail.com>
Authored: Wed Apr 11 16:59:59 2018 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Apr 11 16:59:59 2018 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/cache/CacheMetrics.java   | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/32fc6c3c/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
index 0b1cb87..c466bee 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheMetrics.java
@@ -262,6 +262,7 @@ public interface CacheMetrics {
      *
      * @return Current size of evict queue.
      */
+    @Deprecated
     public int getDhtEvictQueueCurrentSize();
 
     /**
@@ -283,6 +284,7 @@ public interface CacheMetrics {
      *
      * @return Committed transaction queue size.
      */
+    @Deprecated
     public int getTxCommitQueueSize();
 
     /**
@@ -290,6 +292,7 @@ public interface CacheMetrics {
      *
      * @return Prepared transaction queue size.
      */
+    @Deprecated
     public int getTxPrepareQueueSize();
 
     /**
@@ -297,6 +300,7 @@ public interface CacheMetrics {
      *
      * @return Start version counts map size.
      */
+    @Deprecated
     public int getTxStartVersionCountsSize();
 
     /**
@@ -332,6 +336,7 @@ public interface CacheMetrics {
      *
      * @return Committed DHT transaction queue size.
      */
+    @Deprecated
     public int getTxDhtCommitQueueSize();
 
     /**
@@ -339,6 +344,7 @@ public interface CacheMetrics {
      *
      * @return Prepared DHT transaction queue size.
      */
+    @Deprecated
     public int getTxDhtPrepareQueueSize();
 
     /**
@@ -346,6 +352,7 @@ public interface CacheMetrics {
      *
      * @return DHT start version counts map size.
      */
+    @Deprecated
     public int getTxDhtStartVersionCountsSize();
 
     /**


[08/50] [abbrv] ignite git commit: IGNITE-8111 Add extra validation for WAL segment size - Fixes #3768.

Posted by ag...@apache.org.
IGNITE-8111 Add extra validation for WAL segment size - Fixes #3768.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/ignite-7708
Commit: 975246687c9d143830501340e597a35d1a4c492a
Parents: a4653b7
Author: denis.garus <d....@isimplelab.com>
Authored: Wed Apr 11 13:01:22 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Wed Apr 11 13:01:22 2018 +0300

----------------------------------------------------------------------
 .../configuration/DataStorageConfiguration.java |  6 ++--
 .../DataStorageConfigurationValidationTest.java | 33 ++++++++++++++++++--
 .../db/wal/IgniteWalFlushFailoverTest.java      |  4 +--
 ...lFlushMultiNodeFailoverAbstractSelfTest.java |  4 +--
 4 files changed, 39 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/97524668/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
index a433760..747efd8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/DataStorageConfiguration.java
@@ -533,12 +533,14 @@ public class DataStorageConfiguration implements Serializable {
 
     /**
      * Sets size of a WAL segment.
+     * If value is not set (or zero), {@link #DFLT_WAL_SEGMENT_SIZE} will be used.
      *
-     * @param walSegmentSize WAL segment size. 64 MB is used by default.  Maximum value is 2Gb.
+     * @param walSegmentSize WAL segment size. Value must be between 512Kb and 2Gb.
      * @return {@code This} for chaining.
      */
     public DataStorageConfiguration setWalSegmentSize(int walSegmentSize) {
-        A.ensure(walSegmentSize >= 0, "WAL segment size must be non-negative and less than 2 Gb.");
+        if (walSegmentSize != 0)
+            A.ensure(walSegmentSize >= 512 * 1024, "WAL segment size must be between 512Kb and 2Gb.");
 
         this.walSegmentSize = walSegmentSize;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/97524668/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DataStorageConfigurationValidationTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DataStorageConfigurationValidationTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DataStorageConfigurationValidationTest.java
index 7f667ee..9471a82 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DataStorageConfigurationValidationTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/DataStorageConfigurationValidationTest.java
@@ -31,11 +31,10 @@ public class DataStorageConfigurationValidationTest extends TestCase {
      *
      * @throws Exception If failed.
      */
-    public void testWalSegmentSizeOveflow() throws Exception {
+    public void testWalSegmentSizeOverflow() throws Exception {
         final DataStorageConfiguration cfg = new DataStorageConfiguration();
 
         GridTestUtils.assertThrows(null, new Callable<Void>() {
-            /** {@inheritDoc} */
             @Override public Void call() {
                 cfg.setWalSegmentSize(1 << 31);
 
@@ -43,4 +42,34 @@ public class DataStorageConfigurationValidationTest extends TestCase {
             }
         }, IllegalArgumentException.class, null);
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetWalSegmentSizeShouldThrowExceptionWhenSizeLessThen512Kb() throws Exception {
+        final DataStorageConfiguration cfg = new DataStorageConfiguration();
+
+        GridTestUtils.assertThrows(null, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cfg.setWalSegmentSize(512 * 1024 - 1);
+
+                return null;
+            }
+        }, IllegalArgumentException.class, null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetWalSegmentSizeShouldBeOkWhenSizeBetween512KbAnd2Gb() throws Exception {
+        final DataStorageConfiguration cfg = new DataStorageConfiguration();
+
+        cfg.setWalSegmentSize(512 * 1024);
+
+        assertEquals(512 * 1024, cfg.getWalSegmentSize());
+
+        cfg.setWalSegmentSize(Integer.MAX_VALUE);
+
+        assertEquals(Integer.MAX_VALUE, cfg.getWalSegmentSize());
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/97524668/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
index 042a447..351a42c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushFailoverTest.java
@@ -92,8 +92,8 @@ public class IgniteWalFlushFailoverTest extends GridCommonAbstractTest {
                 .setDefaultDataRegionConfiguration(
                  new DataRegionConfiguration().setMaxSize(2048L * 1024 * 1024).setPersistenceEnabled(true))
                 .setWalMode(WALMode.BACKGROUND)
-                .setWalBufferSize(128 * 1024)// Setting WAL Segment size to high values forces flushing by timeout.
-                .setWalSegmentSize(flushByTimeout ? 500_000 : 50_000);
+                .setWalBufferSize(1024 * 1024)// Setting WAL Segment size to high values forces flushing by timeout.
+                .setWalSegmentSize(flushByTimeout ? 2 * 1024 * 1024 : 512 * 1024);
 
         cfg.setDataStorageConfiguration(memCfg);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/97524668/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java
index fe16328..cc0986a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/db/wal/IgniteWalFlushMultiNodeFailoverAbstractSelfTest.java
@@ -106,8 +106,8 @@ public abstract class IgniteWalFlushMultiNodeFailoverAbstractSelfTest extends Gr
                 .setDefaultDataRegionConfiguration(
                         new DataRegionConfiguration().setMaxSize(2048L * 1024 * 1024).setPersistenceEnabled(true))
                 .setWalMode(this.walMode())
-                .setWalSegmentSize(50_000)
-                .setWalBufferSize(50_000);
+                .setWalSegmentSize(512 * 1024)
+                .setWalBufferSize(512 * 1024);
 
         cfg.setDataStorageConfiguration(memCfg);
 


[14/50] [abbrv] ignite git commit: IGNITE-8221: Security for thin clients.

Posted by ag...@apache.org.
IGNITE-8221: Security for thin clients.


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

Branch: refs/heads/ignite-7708
Commit: 5a29276355c4eb8966e5825883e1232ee2a80509
Parents: 747e6c5
Author: Alexey Kukushkin <al...@yahoo.com>
Authored: Wed Apr 11 16:29:07 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Wed Apr 11 16:38:12 2018 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  6 +++
 .../client/ClientAuthenticationException.java   |  2 +-
 .../client/ClientAuthorizationException.java    | 46 ++++++++++++++++++++
 .../internal/client/thin/ClientChannel.java     |  3 +-
 .../internal/client/thin/TcpClientChannel.java  | 39 ++++++++---------
 .../IgniteAuthenticationProcessor.java          |  5 ++-
 .../processors/cache/GridCacheProcessor.java    | 32 ++++++++++++++
 .../processors/cache/GridCacheUtils.java        |  5 +++
 .../client/ClientConnectionContext.java         | 45 ++++++++++++++++++-
 .../platform/client/ClientRequest.java          | 29 ++++++++++++
 .../platform/client/ClientStatus.java           |  3 ++
 .../cache/ClientCacheClearKeyRequest.java       |  3 ++
 .../cache/ClientCacheClearKeysRequest.java      |  3 ++
 .../client/cache/ClientCacheClearRequest.java   |  3 ++
 .../cache/ClientCacheContainsKeyRequest.java    |  3 ++
 .../cache/ClientCacheContainsKeysRequest.java   |  3 ++
 ...ientCacheCreateWithConfigurationRequest.java |  6 ++-
 .../cache/ClientCacheCreateWithNameRequest.java |  3 ++
 .../client/cache/ClientCacheDestroyRequest.java |  3 ++
 .../client/cache/ClientCacheGetAllRequest.java  |  3 ++
 .../ClientCacheGetAndPutIfAbsentRequest.java    |  3 ++
 .../cache/ClientCacheGetAndPutRequest.java      |  3 ++
 .../cache/ClientCacheGetAndRemoveRequest.java   |  3 ++
 .../cache/ClientCacheGetAndReplaceRequest.java  |  3 ++
 ...acheGetOrCreateWithConfigurationRequest.java |  6 ++-
 .../ClientCacheGetOrCreateWithNameRequest.java  |  3 ++
 .../client/cache/ClientCacheGetRequest.java     |  3 ++
 .../client/cache/ClientCacheGetSizeRequest.java |  3 ++
 .../client/cache/ClientCachePutAllRequest.java  |  3 ++
 .../cache/ClientCachePutIfAbsentRequest.java    |  3 ++
 .../client/cache/ClientCachePutRequest.java     |  3 ++
 .../cache/ClientCacheRemoveAllRequest.java      |  3 ++
 .../cache/ClientCacheRemoveIfEqualsRequest.java |  3 ++
 .../cache/ClientCacheRemoveKeyRequest.java      |  3 ++
 .../cache/ClientCacheRemoveKeysRequest.java     |  3 ++
 .../ClientCacheReplaceIfEqualsRequest.java      |  3 ++
 .../client/cache/ClientCacheReplaceRequest.java |  3 ++
 .../client/cache/ClientCacheRequest.java        | 32 ++++++++++++++
 .../cache/ClientCacheScanQueryRequest.java      |  3 ++
 .../cache/ClientCacheSqlFieldsQueryRequest.java |  1 +
 .../cache/ClientCacheSqlQueryRequest.java       |  1 +
 .../plugin/security/AuthenticationContext.java  | 40 +++++++++++++++++
 .../plugin/security/SecurityPermission.java     | 11 ++++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 12 ++++-
 44 files changed, 371 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/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 04eb425..662338c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -835,6 +835,12 @@ public final class IgniteSystemProperties {
     public static final String IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER = "IGNITE_WAL_FSYNC_WITH_DEDICATED_WORKER";
 
     /**
+     * When set to {@code true}, on-heap cache cannot be enabled - see
+     * {@link CacheConfiguration#setOnheapCacheEnabled(boolean)}.
+     * Default is {@code false}.
+     */
+    public static final String IGNITE_DISABLE_ONHEAP_CACHE = "IGNITE_DISABLE_ONHEAP_CACHE";
+    /**
      * When set to {@code false}, loaded pages implementation is switched to previous version of implementation,
      * FullPageIdTable. {@code True} value enables 'Robin Hood hashing: backward shift deletion'.
      * Default is {@code true}.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
index dc39c7a..0c24db8 100644
--- a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthenticationException.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.client;
 
 /**
- * Indicates Ignite server the client is connected to closed the connection and no longer available.
+ * Indicates user name or password is invalid.
  */
 public class ClientAuthenticationException extends ClientException {
     /** Serial version uid. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java
new file mode 100644
index 0000000..cacede6
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/client/ClientAuthorizationException.java
@@ -0,0 +1,46 @@
+/*
+ * 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.client;
+
+/**
+ * Indicates user has no permission to perform operation.
+ */
+public class ClientAuthorizationException extends ClientException {
+    /** Serial version uid. */
+    private static final long serialVersionUID = 0L;
+
+    /** Message. */
+    private static final String MSG = "User is not authorized to perform this operation";
+
+    /**
+     * Default constructor.
+     */
+    public ClientAuthorizationException() {
+        super(MSG);
+    }
+
+    /**
+     * Constructs a new exception with the specified cause and a detail
+     * message of <tt>(cause==null ? null : cause.toString())</tt>.
+     *
+     * @param cause the cause.
+     */
+    public ClientAuthorizationException(Throwable cause) {
+        super(MSG, cause);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java
index 71502a4..eb62c80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/ClientChannel.java
@@ -22,6 +22,7 @@ import java.util.function.Function;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
 import org.apache.ignite.client.ClientConnectionException;
+import org.apache.ignite.client.ClientAuthorizationException;
 
 /**
  * Processing thin client requests and responses.
@@ -41,5 +42,5 @@ interface ClientChannel extends AutoCloseable {
      * @return Received operation payload or {@code null} if response has no payload.
      */
     public <T> T receive(ClientOperation op, long reqId, Function<BinaryInputStream, T> payloadReader)
-        throws ClientConnectionException;
+        throws ClientConnectionException, ClientAuthorizationException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
index 404793a..8e8294f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/thin/TcpClientChannel.java
@@ -50,6 +50,7 @@ import javax.net.ssl.TrustManager;
 import javax.net.ssl.TrustManagerFactory;
 import javax.net.ssl.X509TrustManager;
 import org.apache.ignite.client.ClientAuthenticationException;
+import org.apache.ignite.client.ClientAuthorizationException;
 import org.apache.ignite.client.ClientConnectionException;
 import org.apache.ignite.client.SslMode;
 import org.apache.ignite.client.SslProtocol;
@@ -62,6 +63,7 @@ import org.apache.ignite.internal.binary.streams.BinaryHeapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryInputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOffheapOutputStream;
 import org.apache.ignite.internal.binary.streams.BinaryOutputStream;
+import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 
 /**
  * Implements {@link ClientChannel} over TCP.
@@ -138,7 +140,8 @@ class TcpClientChannel implements ClientChannel {
 
     /** {@inheritDoc} */
     public <T> T receive(ClientOperation op, long reqId, Function<BinaryInputStream, T> payloadReader)
-        throws ClientConnectionException {
+        throws ClientConnectionException, ClientAuthorizationException {
+
         final int MIN_RES_SIZE = 8 + 4; // minimal response size: long (8 bytes) ID + int (4 bytes) status
 
         int resSize = new BinaryHeapInputStream(read(4)).readInt();
@@ -163,7 +166,12 @@ class TcpClientChannel implements ClientChannel {
 
             String err = new BinaryReaderExImpl(null, resIn, null, true).readString();
 
-            throw new ClientServerError(err, status, reqId);
+            switch (status) {
+                case ClientStatus.SECURITY_VIOLATION:
+                    throw new ClientAuthorizationException();
+                default:
+                    throw new ClientServerError(err, status, reqId);
+            }
         }
 
         if (resSize <= MIN_RES_SIZE || payloadReader == null)
@@ -539,16 +547,10 @@ class TcpClientChannel implements ClientChannel {
 
         /** */
         private static KeyStore loadKeyStore(String lb, String path, String type, char[] pwd) {
-            InputStream in = null;
+            KeyStore store;
 
             try {
-                KeyStore store = KeyStore.getInstance(type);
-
-                in = new FileInputStream(new File(path));
-
-                store.load(in, pwd);
-
-                return store;
+                store = KeyStore.getInstance(type);
             }
             catch (KeyStoreException e) {
                 throw new ClientError(
@@ -556,6 +558,13 @@ class TcpClientChannel implements ClientChannel {
                     e
                 );
             }
+
+            try (InputStream in = new FileInputStream(new File(path))) {
+
+                store.load(in, pwd);
+
+                return store;
+            }
             catch (FileNotFoundException e) {
                 throw new ClientError(String.format("%s key store file [%s] does not exist", lb, path), e);
             }
@@ -571,16 +580,6 @@ class TcpClientChannel implements ClientChannel {
             catch (IOException e) {
                 throw new ClientError(String.format("Could not read %s key store", lb), e);
             }
-            finally {
-                if (in != null) {
-                    try {
-                        in.close();
-                    }
-                    catch (IOException ignored) {
-                        // Fail silently
-                    }
-                }
-            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
index c025e8c..93b3a5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/authentication/IgniteAuthenticationProcessor.java
@@ -897,7 +897,10 @@ public class IgniteAuthenticationProcessor extends GridProcessorAdapter implemen
 
             // Can be empty on initial start of PDS cluster (default user will be created and stored after activate)
             if (!F.isEmpty(initUsrs.usrs)) {
-                users.clear();
+                if (users == null)
+                    users = new ConcurrentHashMap<>();
+                else
+                    users.clear();
 
                 for (User u : initUsrs.usrs)
                     users.put(u.name(), u);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/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 7edac73..3aa6603 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -149,6 +149,8 @@ import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.apache.ignite.mxbean.CacheGroupMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMBeanAware;
+import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag;
 import org.apache.ignite.spi.discovery.DiscoveryDataBag.GridDiscoveryData;
@@ -1126,6 +1128,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         CacheConfiguration cfg = cacheCtx.config();
 
+        if (cacheCtx.userCache())
+            authorizeCacheCreate(cacheCtx.name(), cfg);
+
         // Intentionally compare Boolean references using '!=' below to check if the flag has been explicitly set.
         if (cfg.isStoreKeepBinary() && cfg.isStoreKeepBinary() != CacheConfiguration.DFLT_STORE_KEEP_BINARY
             && !(ctx.config().getMarshaller() instanceof BinaryMarshaller))
@@ -3151,6 +3156,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Collection<DynamicCacheChangeRequest> sndReqs = new ArrayList<>(reqs.size());
 
         for (DynamicCacheChangeRequest req : reqs) {
+            authorizeCacheChange(req);
+
             DynamicCacheStartFuture fut = new DynamicCacheStartFuture(req.requestId());
 
             try {
@@ -3216,6 +3223,31 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Authorize dynamic cache management.
+     */
+    private void authorizeCacheChange(DynamicCacheChangeRequest req) {
+        if (req.cacheType() == null || req.cacheType() == CacheType.USER) {
+            if (req.stop())
+                ctx.security().authorize(req.cacheName(), SecurityPermission.CACHE_DESTROY, null);
+            else
+                authorizeCacheCreate(req.cacheName(), req.startCacheConfiguration());
+        }
+    }
+
+    /**
+     * Authorize start/create cache operation.
+     */
+    private void authorizeCacheCreate(String cacheName, CacheConfiguration cacheCfg) {
+        ctx.security().authorize(cacheName, SecurityPermission.CACHE_CREATE, null);
+
+        if (cacheCfg != null && cacheCfg.isOnheapCacheEnabled() &&
+            System.getProperty(IgniteSystemProperties.IGNITE_DISABLE_ONHEAP_CACHE, "false")
+                .toUpperCase().equals("TRUE")
+            )
+            throw new SecurityException("Authorization failed for enabling on-heap cache.");
+    }
+
+    /**
      * @return Non null exception if node is stopping or disconnected.
      */
     @Nullable private IgniteCheckedException checkNodeState() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/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 d672420..e244c75 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
@@ -98,6 +98,8 @@ import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteReducer;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.plugin.CachePluginConfiguration;
+import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.transactions.Transaction;
 import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
@@ -1290,6 +1292,9 @@ public class GridCacheUtils {
         if (e.getCause() instanceof NullPointerException)
             return (NullPointerException)e.getCause();
 
+        if (e.getCause() instanceof SecurityException)
+            return (SecurityException)e.getCause();
+
         C1<IgniteCheckedException, IgniteException> converter = U.getExceptionConverter(e.getClass());
 
         return converter != null ? new CacheException(converter.apply(e)) : new CacheException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
index 7ab2d33..061aab3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientConnectionContext.java
@@ -20,16 +20,24 @@ package org.apache.ignite.internal.processors.platform.client;
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.UUID;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.binary.BinaryReaderExImpl;
 import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
+import org.apache.ignite.internal.processors.authentication.IgniteAccessControlException;
 import org.apache.ignite.internal.processors.odbc.ClientListenerConnectionContext;
 import org.apache.ignite.internal.processors.odbc.ClientListenerMessageParser;
 import org.apache.ignite.internal.processors.odbc.ClientListenerProtocolVersion;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequestHandler;
 
 import java.util.concurrent.atomic.AtomicLong;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.plugin.security.AuthenticationContext;
+import org.apache.ignite.plugin.security.SecurityCredentials;
+
+import static org.apache.ignite.plugin.security.SecuritySubjectType.REMOTE_CLIENT;
 
 /**
  * Thin Client connection context.
@@ -62,6 +70,9 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
     /** Cursor counter. */
     private final AtomicLong curCnt = new AtomicLong();
 
+    /** Security context or {@code null} if security is disabled. */
+    private SecurityContext secCtx = null;
+
     /**
      * Ctor.
      *
@@ -129,7 +140,9 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
             }
         }
 
-        if (kernalCtx.authentication().enabled()) {
+        if (kernalCtx.security().enabled())
+            authCtx = thirdPartyAuthentication(user, pwd).authorizationContext();
+        else if (kernalCtx.authentication().enabled()) {
             if (user == null || user.length() == 0)
                 throw new IgniteCheckedException("Unauthenticated sessions are prohibited.");
 
@@ -179,4 +192,34 @@ public class ClientConnectionContext implements ClientListenerConnectionContext
     public void decrementCursors() {
         curCnt.decrementAndGet();
     }
+
+    /**
+     * @return Security context or {@code null} if security is disabled.
+     */
+    public SecurityContext securityContext() {
+        return secCtx;
+    }
+
+    /**
+     * Do 3-rd party authentication.
+     */
+    private AuthenticationContext thirdPartyAuthentication(String user, String pwd) throws IgniteCheckedException {
+        SecurityCredentials cred = new SecurityCredentials(user, pwd);
+
+        AuthenticationContext authCtx = new AuthenticationContext();
+
+        authCtx.subjectType(REMOTE_CLIENT);
+        authCtx.subjectId(UUID.randomUUID());
+        authCtx.nodeAttributes(Collections.emptyMap());
+        authCtx.credentials(cred);
+
+        secCtx = kernalCtx.security().authenticate(authCtx);
+
+        if (secCtx == null)
+            throw new IgniteAccessControlException(
+                String.format("The user name or password is incorrect [userName=%s]", user)
+            );
+
+        return authCtx;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
index 76823b5..799b3e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientRequest.java
@@ -19,6 +19,9 @@ package org.apache.ignite.internal.processors.platform.client;
 
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.internal.processors.odbc.ClientListenerRequest;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.plugin.security.SecurityException;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Thin client request.
@@ -58,4 +61,30 @@ public class ClientRequest implements ClientListenerRequest {
     public ClientResponse process(ClientConnectionContext ctx) {
         return new ClientResponse(reqId);
     }
+
+    /**
+     * Run the code with converting {@link SecurityException} to {@link IgniteClientException}.
+     */
+    protected static void runWithSecurityExceptionHandler(Runnable runnable) {
+        try {
+            runnable.run();
+        }
+        catch (SecurityException ex) {
+            throw new IgniteClientException(
+                ClientStatus.SECURITY_VIOLATION,
+                "Client is not authorized to perform this operation",
+                ex
+            );
+        }
+    }
+
+    /**
+     * Authorize for specified permission.
+     */
+    protected void authorize(ClientConnectionContext ctx, SecurityPermission perm) {
+        SecurityContext secCtx = ctx.securityContext();
+
+        if (secCtx != null)
+            runWithSecurityExceptionHandler(() -> ctx.kernalContext().security().authorize(null, perm, secCtx));
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
index e0049b4..b8dfb1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/ClientStatus.java
@@ -48,4 +48,7 @@ public final class ClientStatus {
 
     /** Resource does not exist. */
     public static final int RESOURCE_DOES_NOT_EXIST = 1011;
+
+    /** Resource does not exist. */
+    public static final int SECURITY_VIOLATION = 1012;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java
index 6bcbbe8..5f8e952 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeyRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Clear key request.
@@ -37,6 +38,8 @@ public class ClientCacheClearKeyRequest extends ClientCacheKeyRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         cache(ctx).clear(key());
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java
index 04eb7f6..d803f69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearKeysRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Clear keys request.
@@ -37,6 +38,8 @@ public class ClientCacheClearKeysRequest extends ClientCacheKeysRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         cache(ctx).clearAll(keys());
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java
index 0e5f20d..7b84522 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheClearRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache clear request.
@@ -37,6 +38,8 @@ public class ClientCacheClearRequest extends ClientCacheRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         cache(ctx).clear();
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java
index 8470828..386f448 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeyRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * ContainsKey request.
@@ -38,6 +39,8 @@ public class ClientCacheContainsKeyRequest extends ClientCacheKeyRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         boolean val = cache(ctx).containsKey(key());
 
         return new ClientBooleanResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java
index 41e1306..b5184bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheContainsKeysRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * ContainsKeys request.
@@ -38,6 +39,8 @@ public class ClientCacheContainsKeysRequest extends ClientCacheKeysRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         boolean val = cache(ctx).containsKeys(keys());
 
         return new ClientBooleanResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
index 4b4dcec..65f9784 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithConfigurationRequest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache create with configuration request.
@@ -47,8 +48,11 @@ public class ClientCacheCreateWithConfigurationRequest extends ClientRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_CREATE);
+
         try {
-            ctx.kernalContext().grid().createCache(cacheCfg);
+            // Use security exception handler since the code authorizes "enable on-heap cache" permission
+            runWithSecurityExceptionHandler(() -> ctx.kernalContext().grid().createCache(cacheCfg));
         } catch (CacheExistsException e) {
             throw new IgniteClientException(ClientStatus.CACHE_EXISTS, e.getMessage());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java
index 9155d76..cacf099 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheCreateWithNameRequest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache create with name request.
@@ -45,6 +46,8 @@ public class ClientCacheCreateWithNameRequest extends ClientRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_CREATE);
+
         try {
             ctx.kernalContext().grid().createCache(cacheName);
         } catch (CacheExistsException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java
index 6645a03..b6f85ee 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheDestroyRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache destroy request.
@@ -42,6 +43,8 @@ public class ClientCacheDestroyRequest extends ClientRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_DESTROY);
+
         String cacheName = ClientCacheRequest.cacheDescriptor(ctx, cacheId).cacheName();
 
         ctx.kernalContext().grid().destroyCache(cacheName);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java
index 2b33af1..a07305c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAllRequest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionCon
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 
 import java.util.Map;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * GetAll request.
@@ -39,6 +40,8 @@ public class ClientCacheGetAllRequest extends ClientCacheKeysRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         Map val = cache(ctx).getAll(keys());
 
         return new ClientCacheGetAllResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java
index 8360213..8713a21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutIfAbsentRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get and put if absent request.
@@ -38,6 +39,8 @@ public class ClientCacheGetAndPutIfAbsentRequest extends ClientCacheKeyValueRequ
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         Object res = cache(ctx).getAndPutIfAbsent(key(), val());
 
         return new ClientObjectResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java
index 7a540e8..dde5181 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndPutRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get and put request.
@@ -38,6 +39,8 @@ public class ClientCacheGetAndPutRequest extends ClientCacheKeyValueRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         Object res = cache(ctx).getAndPut(key(), val());
 
         return new ClientObjectResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java
index e4fd735..3b9dd4b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndRemoveRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get and remove request.
@@ -38,6 +39,8 @@ public class ClientCacheGetAndRemoveRequest extends ClientCacheKeyRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_REMOVE);
+
         Object val = cache(ctx).getAndRemove(key());
 
         return new ClientObjectResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java
index dba8639..8ba157a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetAndReplaceRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get and replace request.
@@ -38,6 +39,8 @@ public class ClientCacheGetAndReplaceRequest extends ClientCacheKeyValueRequest
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         Object res = cache(ctx).getAndReplace(key(), val());
 
         return new ClientObjectResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
index 267318a..48569b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithConfigurationRequest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get or create with configuration request.
@@ -47,8 +48,11 @@ public class ClientCacheGetOrCreateWithConfigurationRequest extends ClientReques
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_CREATE);
+
         try {
-            ctx.kernalContext().grid().getOrCreateCache(cacheCfg);
+            // Use security exception handler since the code authorizes "enable on-heap cache" permission
+            runWithSecurityExceptionHandler(() -> ctx.kernalContext().grid().getOrCreateCache(cacheCfg));
         } catch (CacheExistsException e) {
             throw new IgniteClientException(ClientStatus.CACHE_EXISTS, e.getMessage());
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java
index 94dd115..3c4ce7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetOrCreateWithNameRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache create with name request.
@@ -42,6 +43,8 @@ public class ClientCacheGetOrCreateWithNameRequest extends ClientRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_CREATE);
+
         ctx.kernalContext().grid().getOrCreateCache(cacheName);
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java
index 41558c2..dc17cbf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientObjectResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get request.
@@ -38,6 +39,8 @@ public class ClientCacheGetRequest extends ClientCacheKeyRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         Object val = cache(ctx).get(key());
 
         return new ClientObjectResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java
index ba185bf..474c206 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheGetSizeRequest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientLongResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache size request.
@@ -50,6 +51,8 @@ public class ClientCacheGetSizeRequest extends ClientCacheRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         long res = cache(ctx).sizeLong(modes);
 
         return new ClientLongResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java
index 28a7fa5..57e3144 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutAllRequest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 
 import java.util.LinkedHashMap;
 import java.util.Map;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * PutAll request.
@@ -50,6 +51,8 @@ public class ClientCachePutAllRequest extends ClientCacheRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_PUT);
+
         cache(ctx).putAll(map);
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java
index 4dd2cde..ec81bc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutIfAbsentRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache put if absent request.
@@ -38,6 +39,8 @@ public class ClientCachePutIfAbsentRequest extends ClientCacheKeyValueRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         boolean res = cache(ctx).putIfAbsent(key(), val());
 
         return new ClientBooleanResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java
index 2c396b7..116460e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCachePutRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache put request.
@@ -37,6 +38,8 @@ public class ClientCachePutRequest extends ClientCacheKeyValueRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_PUT);
+
         cache(ctx).put(key(), val());
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java
index f5adc63..d90d873 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveAllRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache removeAll request.
@@ -37,6 +38,8 @@ public class ClientCacheRemoveAllRequest extends ClientCacheRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         cache(ctx).removeAll();
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java
index b86f2f8..26c191f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveIfEqualsRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache remove request with value.
@@ -38,6 +39,8 @@ public class ClientCacheRemoveIfEqualsRequest extends ClientCacheKeyValueRequest
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_REMOVE);
+
         boolean res = cache(ctx).remove(key(), val());
 
         return new ClientBooleanResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java
index a68c327..5af9743 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeyRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Remove request.
@@ -38,6 +39,8 @@ public class ClientCacheRemoveKeyRequest extends ClientCacheKeyRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         boolean val = cache(ctx).remove(key());
 
         return new ClientBooleanResponse(requestId(), val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java
index 043b568..62dea00 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRemoveKeysRequest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.client.cache;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Remove keys request.
@@ -37,6 +38,8 @@ public class ClientCacheRemoveKeysRequest extends ClientCacheKeysRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_REMOVE);
+
         cache(ctx).removeAll(keys());
 
         return super.process(ctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java
index 8645fbb..056367d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceIfEqualsRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache replace request.
@@ -43,6 +44,8 @@ public class ClientCacheReplaceIfEqualsRequest extends ClientCacheKeyValueReques
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         boolean res = cache(ctx).replace(key(), val(), newVal);
 
         return new ClientBooleanResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java
index bd7a642..ea04593 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheReplaceRequest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.processors.platform.client.ClientBooleanResponse;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache replace request.
@@ -38,6 +39,8 @@ public class ClientCacheReplaceRequest extends ClientCacheKeyValueRequest {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ, SecurityPermission.CACHE_PUT);
+
         boolean res = cache(ctx).replace(key(), val());
 
         return new ClientBooleanResponse(requestId(), res);

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
index 52b799f..9e2d1f1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheRequest.java
@@ -24,6 +24,8 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionCon
 import org.apache.ignite.internal.processors.platform.client.ClientRequest;
 import org.apache.ignite.internal.processors.platform.client.ClientStatus;
 import org.apache.ignite.internal.processors.platform.client.IgniteClientException;
+import org.apache.ignite.internal.processors.security.SecurityContext;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Cache get request.
@@ -119,4 +121,34 @@ class ClientCacheRequest extends ClientRequest {
     protected int cacheId() {
         return cacheId;
     }
+
+    /** {@inheritDoc} */
+    protected void authorize(ClientConnectionContext ctx, SecurityPermission perm) {
+        SecurityContext secCtx = ctx.securityContext();
+
+        if (secCtx != null) {
+            DynamicCacheDescriptor cacheDesc = cacheDescriptor(ctx, cacheId);
+
+            runWithSecurityExceptionHandler(() -> {
+                ctx.kernalContext().security().authorize(cacheDesc.cacheName(), perm, secCtx);
+            });
+        }
+    }
+
+    /**
+     * Authorize for multiple permissions.
+     */
+    protected void authorize(ClientConnectionContext ctx, SecurityPermission... perm)
+        throws IgniteClientException {
+        SecurityContext secCtx = ctx.securityContext();
+
+        if (secCtx != null) {
+            DynamicCacheDescriptor cacheDesc = cacheDescriptor(ctx, cacheId);
+
+            runWithSecurityExceptionHandler(() -> {
+                for (SecurityPermission p : perm)
+                    ctx.kernalContext().security().authorize(cacheDesc.cacheName(), p, secCtx);
+            });
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java
index 26ab236..70b6966 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheScanQueryRequest.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionCon
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Scan query request.
@@ -80,6 +81,8 @@ public class ClientCacheScanQueryRequest extends ClientCacheRequest {
 
     /** {@inheritDoc} */
     @Override public ClientResponse process(ClientConnectionContext ctx) {
+        authorize(ctx, SecurityPermission.CACHE_READ);
+
         IgniteCache cache = filterPlatform == FILTER_PLATFORM_JAVA && !isKeepBinary() ? rawCache(ctx) : cache(ctx);
 
         ScanQuery qry = new ScanQuery()

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
index cfd4498..3aa95bf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlFieldsQueryRequest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.client.ClientConnectionContext;
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 import org.apache.ignite.internal.processors.query.QueryUtils;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Sql query request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java
index 8c21be1..40693e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheSqlQueryRequest.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.platform.client.ClientConnectionCon
 import org.apache.ignite.internal.processors.platform.client.ClientResponse;
 
 import java.util.concurrent.TimeUnit;
+import org.apache.ignite.plugin.security.SecurityPermission;
 
 /**
  * Sql query request.

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
index 91f3379..9f2cfe2 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/AuthenticationContext.java
@@ -21,6 +21,7 @@ import java.net.InetSocketAddress;
 import java.util.Collections;
 import java.util.Map;
 import java.util.UUID;
+import org.apache.ignite.internal.processors.authentication.AuthorizationContext;
 
 /**
  * Authentication context.
@@ -41,6 +42,12 @@ public class AuthenticationContext {
     /** */
     private Map<String, Object> nodeAttrs;
 
+    /** Authorization context. */
+    private AuthorizationContext athrCtx;
+
+    /** True if this is a client node context. */
+    private boolean client;
+
     /**
      * Gets subject type.
      *
@@ -130,4 +137,37 @@ public class AuthenticationContext {
     public void nodeAttributes(Map<String, Object> nodeAttrs) {
         this.nodeAttrs = nodeAttrs;
     }
+
+    /**
+     * @return Native Apache Ignite authorization context acquired after authentication or {@code null} if native
+     * Ignite authentication is not used.
+     */
+    public AuthorizationContext authorizationContext(){
+        return athrCtx;
+    }
+
+    /**
+     * Set authorization context acquired after native Apache Ignite authentication.
+     */
+    public AuthenticationContext authorizationContext(AuthorizationContext newVal) {
+        athrCtx = newVal;
+
+        return this;
+    }
+
+    /**
+     * @return {@code true} if this is a client node context.
+     */
+    public boolean isClient() {
+        return client;
+    }
+
+    /**
+     * Sets flag indicating if this is client node context.
+     */
+    public AuthenticationContext setClient(boolean newVal) {
+        client = newVal;
+
+        return this;
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
index 5436161..bca667d 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
@@ -64,7 +64,16 @@ public enum SecurityPermission {
     SERVICE_CANCEL,
 
     /** Service invoke permission. */
-    SERVICE_INVOKE;
+    SERVICE_INVOKE,
+
+    /** Cache create permission. */
+    CACHE_CREATE,
+
+    /** Cache create permission. */
+    CACHE_DESTROY,
+
+    /** Join as server node permission. */
+    JOIN_AS_SERVER;
 
     /** Enumerated values. */
     private static final SecurityPermission[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/5a292763/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 7bf37e1..6d3864e 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
@@ -99,6 +99,7 @@ import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.security.SecurityCredentials;
+import org.apache.ignite.plugin.security.SecurityPermission;
 import org.apache.ignite.plugin.security.SecurityPermissionSet;
 import org.apache.ignite.spi.IgniteNodeValidationResult;
 import org.apache.ignite.spi.IgniteSpiContext;
@@ -3559,6 +3560,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                             return;
                         }
                         else {
+                            String authFailedMsg = null;
+
                             if (!(subj instanceof Serializable)) {
                                 // Node has not pass authentication.
                                 LT.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() +
@@ -3567,9 +3570,16 @@ class ServerImpl extends TcpDiscoveryImpl {
                                         ", addrs=" +
                                         U.addressesAsString(node) + ']');
 
+                                authFailedMsg = "Authentication subject is not serializable";
+                            }
+                            else if (!node.isClient() &&
+                                !subj.systemOperationAllowed(SecurityPermission.JOIN_AS_SERVER))
+                                authFailedMsg = "Node is not authorised to join as a server node";
+
+                            if (authFailedMsg != null) {
                                 // Always output in debug.
                                 if (log.isDebugEnabled())
-                                    log.debug("Authentication subject is not serializable [nodeId=" + node.id() +
+                                    log.debug(authFailedMsg + " [nodeId=" + node.id() +
                                         ", addrs=" + U.addressesAsString(node));
 
                                 try {


[21/50] [abbrv] ignite git commit: IGNITE-8176: Integrate gradient descent linear regression with partition based dataset

Posted by ag...@apache.org.
IGNITE-8176: Integrate gradient descent linear regression with partition based dataset

this closes #3787


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

Branch: refs/heads/ignite-7708
Commit: df6356d5d1470337a6ea705a332cf07f1dce2222
Parents: 67023a8
Author: dmitrievanthony <dm...@gmail.com>
Authored: Thu Apr 12 11:16:22 2018 +0300
Committer: YuriBabak <y....@gmail.com>
Committed: Thu Apr 12 11:16:22 2018 +0300

----------------------------------------------------------------------
 .../ml/knn/KNNClassificationExample.java        |  11 +-
 .../examples/ml/nn/MLPTrainerExample.java       |   4 +-
 .../ml/preprocessing/NormalizationExample.java  |  17 +--
 ...nWithLSQRTrainerAndNormalizationExample.java |  23 ++--
 ...dLinearRegressionWithLSQRTrainerExample.java |  14 +--
 ...tedLinearRegressionWithQRTrainerExample.java |   9 +-
 ...edLinearRegressionWithSGDTrainerExample.java |  78 +++++++++---
 .../binary/SVMBinaryClassificationExample.java  |  11 +-
 .../SVMMultiClassClassificationExample.java     |  24 ++--
 ...ecisionTreeClassificationTrainerExample.java |   7 +-
 .../DecisionTreeRegressionTrainerExample.java   |   4 +-
 .../org/apache/ignite/ml/nn/Activators.java     |  20 ++++
 .../org/apache/ignite/ml/nn/MLPTrainer.java     |  46 ++++++--
 .../ml/preprocessing/PreprocessingTrainer.java  |  41 ++++++-
 .../normalization/NormalizationTrainer.java     |  35 ++++--
 .../linear/FeatureExtractorWrapper.java         |  55 +++++++++
 .../linear/LinearRegressionLSQRTrainer.java     |  38 +-----
 .../linear/LinearRegressionSGDTrainer.java      | 118 +++++++++++++------
 .../ignite/ml/trainers/DatasetTrainer.java      |  46 ++++++++
 .../ignite/ml/knn/KNNClassificationTest.java    |  20 ++--
 .../ignite/ml/nn/MLPTrainerIntegrationTest.java |  14 +--
 .../org/apache/ignite/ml/nn/MLPTrainerTest.java |  22 ++--
 .../MLPTrainerMnistIntegrationTest.java         |   7 +-
 .../ml/nn/performance/MLPTrainerMnistTest.java  |  11 +-
 .../normalization/NormalizationTrainerTest.java |  10 +-
 .../ml/regressions/RegressionsTestSuite.java    |  15 +--
 ...stributedLinearRegressionSGDTrainerTest.java |  35 ------
 ...stributedLinearRegressionSGDTrainerTest.java |  35 ------
 ...wareAbstractLinearRegressionTrainerTest.java |   3 +
 .../linear/LinearRegressionLSQRTrainerTest.java |  14 ++-
 .../linear/LinearRegressionSGDTrainerTest.java  |  94 +++++++++++++++
 .../LocalLinearRegressionSGDTrainerTest.java    |  35 ------
 .../ignite/ml/svm/SVMBinaryTrainerTest.java     |  11 +-
 .../ignite/ml/svm/SVMMultiClassTrainerTest.java |  11 +-
 ...reeClassificationTrainerIntegrationTest.java |   9 +-
 .../DecisionTreeClassificationTrainerTest.java  |  12 +-
 ...ionTreeRegressionTrainerIntegrationTest.java |   9 +-
 .../tree/DecisionTreeRegressionTrainerTest.java |  12 +-
 .../DecisionTreeMNISTIntegrationTest.java       |   7 +-
 .../tree/performance/DecisionTreeMNISTTest.java |  11 +-
 40 files changed, 612 insertions(+), 386 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
index f3cdbbe..39a8431 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/knn/KNNClassificationExample.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.examples.ml.knn;
 
-import java.util.Arrays;
-import java.util.UUID;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -27,7 +24,6 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
 import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
@@ -35,6 +31,10 @@ import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run kNN multi-class classification trainer over distributed dataset.
  *
@@ -56,7 +56,8 @@ public class KNNClassificationExample {
                 KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
                 KNNClassificationModel knnMdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    ignite,
+                    dataCache,
                     (k, v) -> Arrays.copyOfRange(v, 1, v.length),
                     (k, v) -> v[0]
                 ).withK(3)

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
index efa1ba7..ce44cc6 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/nn/MLPTrainerExample.java
@@ -23,7 +23,6 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.examples.ExampleNodeStartup;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.math.Matrix;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
 import org.apache.ignite.ml.nn.Activators;
@@ -99,7 +98,8 @@ public class MLPTrainerExample {
 
                 // Train neural network and get multilayer perceptron model.
                 MultilayerPerceptron mlp = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, trainingSet),
+                    ignite,
+                    trainingSet,
                     (k, v) -> new double[] {v.x, v.y},
                     (k, v) -> new double[] {v.lb}
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java
index e0bcd08..b2c4e12 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/preprocessing/NormalizationExample.java
@@ -17,21 +17,19 @@
 
 package org.apache.ignite.examples.ml.preprocessing;
 
-import java.util.Arrays;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.examples.ml.dataset.model.Person;
-import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.DatasetFactory;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.dataset.primitive.SimpleDataset;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
-import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor;
 import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer;
 
+import java.util.Arrays;
+
 /**
  * Example that shows how to use normalization preprocessor to normalize data.
  *
@@ -47,8 +45,6 @@ public class NormalizationExample {
 
             IgniteCache<Integer, Person> persons = createCache(ignite);
 
-            DatasetBuilder<Integer, Person> builder = new CacheBasedDatasetBuilder<>(ignite, persons);
-
             // Defines first preprocessor that extracts features from an upstream data.
             IgniteBiFunction<Integer, Person, double[]> featureExtractor = (k, v) -> new double[] {
                 v.getAge(),
@@ -56,14 +52,11 @@ public class NormalizationExample {
             };
 
             // Defines second preprocessor that normalizes features.
-            NormalizationPreprocessor<Integer, Person> preprocessor = new NormalizationTrainer<Integer, Person>()
-                .fit(builder, featureExtractor, 2);
+            IgniteBiFunction<Integer, Person, double[]> preprocessor = new NormalizationTrainer<Integer, Person>()
+                .fit(ignite, persons, featureExtractor);
 
             // Creates a cache based simple dataset containing features and providing standard dataset API.
-            try (SimpleDataset<?> dataset = DatasetFactory.createSimpleDataset(
-                builder,
-                preprocessor
-            )) {
+            try (SimpleDataset<?> dataset = DatasetFactory.createSimpleDataset(ignite, persons, preprocessor)) {
                 // Calculation of the mean value. This calculation will be performed in map-reduce manner.
                 double[] mean = dataset.mean();
                 System.out.println("Mean \n\t" + Arrays.toString(mean));

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
index 567a599..99e6577 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.examples.ml.regression.linear;
 
-import java.util.Arrays;
-import java.util.UUID;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -28,7 +25,7 @@ import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor;
 import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer;
@@ -36,6 +33,10 @@ import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run linear regression model over distributed matrix.
  *
@@ -119,21 +120,17 @@ public class DistributedLinearRegressionWithLSQRTrainerAndNormalizationExample {
                 NormalizationTrainer<Integer, double[]> normalizationTrainer = new NormalizationTrainer<>();
 
                 System.out.println(">>> Perform the training to get the normalization preprocessor.");
-                NormalizationPreprocessor<Integer, double[]> preprocessor = normalizationTrainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
-                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
-                    4
+                IgniteBiFunction<Integer, double[], double[]> preprocessor = normalizationTrainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length)
                 );
 
                 System.out.println(">>> Create new linear regression trainer object.");
                 LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
 
                 System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
-                    preprocessor,
-                    (k, v) -> v[0]
-                );
+                LinearRegressionModel mdl = trainer.fit(ignite, dataCache, preprocessor, (k, v) -> v[0]);
 
                 System.out.println(">>> Linear regression model: " + mdl);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
index a853092..25aec0c 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithLSQRTrainerExample.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.examples.ml.regression.linear;
 
-import java.util.Arrays;
-import java.util.UUID;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -27,13 +24,15 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainer;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run linear regression model over distributed matrix.
  *
@@ -108,7 +107,7 @@ public class DistributedLinearRegressionWithLSQRTrainerExample {
             // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
             // because we create ignite cache internally.
             IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                SparseDistributedMatrixExample.class.getSimpleName(), () -> {
+                DistributedLinearRegressionWithLSQRTrainerExample.class.getSimpleName(), () -> {
                 IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
 
                 System.out.println(">>> Create new linear regression trainer object.");
@@ -116,7 +115,8 @@ public class DistributedLinearRegressionWithLSQRTrainerExample {
 
                 System.out.println(">>> Perform the training to get the model.");
                 LinearRegressionModel mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    ignite,
+                    dataCache,
                     (k, v) -> Arrays.copyOfRange(v, 1, v.length),
                     (k, v) -> v[0]
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
index 2b45aa2..98d5e4e 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithQRTrainerExample.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.examples.ml.regression.linear;
 
-import java.util.Arrays;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
@@ -30,6 +29,8 @@ import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
 import org.apache.ignite.thread.IgniteThread;
 
+import java.util.Arrays;
+
 /**
  * Run linear regression model over distributed matrix.
  *
@@ -113,15 +114,15 @@ public class DistributedLinearRegressionWithQRTrainerExample {
                 Trainer<LinearRegressionModel, Matrix> trainer = new LinearRegressionQRTrainer();
 
                 System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel model = trainer.train(distributedMatrix);
-                System.out.println(">>> Linear regression model: " + model);
+                LinearRegressionModel mdl = trainer.train(distributedMatrix);
+                System.out.println(">>> Linear regression model: " + mdl);
 
                 System.out.println(">>> ---------------------------------");
                 System.out.println(">>> | Prediction\t| Ground Truth\t|");
                 System.out.println(">>> ---------------------------------");
                 for (double[] observation : data) {
                     Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length));
-                    double prediction = model.apply(inputs);
+                    double prediction = mdl.apply(inputs);
                     double groundTruth = observation[0];
                     System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
index f3b2655..44366e1 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionWithSGDTrainerExample.java
@@ -17,20 +17,26 @@
 
 package org.apache.ignite.examples.ml.regression.linear;
 
-import java.util.Arrays;
 import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.query.QueryCursor;
+import org.apache.ignite.cache.query.ScanQuery;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
+import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
 import org.apache.ignite.ml.regressions.linear.LinearRegressionSGDTrainer;
+import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run linear regression model over distributed matrix.
  *
@@ -104,28 +110,43 @@ public class DistributedLinearRegressionWithSGDTrainerExample {
             // Create IgniteThread, we must work with SparseDistributedMatrix inside IgniteThread
             // because we create ignite cache internally.
             IgniteThread igniteThread = new IgniteThread(ignite.configuration().getIgniteInstanceName(),
-                SparseDistributedMatrixExample.class.getSimpleName(), () -> {
+                DistributedLinearRegressionWithSGDTrainerExample.class.getSimpleName(), () -> {
 
-                // Create SparseDistributedMatrix, new cache will be created automagically.
-                System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread.");
-                SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(data);
+                IgniteCache<Integer, double[]> dataCache = getTestCache(ignite);
 
                 System.out.println(">>> Create new linear regression trainer object.");
-                Trainer<LinearRegressionModel, Matrix> trainer = new LinearRegressionSGDTrainer(100_000, 1e-12);
+                LinearRegressionSGDTrainer<?> trainer = new LinearRegressionSGDTrainer<>(new UpdatesStrategy<>(
+                    new RPropUpdateCalculator(),
+                    RPropParameterUpdate::sumLocal,
+                    RPropParameterUpdate::avg
+                ), 100000,  10, 100, 123L);
 
                 System.out.println(">>> Perform the training to get the model.");
-                LinearRegressionModel model = trainer.train(distributedMatrix);
-                System.out.println(">>> Linear regression model: " + model);
+                LinearRegressionModel mdl = trainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
+                    (k, v) -> v[0]
+                );
+
+                System.out.println(">>> Linear regression model: " + mdl);
 
                 System.out.println(">>> ---------------------------------");
                 System.out.println(">>> | Prediction\t| Ground Truth\t|");
                 System.out.println(">>> ---------------------------------");
-                for (double[] observation : data) {
-                    Vector inputs = new SparseDistributedVector(Arrays.copyOfRange(observation, 1, observation.length));
-                    double prediction = model.apply(inputs);
-                    double groundTruth = observation[0];
-                    System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
+
+                try (QueryCursor<Cache.Entry<Integer, double[]>> observations = dataCache.query(new ScanQuery<>())) {
+                    for (Cache.Entry<Integer, double[]> observation : observations) {
+                        double[] val = observation.getValue();
+                        double[] inputs = Arrays.copyOfRange(val, 1, val.length);
+                        double groundTruth = val[0];
+
+                        double prediction = mdl.apply(new DenseLocalOnHeapVector(inputs));
+
+                        System.out.printf(">>> | %.4f\t\t| %.4f\t\t|\n", prediction, groundTruth);
+                    }
                 }
+
                 System.out.println(">>> ---------------------------------");
             });
 
@@ -134,4 +155,23 @@ public class DistributedLinearRegressionWithSGDTrainerExample {
             igniteThread.join();
         }
     }
+
+    /**
+     * Fills cache with data and returns it.
+     *
+     * @param ignite Ignite instance.
+     * @return Filled Ignite Cache.
+     */
+    private static IgniteCache<Integer, double[]> getTestCache(Ignite ignite) {
+        CacheConfiguration<Integer, double[]> cacheConfiguration = new CacheConfiguration<>();
+        cacheConfiguration.setName("TEST_" + UUID.randomUUID());
+        cacheConfiguration.setAffinity(new RendezvousAffinityFunction(false, 10));
+
+        IgniteCache<Integer, double[]> cache = ignite.createCache(cacheConfiguration);
+
+        for (int i = 0; i < data.length; i++)
+            cache.put(i, data[i]);
+
+        return cache;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java
index f8bf521..ce37112 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/svm/binary/SVMBinaryClassificationExample.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.examples.ml.svm.binary;
 
-import java.util.Arrays;
-import java.util.UUID;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -27,12 +24,15 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationModel;
 import org.apache.ignite.ml.svm.SVMLinearBinaryClassificationTrainer;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run SVM binary-class classification model over distributed dataset.
  *
@@ -54,7 +54,8 @@ public class SVMBinaryClassificationExample {
                 SVMLinearBinaryClassificationTrainer trainer = new SVMLinearBinaryClassificationTrainer();
 
                 SVMLinearBinaryClassificationModel mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    ignite,
+                    dataCache,
                     (k, v) -> Arrays.copyOfRange(v, 1, v.length),
                     (k, v) -> v[0]
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java
index f8281e4..4054201 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/svm/multiclass/SVMMultiClassClassificationExample.java
@@ -17,9 +17,6 @@
 
 package org.apache.ignite.examples.ml.svm.multiclass;
 
-import java.util.Arrays;
-import java.util.UUID;
-import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
@@ -27,14 +24,17 @@ import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.ScanQuery;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.preprocessing.normalization.NormalizationPreprocessor;
 import org.apache.ignite.ml.preprocessing.normalization.NormalizationTrainer;
 import org.apache.ignite.ml.svm.SVMLinearMultiClassClassificationModel;
 import org.apache.ignite.ml.svm.SVMLinearMultiClassClassificationTrainer;
 import org.apache.ignite.thread.IgniteThread;
 
+import javax.cache.Cache;
+import java.util.Arrays;
+import java.util.UUID;
+
 /**
  * Run SVM multi-class classification trainer over distributed dataset to build two models:
  * one with normalization and one without normalization.
@@ -57,7 +57,8 @@ public class SVMMultiClassClassificationExample {
                 SVMLinearMultiClassClassificationTrainer trainer = new SVMLinearMultiClassClassificationTrainer();
 
                 SVMLinearMultiClassClassificationModel mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    ignite,
+                    dataCache,
                     (k, v) -> Arrays.copyOfRange(v, 1, v.length),
                     (k, v) -> v[0]
                 );
@@ -67,14 +68,15 @@ public class SVMMultiClassClassificationExample {
 
                 NormalizationTrainer<Integer, double[]> normalizationTrainer = new NormalizationTrainer<>();
 
-                NormalizationPreprocessor<Integer, double[]> preprocessor = normalizationTrainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
-                    (k, v) -> Arrays.copyOfRange(v, 1, v.length),
-                    5
+                IgniteBiFunction<Integer, double[], double[]> preprocessor = normalizationTrainer.fit(
+                    ignite,
+                    dataCache,
+                    (k, v) -> Arrays.copyOfRange(v, 1, v.length)
                 );
 
                 SVMLinearMultiClassClassificationModel mdlWithNormalization = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, dataCache),
+                    ignite,
+                    dataCache,
                     preprocessor,
                     (k, v) -> v[0]
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java
index cef6368..1ecf460 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeClassificationTrainerExample.java
@@ -17,17 +17,17 @@
 
 package org.apache.ignite.examples.ml.tree;
 
-import java.util.Random;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.tree.DecisionTreeClassificationTrainer;
 import org.apache.ignite.ml.tree.DecisionTreeNode;
 import org.apache.ignite.thread.IgniteThread;
 
+import java.util.Random;
+
 /**
  * Example of using distributed {@link DecisionTreeClassificationTrainer}.
  */
@@ -65,7 +65,8 @@ public class DecisionTreeClassificationTrainerExample {
 
                 // Train decision tree model.
                 DecisionTreeNode mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, trainingSet),
+                    ignite,
+                    trainingSet,
                     (k, v) -> new double[]{v.x, v.y},
                     (k, v) -> v.lb
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java
index 61ba5f9..19b15f3 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ml/tree/DecisionTreeRegressionTrainerExample.java
@@ -22,7 +22,6 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.tree.DecisionTreeNode;
 import org.apache.ignite.ml.tree.DecisionTreeRegressionTrainer;
 import org.apache.ignite.thread.IgniteThread;
@@ -61,7 +60,8 @@ public class DecisionTreeRegressionTrainerExample {
 
                 // Train decision tree model.
                 DecisionTreeNode mdl = trainer.fit(
-                    new CacheBasedDatasetBuilder<>(ignite, trainingSet),
+                    ignite,
+                    trainingSet,
                     (k, v) -> new double[] {v.x},
                     (k, v) -> v.y
                 );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java
index f05bde8..4c34cd2 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/Activators.java
@@ -58,4 +58,24 @@ public class Activators {
             return Math.max(val, 0);
         }
     };
+
+    /**
+     * Linear unit activation function.
+     */
+    public static IgniteDifferentiableDoubleToDoubleFunction LINEAR = new IgniteDifferentiableDoubleToDoubleFunction() {
+        /** {@inheritDoc} */
+        @Override public double differential(double pnt) {
+            return 1.0;
+        }
+
+        /**
+         * Differential of linear at pnt.
+         *
+         * @param pnt Point to differentiate at.
+         * @return Differential at pnt.
+         */
+        @Override public Double apply(double pnt) {
+            return pnt;
+        }
+    };
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
index 47d2022..fe955cb 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/nn/MLPTrainer.java
@@ -17,11 +17,6 @@
 
 package org.apache.ignite.ml.nn;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer;
 import org.apache.ignite.ml.dataset.Dataset;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.primitive.builder.context.EmptyContextBuilder;
@@ -37,17 +32,23 @@ import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.nn.initializers.RandomInitializer;
 import org.apache.ignite.ml.optimization.updatecalculators.ParameterUpdateCalculator;
+import org.apache.ignite.ml.trainers.MultiLabelDatasetTrainer;
 import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.ml.util.Utils;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+
 /**
  * Multilayer perceptron trainer based on partition based {@link Dataset}.
  *
  * @param <P> Type of model update used in this trainer.
  */
 public class MLPTrainer<P extends Serializable> implements MultiLabelDatasetTrainer<MultilayerPerceptron> {
-    /** Multilayer perceptron architecture that defines layers and activators. */
-    private final MLPArchitecture arch;
+    /** Multilayer perceptron architecture supplier that defines layers and activators. */
+    private final IgniteFunction<Dataset<EmptyContext, SimpleLabeledDatasetData>, MLPArchitecture> archSupplier;
 
     /** Loss function to be minimized during the training. */
     private final IgniteFunction<Vector, IgniteDifferentiableVectorToDoubleFunction> loss;
@@ -81,7 +82,25 @@ public class MLPTrainer<P extends Serializable> implements MultiLabelDatasetTrai
     public MLPTrainer(MLPArchitecture arch, IgniteFunction<Vector, IgniteDifferentiableVectorToDoubleFunction> loss,
         UpdatesStrategy<? super MultilayerPerceptron, P> updatesStgy, int maxIterations, int batchSize,
         int locIterations, long seed) {
-        this.arch = arch;
+        this(dataset -> arch, loss, updatesStgy, maxIterations, batchSize, locIterations, seed);
+    }
+
+    /**
+     * Constructs a new instance of multilayer perceptron trainer.
+     *
+     * @param archSupplier Multilayer perceptron architecture supplier that defines layers and activators.
+     * @param loss Loss function to be minimized during the training.
+     * @param updatesStgy Update strategy that defines how to update model parameters during the training.
+     * @param maxIterations Maximal number of iterations before the training will be stopped.
+     * @param batchSize Batch size (per every partition).
+     * @param locIterations Maximal number of local iterations before synchronization.
+     * @param seed Random initializer seed.
+     */
+    public MLPTrainer(IgniteFunction<Dataset<EmptyContext, SimpleLabeledDatasetData>, MLPArchitecture> archSupplier,
+        IgniteFunction<Vector, IgniteDifferentiableVectorToDoubleFunction> loss,
+        UpdatesStrategy<? super MultilayerPerceptron, P> updatesStgy, int maxIterations, int batchSize,
+        int locIterations, long seed) {
+        this.archSupplier = archSupplier;
         this.loss = loss;
         this.updatesStgy = updatesStgy;
         this.maxIterations = maxIterations;
@@ -94,13 +113,14 @@ public class MLPTrainer<P extends Serializable> implements MultiLabelDatasetTrai
     public <K, V> MultilayerPerceptron fit(DatasetBuilder<K, V> datasetBuilder,
         IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, double[]> lbExtractor) {
 
-        MultilayerPerceptron mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed));
-        ParameterUpdateCalculator<? super MultilayerPerceptron, P> updater = updatesStgy.getUpdatesCalculator();
-
         try (Dataset<EmptyContext, SimpleLabeledDatasetData> dataset = datasetBuilder.build(
             new EmptyContextBuilder<>(),
             new SimpleLabeledDatasetDataBuilder<>(featureExtractor, lbExtractor)
         )) {
+            MLPArchitecture arch = archSupplier.apply(dataset);
+            MultilayerPerceptron mdl = new MultilayerPerceptron(arch, new RandomInitializer(seed));
+            ParameterUpdateCalculator<? super MultilayerPerceptron, P> updater = updatesStgy.getUpdatesCalculator();
+
             for (int i = 0; i < maxIterations; i += locIterations) {
 
                 MultilayerPerceptron finalMdl = mdl;
@@ -163,12 +183,12 @@ public class MLPTrainer<P extends Serializable> implements MultiLabelDatasetTrai
                 P update = updatesStgy.allUpdatesReducer().apply(totUp);
                 mdl = updater.update(mdl, update);
             }
+
+            return mdl;
         }
         catch (Exception e) {
             throw new RuntimeException(e);
         }
-
-        return mdl;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java
index f5a6bb0..1886ee5 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/PreprocessingTrainer.java
@@ -17,9 +17,15 @@
 
 package org.apache.ignite.ml.preprocessing;
 
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
+import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 
+import java.util.Map;
+
 /**
  * Trainer for preprocessor.
  *
@@ -34,9 +40,40 @@ public interface PreprocessingTrainer<K, V, T, R> {
      *
      * @param datasetBuilder Dataset builder.
      * @param basePreprocessor Base preprocessor.
-     * @param cols Number of columns.
      * @return Preprocessor.
      */
     public IgniteBiFunction<K, V, R> fit(DatasetBuilder<K, V> datasetBuilder,
-        IgniteBiFunction<K, V, T> basePreprocessor, int cols);
+        IgniteBiFunction<K, V, T> basePreprocessor);
+
+    /**
+     * Fits preprocessor.
+     *
+     * @param ignite Ignite instance.
+     * @param cache Ignite cache.
+     * @param basePreprocessor Base preprocessor.
+     * @return Preprocessor.
+     */
+    public default IgniteBiFunction<K, V, R> fit(Ignite ignite, IgniteCache<K, V> cache,
+        IgniteBiFunction<K, V, T> basePreprocessor) {
+        return fit(
+            new CacheBasedDatasetBuilder<>(ignite, cache),
+            basePreprocessor
+        );
+    }
+
+    /**
+     * Fits preprocessor.
+     *
+     * @param data Data.
+     * @param parts Number of partitions.
+     * @param basePreprocessor Base preprocessor.
+     * @return Preprocessor.
+     */
+    public default IgniteBiFunction<K, V, R> fit(Map<K, V> data, int parts,
+        IgniteBiFunction<K, V, T> basePreprocessor) {
+        return fit(
+            new LocalDatasetBuilder<>(data, parts),
+            basePreprocessor
+        );
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java
index 16623ba..57acbad 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainer.java
@@ -33,33 +33,48 @@ import org.apache.ignite.ml.preprocessing.PreprocessingTrainer;
 public class NormalizationTrainer<K, V> implements PreprocessingTrainer<K, V, double[], double[]> {
     /** {@inheritDoc} */
     @Override public NormalizationPreprocessor<K, V> fit(DatasetBuilder<K, V> datasetBuilder,
-        IgniteBiFunction<K, V, double[]> basePreprocessor, int cols) {
+        IgniteBiFunction<K, V, double[]> basePreprocessor) {
         try (Dataset<EmptyContext, NormalizationPartitionData> dataset = datasetBuilder.build(
             (upstream, upstreamSize) -> new EmptyContext(),
             (upstream, upstreamSize, ctx) -> {
-                double[] min = new double[cols];
-                double[] max = new double[cols];
-
-                for (int i = 0; i < cols; i++) {
-                    min[i] = Double.MAX_VALUE;
-                    max[i] = -Double.MAX_VALUE;
-                }
+                double[] min = null;
+                double[] max = null;
 
                 while (upstream.hasNext()) {
                     UpstreamEntry<K, V> entity = upstream.next();
                     double[] row = basePreprocessor.apply(entity.getKey(), entity.getValue());
-                    for (int i = 0; i < cols; i++) {
+
+                    if (min == null) {
+                        min = new double[row.length];
+                        for (int i = 0; i < min.length; i++)
+                            min[i] = Double.MAX_VALUE;
+                    }
+                    else
+                        assert min.length == row.length : "Base preprocessor must return exactly " + min.length
+                            + " features";
+
+                    if (max == null) {
+                        max = new double[row.length];
+                        for (int i = 0; i < max.length; i++)
+                            max[i] = -Double.MAX_VALUE;
+                    }
+                    else
+                        assert max.length == row.length : "Base preprocessor must return exactly " + min.length
+                            + " features";
+
+                    for (int i = 0; i < row.length; i++) {
                         if (row[i] < min[i])
                             min[i] = row[i];
                         if (row[i] > max[i])
                             max[i] = row[i];
                     }
                 }
+
                 return new NormalizationPartitionData(min, max);
             }
         )) {
             double[][] minMax = dataset.compute(
-                data -> new double[][]{ data.getMin(), data.getMax() },
+                data -> data.getMin() != null ? new double[][]{ data.getMin(), data.getMax() } : null,
                 (a, b) -> {
                     if (a == null)
                         return b;

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java
new file mode 100644
index 0000000..8e8f467
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/FeatureExtractorWrapper.java
@@ -0,0 +1,55 @@
+/*
+ * 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.ml.regressions.linear;
+
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+
+import java.util.Arrays;
+
+/**
+ * Feature extractor wrapper that adds additional column filled by 1.
+ *
+ * @param <K> Type of a key in {@code upstream} data.
+ * @param <V> Type of a value in {@code upstream} data.
+ */
+public class FeatureExtractorWrapper<K, V> implements IgniteBiFunction<K, V, double[]> {
+    /** */
+    private static final long serialVersionUID = -2686524650955735635L;
+
+    /** Underlying feature extractor. */
+    private final IgniteBiFunction<K, V, double[]> featureExtractor;
+
+    /**
+     * Constructs a new instance of feature extractor wrapper.
+     *
+     * @param featureExtractor Underlying feature extractor.
+     */
+    FeatureExtractorWrapper(IgniteBiFunction<K, V, double[]> featureExtractor) {
+        this.featureExtractor = featureExtractor;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double[] apply(K k, V v) {
+        double[] featureRow = featureExtractor.apply(k, v);
+        double[] row = Arrays.copyOf(featureRow, featureRow.length + 1);
+
+        row[featureRow.length] = 1.0;
+
+        return row;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
index ae15f2f..9526db1 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainer.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.ml.regressions.linear;
 
-import java.util.Arrays;
-import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
@@ -27,6 +25,9 @@ import org.apache.ignite.ml.math.isolve.LinSysPartitionDataBuilderOnHeap;
 import org.apache.ignite.ml.math.isolve.lsqr.AbstractLSQR;
 import org.apache.ignite.ml.math.isolve.lsqr.LSQROnHeap;
 import org.apache.ignite.ml.math.isolve.lsqr.LSQRResult;
+import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
+
+import java.util.Arrays;
 
 /**
  * Trainer of the linear regression model based on LSQR algorithm.
@@ -55,37 +56,4 @@ public class LinearRegressionLSQRTrainer implements SingleLabelDatasetTrainer<Li
 
         return new LinearRegressionModel(weights, x[x.length - 1]);
     }
-
-    /**
-     * Feature extractor wrapper that adds additional column filled by 1.
-     *
-     * @param <K> Type of a key in {@code upstream} data.
-     * @param <V> Type of a value in {@code upstream} data.
-     */
-    private static class FeatureExtractorWrapper<K, V> implements IgniteBiFunction<K, V, double[]> {
-        /** */
-        private static final long serialVersionUID = -2686524650955735635L;
-
-        /** Underlying feature extractor. */
-        private final IgniteBiFunction<K, V, double[]> featureExtractor;
-
-        /**
-         * Constructs a new instance of feature extractor wrapper.
-         *
-         * @param featureExtractor Underlying feature extractor.
-         */
-        FeatureExtractorWrapper(IgniteBiFunction<K, V, double[]> featureExtractor) {
-            this.featureExtractor = featureExtractor;
-        }
-
-        /** {@inheritDoc} */
-        @Override public double[] apply(K k, V v) {
-            double[] featureRow = featureExtractor.apply(k, v);
-            double[] row = Arrays.copyOf(featureRow, featureRow.length + 1);
-
-            row[featureRow.length] = 1.0;
-
-            return row;
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
index aad4c7a..9be3fdd 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
@@ -17,51 +17,99 @@
 
 package org.apache.ignite.ml.regressions.linear;
 
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.optimization.BarzilaiBorweinUpdater;
-import org.apache.ignite.ml.optimization.GradientDescent;
-import org.apache.ignite.ml.optimization.LeastSquaresGradientFunction;
-import org.apache.ignite.ml.optimization.SimpleUpdater;
+import org.apache.ignite.ml.dataset.Dataset;
+import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.primitive.context.EmptyContext;
+import org.apache.ignite.ml.dataset.primitive.data.SimpleLabeledDatasetData;
+import org.apache.ignite.ml.math.functions.IgniteBiFunction;
+import org.apache.ignite.ml.math.functions.IgniteFunction;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.nn.Activators;
+import org.apache.ignite.ml.nn.MLPTrainer;
+import org.apache.ignite.ml.nn.MultilayerPerceptron;
+import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
+import org.apache.ignite.ml.optimization.LossFunctions;
+import org.apache.ignite.ml.trainers.SingleLabelDatasetTrainer;
+import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
+
+import java.io.Serializable;
+import java.util.Arrays;
 
 /**
- * Linear regression trainer based on least squares loss function and gradient descent optimization algorithm.
+ * Trainer of the linear regression model based on stochastic gradient descent algorithm.
  */
-public class LinearRegressionSGDTrainer implements Trainer<LinearRegressionModel, Matrix> {
-    /**
-     * Gradient descent optimizer.
-     */
-    private final GradientDescent gradientDescent;
+public class LinearRegressionSGDTrainer<P extends Serializable> implements SingleLabelDatasetTrainer<LinearRegressionModel> {
+    /** Update strategy. */
+    private final UpdatesStrategy<? super MultilayerPerceptron, P> updatesStgy;
 
-    /** */
-    public LinearRegressionSGDTrainer(GradientDescent gradientDescent) {
-        this.gradientDescent = gradientDescent;
-    }
+    /** Max number of iteration. */
+    private final int maxIterations;
 
-    /** */
-    public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol) {
-        this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new BarzilaiBorweinUpdater())
-            .withMaxIterations(maxIterations)
-            .withConvergenceTol(convergenceTol);
-    }
+    /** Batch size. */
+    private final int batchSize;
 
-    /** */
-    public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol, double learningRate) {
-        this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new SimpleUpdater(learningRate))
-            .withMaxIterations(maxIterations)
-            .withConvergenceTol(convergenceTol);
-    }
+    /** Number of local iterations. */
+    private final int locIterations;
+
+    /** Seed for random generator. */
+    private final long seed;
 
     /**
-     * {@inheritDoc}
+     * Constructs a new instance of linear regression SGD trainer.
+     *
+     * @param updatesStgy Update strategy.
+     * @param maxIterations Max number of iteration.
+     * @param batchSize Batch size.
+     * @param locIterations Number of local iterations.
+     * @param seed Seed for random generator.
      */
-    @Override public LinearRegressionModel train(Matrix data) {
-        Vector variables = gradientDescent.optimize(data, data.likeVector(data.columnSize()));
-        Vector weights = variables.viewPart(1, variables.size() - 1);
+    public LinearRegressionSGDTrainer(UpdatesStrategy<? super MultilayerPerceptron, P> updatesStgy, int maxIterations,
+        int batchSize, int locIterations, long seed) {
+        this.updatesStgy = updatesStgy;
+        this.maxIterations = maxIterations;
+        this.batchSize = batchSize;
+        this.locIterations = locIterations;
+        this.seed = seed;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <K, V> LinearRegressionModel fit(DatasetBuilder<K, V> datasetBuilder,
+        IgniteBiFunction<K, V, double[]> featureExtractor, IgniteBiFunction<K, V, Double> lbExtractor) {
+
+        IgniteFunction<Dataset<EmptyContext, SimpleLabeledDatasetData>, MLPArchitecture> archSupplier = dataset -> {
+
+            int cols = dataset.compute(data -> {
+                if (data.getFeatures() == null)
+                    return null;
+                return data.getFeatures().length / data.getRows();
+            }, (a, b) -> a == null ? b : a);
+
+            MLPArchitecture architecture = new MLPArchitecture(cols);
+            architecture = architecture.withAddedLayer(1, true, Activators.LINEAR);
+
+            return architecture;
+        };
+
+        MLPTrainer<?> trainer = new MLPTrainer<>(
+            archSupplier,
+            LossFunctions.MSE,
+            updatesStgy,
+            maxIterations,
+            batchSize,
+            locIterations,
+            seed
+        );
+
+        IgniteBiFunction<K, V, double[]> lbE = new IgniteBiFunction<K, V, double[]>() {
+            @Override public double[] apply(K k, V v) {
+                return new double[]{lbExtractor.apply(k, v)};
+            }
+        };
+
+        MultilayerPerceptron mlp = trainer.fit(datasetBuilder, featureExtractor, lbE);
 
-        double intercept = variables.get(0);
+        double[] p = mlp.parameters().getStorage().data();
 
-        return new LinearRegressionModel(weights, intercept);
+        return new LinearRegressionModel(new DenseLocalOnHeapVector(Arrays.copyOf(p, p.length - 1)), p[p.length - 1]);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
index 8119a29..fcde3f5 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/trainers/DatasetTrainer.java
@@ -17,10 +17,16 @@
 
 package org.apache.ignite.ml.trainers;
 
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
 import org.apache.ignite.ml.Model;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
+import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
+import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.math.functions.IgniteBiFunction;
 
+import java.util.Map;
+
 /**
  * Interface for trainers. Trainer is just a function which produces model from the data.
  *
@@ -40,4 +46,44 @@ public interface DatasetTrainer<M extends Model, L> {
      */
     public <K, V> M fit(DatasetBuilder<K, V> datasetBuilder, IgniteBiFunction<K, V, double[]> featureExtractor,
         IgniteBiFunction<K, V, L> lbExtractor);
+
+    /**
+     * Trains model based on the specified data.
+     *
+     * @param ignite Ignite instance.
+     * @param cache Ignite cache.
+     * @param featureExtractor Feature extractor.
+     * @param lbExtractor Label extractor.
+     * @param <K> Type of a key in {@code upstream} data.
+     * @param <V> Type of a value in {@code upstream} data.
+     * @return Model.
+     */
+    public default <K, V> M fit(Ignite ignite, IgniteCache<K, V> cache, IgniteBiFunction<K, V, double[]> featureExtractor,
+        IgniteBiFunction<K, V, L> lbExtractor) {
+        return fit(
+            new CacheBasedDatasetBuilder<>(ignite, cache),
+            featureExtractor,
+            lbExtractor
+        );
+    }
+
+    /**
+     * Trains model based on the specified data.
+     *
+     * @param data Data.
+     * @param parts Number of partitions.
+     * @param featureExtractor Feature extractor.
+     * @param lbExtractor Label extractor.
+     * @param <K> Type of a key in {@code upstream} data.
+     * @param <V> Type of a value in {@code upstream} data.
+     * @return Model.
+     */
+    public default <K, V> M fit(Map<K, V> data, int parts, IgniteBiFunction<K, V, double[]> featureExtractor,
+        IgniteBiFunction<K, V, L> lbExtractor) {
+        return fit(
+            new LocalDatasetBuilder<>(data, parts),
+            featureExtractor,
+            lbExtractor
+        );
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
index b5a4b54..b27fcba 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNClassificationTest.java
@@ -17,11 +17,7 @@
 
 package org.apache.ignite.ml.knn;
 
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.knn.classification.KNNClassificationModel;
 import org.apache.ignite.ml.knn.classification.KNNClassificationTrainer;
 import org.apache.ignite.ml.knn.classification.KNNStrategy;
@@ -29,6 +25,10 @@ import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
 /** Tests behaviour of KNNClassificationTest. */
 public class KNNClassificationTest extends BaseKNNTest {
     /** */
@@ -46,7 +46,8 @@ public class KNNClassificationTest extends BaseKNNTest {
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            data,
+            2,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)
@@ -74,7 +75,8 @@ public class KNNClassificationTest extends BaseKNNTest {
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            data,
+            2,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(1)
@@ -102,7 +104,8 @@ public class KNNClassificationTest extends BaseKNNTest {
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            data,
+            2,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)
@@ -128,7 +131,8 @@ public class KNNClassificationTest extends BaseKNNTest {
         KNNClassificationTrainer trainer = new KNNClassificationTrainer();
 
         KNNClassificationModel knnMdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, 2),
+            data,
+            2,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[2]
         ).withK(3)

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
index 5ca661f..038b880 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerIntegrationTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.ml.nn;
 
-import java.io.Serializable;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
@@ -25,22 +24,18 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.math.Matrix;
 import org.apache.ignite.ml.math.Tracer;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
-import org.apache.ignite.ml.optimization.updatecalculators.NesterovParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.NesterovUpdateCalculator;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
+import org.apache.ignite.ml.optimization.updatecalculators.*;
 import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.io.Serializable;
+
 /**
  * Tests for {@link MLPTrainer} that require to start the whole Ignite infrastructure.
  */
@@ -137,7 +132,8 @@ public class MLPTrainerIntegrationTest extends GridCommonAbstractTest {
             );
 
             MultilayerPerceptron mlp = trainer.fit(
-                new CacheBasedDatasetBuilder<>(ignite, xorCache),
+                ignite,
+                xorCache,
                 (k, v) -> new double[]{ v.x, v.y },
                 (k, v) -> new double[]{ v.lb}
             );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
index 6906424..c53f6f1 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/MLPTrainerTest.java
@@ -17,24 +17,13 @@
 
 package org.apache.ignite.ml.nn;
 
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
 import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.math.Matrix;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.nn.architecture.MLPArchitecture;
 import org.apache.ignite.ml.optimization.LossFunctions;
-import org.apache.ignite.ml.optimization.updatecalculators.NesterovParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.NesterovUpdateCalculator;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.RPropUpdateCalculator;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDParameterUpdate;
-import org.apache.ignite.ml.optimization.updatecalculators.SimpleGDUpdateCalculator;
+import org.apache.ignite.ml.optimization.updatecalculators.*;
 import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.junit.Before;
 import org.junit.Test;
@@ -42,6 +31,12 @@ import org.junit.experimental.runners.Enclosed;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
 /**
  * Tests for {@link MLPTrainer} that don't require to start the whole Ignite infrastructure.
  */
@@ -140,7 +135,8 @@ public class MLPTrainerTest {
             );
 
             MultilayerPerceptron mlp = trainer.fit(
-                new LocalDatasetBuilder<>(xorData, parts),
+                xorData,
+                parts,
                 (k, v) -> v[0],
                 (k, v) -> v[1]
             );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
index c787a47..a64af9b 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistIntegrationTest.java
@@ -17,13 +17,11 @@
 
 package org.apache.ignite.ml.nn.performance;
 
-import java.io.IOException;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.dataset.impl.cache.CacheBasedDatasetBuilder;
 import org.apache.ignite.ml.math.Matrix;
 import org.apache.ignite.ml.math.VectorUtils;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
@@ -38,6 +36,8 @@ import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
+import java.io.IOException;
+
 /**
  * Tests {@link MLPTrainer} on the MNIST dataset that require to start the whole Ignite infrastructure.
  */
@@ -104,7 +104,8 @@ public class MLPTrainerMnistIntegrationTest extends GridCommonAbstractTest {
         System.out.println("Start training...");
         long start = System.currentTimeMillis();
         MultilayerPerceptron mdl = trainer.fit(
-            new CacheBasedDatasetBuilder<>(ignite, trainingSet),
+            ignite,
+            trainingSet,
             (k, v) -> v.getPixels(),
             (k, v) -> VectorUtils.num2Vec(v.getLabel(), 10).getStorage().data()
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
index 354af2c..d966484 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/nn/performance/MLPTrainerMnistTest.java
@@ -17,10 +17,6 @@
 
 package org.apache.ignite.ml.nn.performance;
 
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.apache.ignite.ml.math.Matrix;
 import org.apache.ignite.ml.math.VectorUtils;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
@@ -35,6 +31,10 @@ import org.apache.ignite.ml.trainers.group.UpdatesStrategy;
 import org.apache.ignite.ml.util.MnistUtils;
 import org.junit.Test;
 
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -74,7 +74,8 @@ public class MLPTrainerMnistTest {
         System.out.println("Start training...");
         long start = System.currentTimeMillis();
         MultilayerPerceptron mdl = trainer.fit(
-            new LocalDatasetBuilder<>(trainingSet, 1),
+            trainingSet,
+            1,
             (k, v) -> v.getPixels(),
             (k, v) -> VectorUtils.num2Vec(v.getLabel(), 10).getStorage().data()
         );

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java
index 1548253..e7a0d47 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/preprocessing/normalization/NormalizationTrainerTest.java
@@ -17,15 +17,16 @@
 
 package org.apache.ignite.ml.preprocessing.normalization;
 
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Map;
 import org.apache.ignite.ml.dataset.DatasetBuilder;
 import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
 import static org.junit.Assert.assertArrayEquals;
 
 /**
@@ -66,8 +67,7 @@ public class NormalizationTrainerTest {
 
         NormalizationPreprocessor<Integer, double[]> preprocessor = standardizationTrainer.fit(
             datasetBuilder,
-            (k, v) -> v,
-            3
+            (k, v) -> v
         );
 
         assertArrayEquals(new double[] {0, 4, 1}, preprocessor.getMin(), 1e-8);

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
index 82b3a1b..b3c9368 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/RegressionsTestSuite.java
@@ -17,14 +17,7 @@
 
 package org.apache.ignite.ml.regressions;
 
-import org.apache.ignite.ml.regressions.linear.BlockDistributedLinearRegressionQRTrainerTest;
-import org.apache.ignite.ml.regressions.linear.BlockDistributedLinearRegressionSGDTrainerTest;
-import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionQRTrainerTest;
-import org.apache.ignite.ml.regressions.linear.DistributedLinearRegressionSGDTrainerTest;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionLSQRTrainerTest;
-import org.apache.ignite.ml.regressions.linear.LinearRegressionModelTest;
-import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionQRTrainerTest;
-import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionSGDTrainerTest;
+import org.apache.ignite.ml.regressions.linear.*;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
@@ -35,12 +28,10 @@ import org.junit.runners.Suite;
 @Suite.SuiteClasses({
     LinearRegressionModelTest.class,
     LocalLinearRegressionQRTrainerTest.class,
-    LocalLinearRegressionSGDTrainerTest.class,
     DistributedLinearRegressionQRTrainerTest.class,
-    DistributedLinearRegressionSGDTrainerTest.class,
     BlockDistributedLinearRegressionQRTrainerTest.class,
-    BlockDistributedLinearRegressionSGDTrainerTest.class,
-    LinearRegressionLSQRTrainerTest.class
+    LinearRegressionLSQRTrainerTest.class,
+    LinearRegressionSGDTrainerTest.class
 })
 public class RegressionsTestSuite {
     // No-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java
deleted file mode 100644
index 58037e2..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.SparseBlockDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
-
-/**
- * Tests for {@link LinearRegressionSGDTrainer} on {@link SparseBlockDistributedMatrix}.
- */
-public class BlockDistributedLinearRegressionSGDTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest {
-    /** */
-    public BlockDistributedLinearRegressionSGDTrainerTest() {
-        super(
-            new LinearRegressionSGDTrainer(100_000, 1e-12),
-            SparseBlockDistributedMatrix::new,
-            SparseBlockDistributedVector::new,
-            1e-2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java
deleted file mode 100644
index 71d3b3b..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java
+++ /dev/null
@@ -1,35 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions.linear;
-
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
-
-/**
- * Tests for {@link LinearRegressionSGDTrainer} on {@link SparseDistributedMatrix}.
- */
-public class DistributedLinearRegressionSGDTrainerTest extends GridAwareAbstractLinearRegressionTrainerTest {
-    /** */
-    public DistributedLinearRegressionSGDTrainerTest() {
-        super(
-            new LinearRegressionSGDTrainer(100_000, 1e-12),
-            SparseDistributedMatrix::new,
-            SparseDistributedVector::new,
-            1e-2);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
index 1a60b80..9b75bd4 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
@@ -26,6 +26,9 @@ import org.apache.ignite.ml.math.functions.IgniteFunction;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.junit.Test;
 
+/**
+ * Grid aware abstract linear regression trainer test.
+ */
 public abstract class GridAwareAbstractLinearRegressionTrainerTest extends GridCommonAbstractTest {
     /** Number of nodes in grid */
     private static final int NODE_COUNT = 3;

http://git-wip-us.apache.org/repos/asf/ignite/blob/df6356d5/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java
index e3f60ec..2414236 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionLSQRTrainerTest.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.ml.regressions.linear;
 
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
-import org.apache.ignite.ml.dataset.impl.local.LocalDatasetBuilder;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
@@ -72,7 +72,8 @@ public class LinearRegressionLSQRTrainerTest {
         LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
 
         LinearRegressionModel mdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, parts),
+            data,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[4]
         );
@@ -110,7 +111,8 @@ public class LinearRegressionLSQRTrainerTest {
         LinearRegressionLSQRTrainer trainer = new LinearRegressionLSQRTrainer();
 
         LinearRegressionModel mdl = trainer.fit(
-            new LocalDatasetBuilder<>(data, parts),
+            data,
+            parts,
             (k, v) -> Arrays.copyOfRange(v, 0, v.length - 1),
             (k, v) -> v[coef.length]
         );


[06/50] [abbrv] ignite git commit: IGNITE-8216 Fixed javadoc for release build

Posted by ag...@apache.org.
IGNITE-8216 Fixed javadoc for release build


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

Branch: refs/heads/ignite-7708
Commit: 6557fe62696ac24c740e445b53482da298b59b27
Parents: 780fc07
Author: Sergey Chugunov <se...@gmail.com>
Authored: Wed Apr 11 12:28:40 2018 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Wed Apr 11 12:28:40 2018 +0300

----------------------------------------------------------------------
 parent/pom.xml | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6557fe62/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 16a9395..3decc16 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -404,6 +404,10 @@
                                 <packages>org.apache.ignite.spi.eventstorage*</packages>
                             </group>
                             <group>
+                                <title>Communication Failure Detection</title>
+                                <packages>org.apache.ignite.failure</packages>
+                            </group>
+                            <group>
                                 <title>Segmentation Detection</title>
                                 <packages>org.apache.ignite.plugin.segmentation</packages>
                             </group>


[02/50] [abbrv] ignite git commit: IGNITE-8153 Nodes fail to connect each other when SSL is enabled - Fixes #3773.

Posted by ag...@apache.org.
IGNITE-8153 Nodes fail to connect each other when SSL is enabled - Fixes #3773.

Signed-off-by: Valentin Kulichenko <va...@gmail.com>


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

Branch: refs/heads/ignite-7708
Commit: f4de6df71b256506ce36c1c4e16533bb063782a0
Parents: a96ac04
Author: mcherkasov <mc...@gridgain.com>
Authored: Tue Apr 10 17:23:29 2018 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Tue Apr 10 17:23:29 2018 -0700

----------------------------------------------------------------------
 .../ignite/internal/util/nio/ssl/BlockingSslHandler.java  | 10 +++++++---
 1 file changed, 7 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f4de6df7/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
index 638106f..0099c46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
@@ -373,9 +373,10 @@ public class BlockingSslHandler {
      * @throws GridNioException If failed to pass event to the next filter.
      */
     private Status unwrapHandshake() throws SSLException, IgniteCheckedException {
-        // Flip input buffer so we can read the collected data.
-        readFromNet();
+        if(!inNetBuf.hasRemaining())
+            readFromNet();
 
+        // Flip input buffer so we can read the collected data.
         inNetBuf.flip();
 
         SSLEngineResult res = unwrap0();
@@ -399,7 +400,10 @@ public class BlockingSslHandler {
         else if (res.getStatus() == BUFFER_UNDERFLOW) {
             inNetBuf.compact();
 
-            inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2);
+            if(inNetBuf.capacity() == inNetBuf.limit())
+                inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() * 2);
+
+            readFromNet();
         }
         else
             // prepare to be written again