You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/12/29 09:28:18 UTC

[01/15] ignite git commit: IGNITE-7288 Thin client: allow partial cache configuration

Repository: ignite
Updated Branches:
  refs/heads/ignite-zk 26ffa0dbb -> 4c63e4238


IGNITE-7288 Thin client: allow partial cache configuration

This closes #3286


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

Branch: refs/heads/ignite-zk
Commit: 3c5d376605c7c12fad0dc67b9b58fc61c42ddfbd
Parents: 1a939b0
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Dec 28 12:58:11 2017 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 28 12:58:11 2017 +0300

----------------------------------------------------------------------
 .../ClientCacheConfigurationSerializer.java     | 284 +++++++++++++++----
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   5 +
 .../Cache/CacheConfigurationTest.cs             |   3 +-
 .../Cache/ClientCacheConfigurationTest.cs       |  29 +-
 .../Client/Cache/CreateCacheTest.cs             |  42 ++-
 .../Client/ClientTestBase.cs                    |  18 ++
 .../Client/Cache/CacheClientConfiguration.cs    |   9 +-
 .../ClientConnectorConfiguration.cs             |   2 +-
 .../Cache/ClientCacheConfigurationSerializer.cs | 123 +++++++-
 9 files changed, 424 insertions(+), 91 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
index 0f28b81..839720b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/client/cache/ClientCacheConfigurationSerializer.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.platform.client.cache;
 
 import org.apache.ignite.binary.BinaryRawReader;
-import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheKeyConfiguration;
 import org.apache.ignite.cache.CacheMode;
@@ -40,6 +39,97 @@ import static org.apache.ignite.internal.processors.platform.utils.PlatformConfi
  * Cache configuration serializer.
  */
 public class ClientCacheConfigurationSerializer {
+    /** Name. */
+    private static final short NAME = 0;
+
+    /** Common properties. */
+    private static final short CACHE_MODE = 1;
+
+    /** */
+    private static final short ATOMICITY_MODE = 2;
+
+    /** */
+    private static final short BACKUPS = 3;
+
+    /** */
+    private static final short WRITE_SYNCHRONIZATION_MODE = 4;
+
+    /** */
+    private static final short COPY_ON_READ = 5;
+
+    /** */
+    private static final short READ_FROM_BACKUP = 6;
+
+    /** Memory settings. */
+    private static final short DATA_REGION_NAME = 100;
+
+    /** */
+    private static final short ONHEAP_CACHE_ENABLED = 101;
+
+    /** SQL. */
+    private static final short QUERY_ENTITIES = 200;
+
+    /** */
+    private static final short QUERY_PARALLELISM = 201;
+
+    /** */
+    private static final short QUERY_DETAIL_METRICS_SIZE = 202;
+
+    /** */
+    private static final short SQL_SCHEMA = 203;
+
+    /** */
+    private static final short SQL_INDEX_MAX_INLINE_SIZE = 204;
+
+    /** */
+    private static final short SQL_ESCAPE_ALL = 205;
+
+    /** */
+    private static final short MAX_QUERY_ITERATORS_COUNT = 206;
+
+    /** Rebalance. */
+    private static final short REBALANCE_MODE = 300;
+
+    /** */
+    private static final short REBALANCE_DELAY = 301;
+
+    /** */
+    private static final short REBALANCE_TIMEOUT = 302;
+
+    /** */
+    private static final short REBALANCE_BATCH_SIZE = 303;
+
+    /** */
+    private static final short REBALANCE_BATCHES_PREFETCH_COUNT = 304;
+
+    /** */
+    private static final short REBALANCE_ORDER = 305;
+
+    /** */
+    private static final short REBALANCE_THROTTLE = 306;
+
+    /** Advanced. */
+    private static final short GROUP_NAME = 400;
+
+    /** */
+    private static final short KEY_CONFIGURATION = 401;
+
+    /** */
+    private static final short DEFAULT_LOCK_TIMEOUT = 402;
+
+    /** */
+    private static final short MAX_CONCURRENT_ASYNC_OPERATIONS = 403;
+
+    /** */
+    private static final short PARTITION_LOSS_POLICY = 404;
+
+    /** */
+    private static final short EAGER_TTL = 405;
+
+    /** */
+    private static final short STATISTICS_ENABLED = 406;
+
+
     /**
      * Writes the cache configuration.
      * @param writer Writer.
@@ -60,7 +150,6 @@ public class ClientCacheConfigurationSerializer {
         writer.writeBoolean(cfg.isEagerTtl());
         writer.writeBoolean(cfg.isStatisticsEnabled());
         writer.writeString(cfg.getGroupName());
-        writer.writeBoolean(cfg.isInvalidate());
         writer.writeLong(cfg.getDefaultLockTimeout());
         writer.writeInt(cfg.getMaxConcurrentAsyncOperations());
         writer.writeInt(cfg.getMaxQueryIteratorsCount());
@@ -119,60 +208,153 @@ public class ClientCacheConfigurationSerializer {
     static CacheConfiguration read(BinaryRawReader reader) {
         reader.readInt();  // Skip length.
 
-        CacheConfiguration cfg = new CacheConfiguration()
-                .setAtomicityMode(CacheAtomicityMode.fromOrdinal(reader.readInt()))
-                .setBackups(reader.readInt())
-                .setCacheMode(CacheMode.fromOrdinal(reader.readInt()))
-                .setCopyOnRead(reader.readBoolean())
-                .setDataRegionName(reader.readString())
-                .setEagerTtl(reader.readBoolean())
-                .setStatisticsEnabled(reader.readBoolean())
-                .setGroupName(reader.readString())
-                .setInvalidate(reader.readBoolean())
-                .setDefaultLockTimeout(reader.readLong())
-                .setMaxConcurrentAsyncOperations(reader.readInt())
-                .setMaxQueryIteratorsCount(reader.readInt())
-                .setName(reader.readString())
-                .setOnheapCacheEnabled(reader.readBoolean())
-                .setPartitionLossPolicy(PartitionLossPolicy.fromOrdinal((byte)reader.readInt()))
-                .setQueryDetailMetricsSize(reader.readInt())
-                .setQueryParallelism(reader.readInt())
-                .setReadFromBackup(reader.readBoolean())
-                .setRebalanceBatchSize(reader.readInt())
-                .setRebalanceBatchesPrefetchCount(reader.readLong())
-                .setRebalanceDelay(reader.readLong())
-                .setRebalanceMode(CacheRebalanceMode.fromOrdinal(reader.readInt()))
-                .setRebalanceOrder(reader.readInt())
-                .setRebalanceThrottle(reader.readLong())
-                .setRebalanceTimeout(reader.readLong())
-                .setSqlEscapeAll(reader.readBoolean())
-                .setSqlIndexMaxInlineSize(reader.readInt())
-                .setSqlSchema(reader.readString())
-                .setWriteSynchronizationMode(CacheWriteSynchronizationMode.fromOrdinal(reader.readInt()));
-
-        // Key configuration.
-        int keyCnt = reader.readInt();
-
-        if (keyCnt > 0) {
-            CacheKeyConfiguration[] keys = new CacheKeyConfiguration[keyCnt];
-
-            for (int i = 0; i < keyCnt; i++) {
-                keys[i] = new CacheKeyConfiguration(reader.readString(), reader.readString());
-            }
+        short propCnt = reader.readShort();
 
-            cfg.setKeyConfiguration(keys);
-        }
+        CacheConfiguration cfg = new CacheConfiguration();
+
+        for (int i = 0; i < propCnt; i++) {
+            short code = reader.readShort();
 
-        // Query entities.
-        int qryEntCnt = reader.readInt();
+            switch (code) {
+                case ATOMICITY_MODE:
+                    cfg.setAtomicityMode(CacheAtomicityMode.fromOrdinal(reader.readInt()));
+                    break;
 
-        if (qryEntCnt > 0) {
-            Collection<QueryEntity> entities = new ArrayList<>(qryEntCnt);
+                case BACKUPS:
+                    cfg.setBackups(reader.readInt());
+                    break;
 
-            for (int i = 0; i < qryEntCnt; i++)
-                entities.add(readQueryEntity(reader));
+                case CACHE_MODE:
+                    cfg.setCacheMode(CacheMode.fromOrdinal(reader.readInt()));
+                    break;
 
-            cfg.setQueryEntities(entities);
+                case COPY_ON_READ:
+                    cfg.setCopyOnRead(reader.readBoolean());
+                    break;
+
+                case DATA_REGION_NAME:
+                    cfg.setDataRegionName(reader.readString());
+                    break;
+
+                case EAGER_TTL:
+                    cfg.setEagerTtl(reader.readBoolean());
+                    break;
+
+                case STATISTICS_ENABLED:
+                    cfg.setStatisticsEnabled(reader.readBoolean());
+                    break;
+
+                case GROUP_NAME:
+                    cfg.setGroupName(reader.readString());
+                    break;
+
+                case DEFAULT_LOCK_TIMEOUT:
+                    cfg.setDefaultLockTimeout(reader.readLong());
+                    break;
+
+                case MAX_CONCURRENT_ASYNC_OPERATIONS:
+                    cfg.setMaxConcurrentAsyncOperations(reader.readInt());
+                    break;
+
+                case MAX_QUERY_ITERATORS_COUNT:
+                    cfg.setMaxQueryIteratorsCount(reader.readInt());
+                    break;
+
+                case NAME:
+                    cfg.setName(reader.readString());
+                    break;
+
+                case ONHEAP_CACHE_ENABLED:
+                    cfg.setOnheapCacheEnabled(reader.readBoolean());
+                    break;
+
+                case PARTITION_LOSS_POLICY:
+                    cfg.setPartitionLossPolicy(PartitionLossPolicy.fromOrdinal((byte) reader.readInt()));
+                    break;
+
+                case QUERY_DETAIL_METRICS_SIZE:
+                    cfg.setQueryDetailMetricsSize(reader.readInt());
+                    break;
+
+                case QUERY_PARALLELISM:
+                    cfg.setQueryParallelism(reader.readInt());
+                    break;
+
+                case READ_FROM_BACKUP:
+                    cfg.setReadFromBackup(reader.readBoolean());
+                    break;
+
+                case REBALANCE_BATCH_SIZE:
+                    cfg.setRebalanceBatchSize(reader.readInt());
+                    break;
+
+                case REBALANCE_BATCHES_PREFETCH_COUNT:
+                    cfg.setRebalanceBatchesPrefetchCount(reader.readLong());
+                    break;
+
+                case REBALANCE_DELAY:
+                    cfg.setRebalanceDelay(reader.readLong());
+                    break;
+
+                case REBALANCE_MODE:
+                    cfg.setRebalanceMode(CacheRebalanceMode.fromOrdinal(reader.readInt()));
+                    break;
+
+                case REBALANCE_ORDER:
+                    cfg.setRebalanceOrder(reader.readInt());
+                    break;
+
+                case REBALANCE_THROTTLE:
+                    cfg.setRebalanceThrottle(reader.readLong());
+                    break;
+
+                case REBALANCE_TIMEOUT:
+                    cfg.setRebalanceTimeout(reader.readLong());
+                    break;
+
+                case SQL_ESCAPE_ALL:
+                    cfg.setSqlEscapeAll(reader.readBoolean());
+                    break;
+
+                case SQL_INDEX_MAX_INLINE_SIZE:
+                    cfg.setSqlIndexMaxInlineSize(reader.readInt());
+                    break;
+
+                case SQL_SCHEMA:
+                    cfg.setSqlSchema(reader.readString());
+                    break;
+
+                case WRITE_SYNCHRONIZATION_MODE:
+                    cfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.fromOrdinal(reader.readInt()));
+                    break;
+
+                case KEY_CONFIGURATION:
+                    int keyCnt = reader.readInt();
+
+                    if (keyCnt > 0) {
+                        CacheKeyConfiguration[] keys = new CacheKeyConfiguration[keyCnt];
+
+                        for (int j = 0; j < keyCnt; j++) {
+                            keys[j] = new CacheKeyConfiguration(reader.readString(), reader.readString());
+                        }
+
+                        cfg.setKeyConfiguration(keys);
+                    }
+                    break;
+
+                case QUERY_ENTITIES:
+                    int qryEntCnt = reader.readInt();
+
+                    if (qryEntCnt > 0) {
+                        Collection<QueryEntity> entities = new ArrayList<>(qryEntCnt);
+
+                        for (int j = 0; j < qryEntCnt; j++)
+                            entities.add(readQueryEntity(reader));
+
+                        cfg.setQueryEntities(entities);
+                    }
+                    break;
+            }
         }
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
index f1ccb06..d660b62 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
@@ -53,6 +53,7 @@
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\BinarizableTestException.cs" Link="Cache\BinarizableTestException.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CacheAbstractTest.cs" Link="Cache\CacheAbstractTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CacheAbstractTransactionalTest.cs" Link="Cache\CacheAbstractTransactionalTest.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CacheConfigurationTest.cs" Link="Cache\CacheConfigurationTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CachePartitionedTest.cs" Link="Cache\CachePartitionedTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CacheTestAsyncWrapper.cs" Link="Cache\CacheTestAsyncWrapper.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Cache\CacheTestKey.cs" Link="Cache\CacheTestKey.cs" />
@@ -86,6 +87,7 @@
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\CacheTestAsync.cs" Link="ThinClient\Cache\CacheTestAsync.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\CacheClientAsyncWrapper.cs" Link="ThinClient\Cache\CacheClientAsyncWrapper.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\CacheTestKeepBinary.cs" Link="ThinClient\Cache\CacheTestKeepBinary.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\CreateCacheTest.cs" Link="ThinClient\Cache\CreateCacheTest.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\EmptyObject.cs" Link="ThinClient\Cache\EmptyObject.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\Person.cs" Link="ThinClient\Cache\Person.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\Client\Cache\ScanQueryTest.cs" Link="ThinClient\Cache\ScanQueryTest.cs" />
@@ -118,6 +120,9 @@
     <Content Include="..\Apache.Ignite.Core.Tests\Config\cache-binarizables.xml" Link="Config\cache-binarizables.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>
+    <Content Include="..\Apache.Ignite.Core.Tests\Config\cache-default.xml" Link="Config\cache-default.xml">
+      <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
+    </Content>
     <Content Include="..\Apache.Ignite.Core.Tests\Config\cache-query.xml" Link="Config\cache-query.xml">
       <CopyToOutputDirectory>PreserveNewest</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/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 256c245..536cb18 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Cache/CacheConfigurationTest.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Tests.Cache
 {
     using System;
     using System.Collections.Generic;
+    using System.IO;
     using System.Linq;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache.Affinity;
@@ -92,7 +93,7 @@ namespace Apache.Ignite.Core.Tests.Cache
                 },
 #pragma warning restore 618
                 DataStorageConfiguration = null,
-                SpringConfigUrl = "Config\\cache-default.xml"
+                SpringConfigUrl = Path.Combine("Config", "cache-default.xml")
             };
 
             _ignite = Ignition.Start(cfg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ClientCacheConfigurationTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ClientCacheConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ClientCacheConfigurationTest.cs
index 41770b8..903b038 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ClientCacheConfigurationTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/ClientCacheConfigurationTest.cs
@@ -112,11 +112,11 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
             var clientCfg = new CacheClientConfiguration(
                 CacheConfigurationTest.GetCustomCacheConfiguration("z"), true);
 
-            AssertClientConfigsAreEqual(clientCfg, new CacheClientConfiguration(clientCfg));
+            ClientTestBase.AssertClientConfigsAreEqual(clientCfg, new CacheClientConfiguration(clientCfg));
 
             // Convert to server cfg.
             var serverCfg = clientCfg.ToCacheConfiguration();
-            AssertClientConfigsAreEqual(clientCfg, new CacheClientConfiguration(serverCfg, false));
+            ClientTestBase.AssertClientConfigsAreEqual(clientCfg, new CacheClientConfiguration(serverCfg, false));
         }
 
         /// <summary>
@@ -129,14 +129,14 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
             var clientCfg = new CacheClientConfiguration();
             var defCfg = new CacheClientConfiguration(new CacheConfiguration(), false);
 
-            AssertClientConfigsAreEqual(defCfg, clientCfg);
+            ClientTestBase.AssertClientConfigsAreEqual(defCfg, clientCfg);
 
             // Name.
             clientCfg = new CacheClientConfiguration("foo");
             Assert.AreEqual("foo", clientCfg.Name);
 
             clientCfg.Name = null;
-            AssertClientConfigsAreEqual(defCfg, clientCfg);
+            ClientTestBase.AssertClientConfigsAreEqual(defCfg, clientCfg);
 
             // Query entities.
             clientCfg = new CacheClientConfiguration("bar", typeof(QueryPerson));
@@ -168,24 +168,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         {
             var clientCfg = new CacheClientConfiguration(cfg, false);
 
-            AssertClientConfigsAreEqual(clientCfg, SerializeDeserialize(clientCfg));
-        }
-
-        /// <summary>
-        /// Asserts the client configs are equal.
-        /// </summary>
-        public static void AssertClientConfigsAreEqual(CacheClientConfiguration cfg, CacheClientConfiguration cfg2)
-        {
-            if (cfg2.QueryEntities != null)
-            {
-                // Remove identical aliases which are added during config roundtrip.
-                foreach (var e in cfg2.QueryEntities)
-                {
-                    e.Aliases = e.Aliases.Where(x => x.Alias != x.FullName).ToArray();
-                }
-            }
-
-            AssertExtensions.ReflectionEqual(cfg, cfg2);
+            ClientTestBase.AssertClientConfigsAreEqual(clientCfg, SerializeDeserialize(clientCfg));
         }
 
         /// <summary>
@@ -195,7 +178,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
         {
             using (var stream = new BinaryHeapStream(128))
             {
-                ClientCacheConfigurationSerializer.Write(stream, cfg);
+                ClientCacheConfigurationSerializer.Write(stream, cfg, true);
                 stream.Seek(0, SeekOrigin.Begin);
                 return new CacheClientConfiguration(stream);
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
index 94ae6cd..92804be 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CreateCacheTest.cs
@@ -22,6 +22,10 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
     using Apache.Ignite.Core.Client;
     using Apache.Ignite.Core.Client.Cache;
     using Apache.Ignite.Core.Configuration;
+#if !NETCOREAPP2_0
+    using Apache.Ignite.Core.Impl.Client.Cache;
+    using Apache.Ignite.Core.Impl.Client;
+#endif
     using Apache.Ignite.Core.Tests.Cache;
     using NUnit.Framework;
 
@@ -145,9 +149,43 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
             cfg = GetFullCacheConfiguration("b");
 
             cache = Client.CreateCache<int, int>(cfg);
-            ClientCacheConfigurationTest.AssertClientConfigsAreEqual(cfg, cache.GetConfiguration());
+            AssertClientConfigsAreEqual(cfg, cache.GetConfiguration());
         }
 
+#if !NETCOREAPP2_0
+        /// <summary>
+        /// Tests cache creation from partial configuration.
+        /// </summary>
+        [Test]
+        public void TestCreateFromPartialConfiguration()
+        {
+            // Default config.
+            var cfg = new CacheClientConfiguration("a") {Backups = 7};
+            var client = (IgniteClient) Client;
+
+            // Create cache directly through a socket with only some config properties provided.
+            client.Socket.DoOutInOp<object>(ClientOp.CacheCreateWithConfiguration, s =>
+            {
+                var w = client.Marshaller.StartMarshal(s);
+
+                w.WriteInt(2 + 2 + 6 + 2 + 4);  // config length in bytes.
+
+                w.WriteShort(2);  // 2 properties.
+                
+                w.WriteShort(3);  // backups opcode.
+                w.WriteInt(cfg.Backups);
+
+                w.WriteShort(0);  // name opcode.
+                w.WriteString(cfg.Name);
+
+            }, null);
+            
+            var cache = new CacheClient<int, int>(client, cfg.Name);
+
+            AssertExtensions.ReflectionEqual(cfg, cache.GetConfiguration());
+        }
+#endif
+
         /// <summary>
         /// Tests cache creation from configuration.
         /// </summary>
@@ -168,7 +206,7 @@ namespace Apache.Ignite.Core.Tests.Client.Cache
             cfg = GetFullCacheConfiguration("b");
 
             cache = Client.GetOrCreateCache<int, int>(cfg);
-            ClientCacheConfigurationTest.AssertClientConfigsAreEqual(cfg, cache.GetConfiguration());
+            AssertClientConfigsAreEqual(cfg, cache.GetConfiguration());
         }
 
         /// <summary>

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientTestBase.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientTestBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientTestBase.cs
index 78d571b..9fea52c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientTestBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/ClientTestBase.cs
@@ -18,6 +18,7 @@
 namespace Apache.Ignite.Core.Tests.Client
 {
     using System;
+    using System.Linq;
     using System.Net;
     using Apache.Ignite.Core.Binary;
     using Apache.Ignite.Core.Cache;
@@ -190,5 +191,22 @@ namespace Apache.Ignite.Core.Tests.Client
         {
             return ToBinary(new Person(id) { DateTime = DateTime.MinValue.ToUniversalTime() });
         }
+
+        /// <summary>
+        /// Asserts the client configs are equal.
+        /// </summary>
+        public static void AssertClientConfigsAreEqual(CacheClientConfiguration cfg, CacheClientConfiguration cfg2)
+        {
+            if (cfg2.QueryEntities != null)
+            {
+                // Remove identical aliases which are added during config roundtrip.
+                foreach (var e in cfg2.QueryEntities)
+                {
+                    e.Aliases = e.Aliases.Where(x => x.Alias != x.FullName).ToArray();
+                }
+            }
+
+            AssertExtensions.ReflectionEqual(cfg, cfg2);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core/Client/Cache/CacheClientConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Client/Cache/CacheClientConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Cache/CacheClientConfiguration.cs
index 21ca247..1c1874f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Client/Cache/CacheClientConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Client/Cache/CacheClientConfiguration.cs
@@ -65,7 +65,6 @@ namespace Apache.Ignite.Core.Client.Cache
             CopyOnRead = CacheConfiguration.DefaultCopyOnRead;
             WriteSynchronizationMode = CacheConfiguration.DefaultWriteSynchronizationMode;
             EagerTtl = CacheConfiguration.DefaultEagerTtl;
-            Invalidate = CacheConfiguration.DefaultInvalidate;
             LockTimeout = CacheConfiguration.DefaultLockTimeout;
             MaxConcurrentAsyncOperations = CacheConfiguration.DefaultMaxConcurrentAsyncOperations;
             ReadFromBackup = CacheConfiguration.DefaultReadFromBackup;
@@ -116,7 +115,7 @@ namespace Apache.Ignite.Core.Client.Cache
             {
                 using (var stream = IgniteManager.Memory.Allocate().GetStream())
                 {
-                    ClientCacheConfigurationSerializer.Write(stream, other);
+                    ClientCacheConfigurationSerializer.Write(stream, other, true);
 
                     stream.SynchronizeOutput();
                     stream.Seek(0, SeekOrigin.Begin);
@@ -240,12 +239,6 @@ namespace Apache.Ignite.Core.Client.Cache
         public TimeSpan LockTimeout { get; set; }
 
         /// <summary>
-        /// Invalidation flag. If true, values will be invalidated (nullified) upon commit in near cache.
-        /// </summary>
-        [DefaultValue(CacheConfiguration.DefaultInvalidate)]
-        public bool Invalidate { get; set; }
-
-        /// <summary>
         /// Gets or sets cache rebalance mode.
         /// </summary>
         [DefaultValue(CacheConfiguration.DefaultRebalanceMode)]

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
index a5b2403..aff2716 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/ClientConnectorConfiguration.cs
@@ -61,7 +61,7 @@ namespace Apache.Ignite.Core.Configuration
         /// <summary>
         /// Default idle timeout.
         /// </summary>
-        public static TimeSpan DefaultIdleTimeout = TimeSpan.Zero;
+        public static readonly TimeSpan DefaultIdleTimeout = TimeSpan.Zero;
 
         /// <summary>
         /// Initializes a new instance of the <see cref="ClientConnectorConfiguration"/> class.

http://git-wip-us.apache.org/repos/asf/ignite/blob/3c5d3766/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/ClientCacheConfigurationSerializer.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/ClientCacheConfigurationSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/ClientCacheConfigurationSerializer.cs
index 160d09f..b11b175 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/ClientCacheConfigurationSerializer.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/ClientCacheConfigurationSerializer.cs
@@ -32,6 +32,55 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
     /// </summary>
     internal static class ClientCacheConfigurationSerializer
     {
+        /** Config property opcodes. */
+        private enum Op : short
+        {
+            // Name is required.
+            Name = 0,
+
+            // Most common properties.
+            CacheMode = 1,
+            AtomicityMode = 2,
+            Backups = 3,
+            WriteSynchronizationMode = 4,
+            CopyOnRead = 5,
+            ReadFromBackup = 6,
+
+            // Memory settings.
+            DataRegionName = 100,
+            OnheapCacheEnabled = 101,
+
+            // SQL.
+            QueryEntities = 200,
+            QueryParallelism = 201,
+            QueryDetailMetricsSize = 202,
+            SqlSchema = 203,
+            SqlIndexMaxInlineSize = 204,
+            SqlEscapeAll = 205,
+            MaxQueryIteratorsCount = 206,
+
+            // Rebalance.
+            RebalanceMode = 300,
+            RebalanceDelay = 301,
+            RebalanceTimeout = 302,
+            RebalanceBatchSize = 303,
+            RebalanceBatchesPrefetchCount = 304,
+            RebalanceOrder = 305,
+            RebalanceThrottle = 306,
+
+            // Advanced.
+            GroupName = 400,
+            KeyConfiguration = 401,
+            DefaultLockTimeout = 402,
+            MaxConcurrentAsyncOperations = 403,
+            PartitionLossPolicy = 404,
+            EagerTtl = 405, 
+            StatisticsEnabled = 406
+        }
+
+        /** Property count. */
+        private static readonly short PropertyCount = (short) Enum.GetValues(typeof(Op)).Length;
+        
         /// <summary>
         /// Copies one cache configuration to another.
         /// </summary>
@@ -48,7 +97,6 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             to.EagerTtl = from.EagerTtl;
             to.EnableStatistics = from.EnableStatistics;
             to.GroupName = from.GroupName;
-            to.Invalidate = from.Invalidate;
             to.LockTimeout = from.LockTimeout;
             to.MaxConcurrentAsyncOperations = from.MaxConcurrentAsyncOperations;
             to.MaxQueryIteratorsCount = from.MaxQueryIteratorsCount;
@@ -119,7 +167,6 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             to.EagerTtl = from.EagerTtl;
             to.EnableStatistics = from.EnableStatistics;
             to.GroupName = from.GroupName;
-            to.Invalidate = from.Invalidate;
             to.LockTimeout = from.LockTimeout;
             to.MaxConcurrentAsyncOperations = from.MaxConcurrentAsyncOperations;
             to.MaxQueryIteratorsCount = from.MaxQueryIteratorsCount;
@@ -148,7 +195,7 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
         /// <summary>
         /// Writes the specified config.
         /// </summary>
-        public static void Write(IBinaryStream stream, CacheClientConfiguration cfg)
+        public static void Write(IBinaryStream stream, CacheClientConfiguration cfg, bool skipCodes = false)
         {
             Debug.Assert(stream != null);
             Debug.Assert(cfg != null);
@@ -158,43 +205,110 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             var pos = writer.Stream.Position;
             writer.WriteInt(0);  // Reserve for length.
 
+            if (!skipCodes)
+            {
+                writer.WriteShort(PropertyCount); // Property count.
+            }
+
+            var code = skipCodes
+                ? (Action<Op>) (o => { })
+                : o => writer.WriteShort((short) o);
+
+            code(Op.AtomicityMode);
             writer.WriteInt((int)cfg.AtomicityMode);
+            
+            code(Op.Backups);
             writer.WriteInt(cfg.Backups);
+            
+            code(Op.CacheMode);
             writer.WriteInt((int)cfg.CacheMode);
+            
+            code(Op.CopyOnRead);
             writer.WriteBoolean(cfg.CopyOnRead);
+            
+            code(Op.DataRegionName);
             writer.WriteString(cfg.DataRegionName);
+            
+            code(Op.EagerTtl);
             writer.WriteBoolean(cfg.EagerTtl);
+            
+            code(Op.StatisticsEnabled);
             writer.WriteBoolean(cfg.EnableStatistics);
+            
+            code(Op.GroupName);
             writer.WriteString(cfg.GroupName);
-            writer.WriteBoolean(cfg.Invalidate);
+            
+            code(Op.DefaultLockTimeout);
             writer.WriteTimeSpanAsLong(cfg.LockTimeout);
+
+            code(Op.MaxConcurrentAsyncOperations);
             writer.WriteInt(cfg.MaxConcurrentAsyncOperations);
+
+            code(Op.MaxQueryIteratorsCount);
             writer.WriteInt(cfg.MaxQueryIteratorsCount);
+
+            code(Op.Name);
             writer.WriteString(cfg.Name);
+            
+            code(Op.OnheapCacheEnabled);
             writer.WriteBoolean(cfg.OnheapCacheEnabled);
+            
+            code(Op.PartitionLossPolicy);
             writer.WriteInt((int)cfg.PartitionLossPolicy);
+            
+            code(Op.QueryDetailMetricsSize);
             writer.WriteInt(cfg.QueryDetailMetricsSize);
+            
+            code(Op.QueryParallelism);
             writer.WriteInt(cfg.QueryParallelism);
+            
+            code(Op.ReadFromBackup);
             writer.WriteBoolean(cfg.ReadFromBackup);
+            
+            code(Op.RebalanceBatchSize);
             writer.WriteInt(cfg.RebalanceBatchSize);
+            
+            code(Op.RebalanceBatchesPrefetchCount);
             writer.WriteLong(cfg.RebalanceBatchesPrefetchCount);
+            
+            code(Op.RebalanceDelay);
             writer.WriteTimeSpanAsLong(cfg.RebalanceDelay);
+            
+            code(Op.RebalanceMode);
             writer.WriteInt((int)cfg.RebalanceMode);
+            
+            code(Op.RebalanceOrder);
             writer.WriteInt(cfg.RebalanceOrder);
+            
+            code(Op.RebalanceThrottle);
             writer.WriteTimeSpanAsLong(cfg.RebalanceThrottle);
+            
+            code(Op.RebalanceTimeout);
             writer.WriteTimeSpanAsLong(cfg.RebalanceTimeout);
+            
+            code(Op.SqlEscapeAll);
             writer.WriteBoolean(cfg.SqlEscapeAll);
+            
+            code(Op.SqlIndexMaxInlineSize);
             writer.WriteInt(cfg.SqlIndexMaxInlineSize);
+            
+            code(Op.SqlSchema);
             writer.WriteString(cfg.SqlSchema);
+            
+            code(Op.WriteSynchronizationMode);
             writer.WriteInt((int)cfg.WriteSynchronizationMode);
 
+            code(Op.KeyConfiguration);
             writer.WriteCollectionRaw(cfg.KeyConfiguration);
+            
+            code(Op.QueryEntities);
             writer.WriteCollectionRaw(cfg.QueryEntities);
 
             // Write length (so that part of the config can be skipped).
             var len = writer.Stream.Position - pos - 4;
             writer.Stream.WriteInt(pos, len);
         }
+        
         /// <summary>
         /// Reads the config.
         /// </summary>
@@ -216,7 +330,6 @@ namespace Apache.Ignite.Core.Impl.Client.Cache
             cfg.EagerTtl = reader.ReadBoolean();
             cfg.EnableStatistics = reader.ReadBoolean();
             cfg.GroupName = reader.ReadString();
-            cfg.Invalidate = reader.ReadBoolean();
             cfg.LockTimeout = reader.ReadLongAsTimespan();
             cfg.MaxConcurrentAsyncOperations = reader.ReadInt();
             cfg.MaxQueryIteratorsCount = reader.ReadInt();


[08/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/resources/datasets/regression/diabetes.csv
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/resources/datasets/regression/diabetes.csv b/modules/ml/src/test/resources/datasets/regression/diabetes.csv
new file mode 100644
index 0000000..6342081
--- /dev/null
+++ b/modules/ml/src/test/resources/datasets/regression/diabetes.csv
@@ -0,0 +1,442 @@
+151.0, 0.0380759064334, 0.0506801187398, 0.0616962065187, 0.021872354995, -0.0442234984244, -0.0348207628377, -0.043400845652, -0.00259226199818, 0.0199084208763, -0.0176461251598
+75.0, -0.00188201652779, -0.044641636507, -0.0514740612388, -0.0263278347174, -0.00844872411122, -0.0191633397482, 0.0744115640788, -0.0394933828741, -0.0683297436244, -0.0922040496268
+141.0, 0.0852989062967, 0.0506801187398, 0.0444512133366, -0.00567061055493, -0.0455994512826, -0.0341944659141, -0.0323559322398, -0.00259226199818, 0.00286377051894, -0.0259303389895
+206.0, -0.0890629393523, -0.044641636507, -0.0115950145052, -0.0366564467986, 0.0121905687618, 0.0249905933641, -0.0360375700439, 0.0343088588777, 0.0226920225667, -0.00936191133014
+135.0, 0.00538306037425, -0.044641636507, -0.0363846922045, 0.021872354995, 0.00393485161259, 0.0155961395104, 0.00814208360519, -0.00259226199818, -0.0319914449414, -0.0466408735636
+97.0, -0.0926954778033, -0.044641636507, -0.0406959405, -0.0194420933299, -0.0689906498721, -0.0792878444118, 0.041276823842, -0.07639450375, -0.041180385188, -0.0963461565417
+138.0, -0.04547247794, 0.0506801187398, -0.0471628129433, -0.0159992226361, -0.0400956398498, -0.0248000120604, 0.000778807997018, -0.0394933828741, -0.0629129499163, -0.038356659734
+63.0, 0.0635036755906, 0.0506801187398, -0.00189470584028, 0.0666296740135, 0.0906198816793, 0.108914381124, 0.0228686348215, 0.0177033544836, -0.0358167281015, 0.00306440941437
+110.0, 0.0417084448844, 0.0506801187398, 0.0616962065187, -0.0400993174923, -0.013952535544, 0.00620168565673, -0.0286742944357, -0.00259226199818, -0.0149564750249, 0.011348623244
+310.0, -0.0709002470972, -0.044641636507, 0.0390621529672, -0.0332135761048, -0.0125765826858, -0.0345076143759, -0.0249926566316, -0.00259226199818, 0.0677363261103, -0.013504018245
+101.0, -0.0963280162543, -0.044641636507, -0.0838084234552, 0.00810087222001, -0.103389471327, -0.0905611890362, -0.0139477432193, -0.07639450375, -0.0629129499163, -0.0342145528191
+69.0, 0.0271782910804, 0.0506801187398, 0.0175059114896, -0.0332135761048, -0.00707277125302, 0.045971540304, -0.0654906724765, 0.0712099797536, -0.0964332228918, -0.0590671943082
+179.0, 0.0162806757273, -0.044641636507, -0.0288400076873, -0.00911348124867, -0.00432086553661, -0.00976888589454, 0.0449584616461, -0.0394933828741, -0.0307512098646, -0.0424987666488
+185.0, 0.00538306037425, 0.0506801187398, -0.00189470584028, 0.00810087222001, -0.00432086553661, -0.0157187066685, -0.00290282980707, -0.00259226199818, 0.0383932482117, -0.013504018245
+118.0, 0.0453409833355, -0.044641636507, -0.0256065714657, -0.0125563519424, 0.0176943801946, -6.12835790605e-05, 0.0817748396869, -0.0394933828741, -0.0319914449414, -0.0756356219675
+171.0, -0.0527375548421, 0.0506801187398, -0.0180618869485, 0.0804011567885, 0.0892439288211, 0.107661787277, -0.0397192078479, 0.10811110063, 0.0360557900898, -0.0424987666488
+166.0, -0.00551455497881, -0.044641636507, 0.0422955891888, 0.0494153205448, 0.0245741444856, -0.0238605666751, 0.0744115640788, -0.0394933828741, 0.0522799997968, 0.0279170509034
+144.0, 0.0707687524926, 0.0506801187398, 0.0121168511202, 0.0563010619323, 0.034205814493, 0.0494161733837, -0.0397192078479, 0.0343088588777, 0.0273677075426, -0.00107769750047
+97.0, -0.038207401038, -0.044641636507, -0.0105172024313, -0.0366564467986, -0.0373437341334, -0.01947648821, -0.0286742944357, -0.00259226199818, -0.0181182673079, -0.0176461251598
+168.0, -0.0273097856849, -0.044641636507, -0.0180618869485, -0.0400993174923, -0.00294491267841, -0.0113346282035, 0.0375951860379, -0.0394933828741, -0.0089440189578, -0.0549250873933
+68.0, -0.049105016391, -0.044641636507, -0.0568631216082, -0.043542188186, -0.0455994512826, -0.043275771306, 0.000778807997018, -0.0394933828741, -0.0119006848015, 0.0154907301589
+49.0, -0.0854304009012, 0.0506801187398, -0.022373135244, 0.00121513083254, -0.0373437341334, -0.0263657543694, 0.0155053592134, -0.0394933828741, -0.072128454602, -0.0176461251598
+68.0, -0.0854304009012, -0.044641636507, -0.00405032998805, -0.00911348124867, -0.00294491267841, 0.00776742796568, 0.0228686348215, -0.0394933828741, -0.0611765950943, -0.013504018245
+245.0, 0.0453409833355, 0.0506801187398, 0.0606183944448, 0.0310533436263, 0.0287020030602, -0.0473467013093, -0.0544457590643, 0.0712099797536, 0.133598980013, 0.135611830689
+184.0, -0.0636351701951, -0.044641636507, 0.0358287167455, -0.0228849640236, -0.0304639698424, -0.0188501912864, -0.00658446761116, -0.00259226199818, -0.0259524244352, -0.0549250873933
+202.0, -0.0672677086461, 0.0506801187398, -0.0126728265791, -0.0400993174923, -0.0153284884022, 0.00463594334778, -0.0581273968684, 0.0343088588777, 0.0191990330786, -0.0342145528191
+137.0, -0.107225631607, -0.044641636507, -0.0773415510119, -0.0263278347174, -0.0896299427451, -0.0961978613484, 0.0265502726256, -0.07639450375, -0.0425721049228, -0.0052198044153
+85.0, -0.0236772472339, -0.044641636507, 0.0595405823709, -0.0400993174923, -0.0428475455662, -0.0435889197678, 0.0118237214093, -0.0394933828741, -0.0159982677581, 0.0403433716479
+131.0, 0.0526060602375, -0.044641636507, -0.0212953231701, -0.0745280244297, -0.0400956398498, -0.0376390989938, -0.00658446761116, -0.0394933828741, -0.000609254186102, -0.0549250873933
+283.0, 0.0671362140416, 0.0506801187398, -0.00620595413581, 0.0631868033198, -0.0428475455662, -0.0958847128867, 0.0523217372542, -0.07639450375, 0.0594238004448, 0.0527696923924
+129.0, -0.0600026317441, -0.044641636507, 0.0444512133366, -0.0194420933299, -0.00982467696942, -0.00757684666201, 0.0228686348215, -0.0394933828741, -0.0271286455543, -0.00936191133014
+59.0, -0.0236772472339, -0.044641636507, -0.0654856181993, -0.0814137658171, -0.0387196869916, -0.0536096705451, 0.0596850128624, -0.07639450375, -0.0371283460105, -0.0424987666488
+341.0, 0.0344433679824, 0.0506801187398, 0.125287118878, 0.0287580963824, -0.0538551684319, -0.0129003705124, -0.102307050517, 0.10811110063, 0.000271485727907, 0.0279170509034
+87.0, 0.0308108295314, -0.044641636507, -0.0503962491649, -0.0022277398612, -0.0442234984244, -0.0899348921127, 0.118591217728, -0.07639450375, -0.0181182673079, 0.00306440941437
+65.0, 0.0162806757273, -0.044641636507, -0.0633299940515, -0.057313670961, -0.0579830270065, -0.0489124436182, 0.00814208360519, -0.0394933828741, -0.0594726974107, -0.0673514081378
+102.0, 0.0489735217865, 0.0506801187398, -0.0309956318351, -0.0492803060204, 0.0493412959332, -0.00413221358232, 0.133317768944, -0.0535158088069, 0.0213108465682, 0.0196328370737
+265.0, 0.0126481372763, -0.044641636507, 0.022894971859, 0.0528581912386, 0.0080627101872, -0.0285577936019, 0.0375951860379, -0.0394933828741, 0.0547240033482, -0.0259303389895
+276.0, -0.00914709342983, -0.044641636507, 0.0110390390463, -0.057313670961, -0.0249601584096, -0.0429626228442, 0.0302319104297, -0.0394933828741, 0.0170371324148, -0.0052198044153
+252.0, -0.00188201652779, 0.0506801187398, 0.0713965151836, 0.0976155102572, 0.0878679759629, 0.0754074957122, -0.0213110188275, 0.0712099797536, 0.0714240327806, 0.0237749439885
+90.0, -0.00188201652779, 0.0506801187398, 0.0142724752679, -0.0745280244297, 0.00255889875439, 0.00620168565673, -0.0139477432193, -0.00259226199818, 0.0191990330786, 0.00306440941437
+100.0, 0.00538306037425, 0.0506801187398, -0.00836157828357, 0.021872354995, 0.054845107366, 0.0732154564797, -0.0249926566316, 0.0343088588777, 0.0125531528134, 0.0941907615407
+55.0, -0.0999605547053, -0.044641636507, -0.067641242347, -0.108956731367, -0.0744944613049, -0.0727117267142, 0.0155053592134, -0.0394933828741, -0.0498684677352, -0.00936191133014
+61.0, -0.0600026317441, 0.0506801187398, -0.0105172024313, -0.014851599083, -0.0497273098573, -0.0235474182133, -0.0581273968684, 0.0158582984398, -0.00991895736315, -0.0342145528191
+92.0, 0.0199132141783, -0.044641636507, -0.0234509473179, -0.0710851537359, 0.020446285911, -0.0100820343563, 0.118591217728, -0.07639450375, -0.0425721049228, 0.0734802269666
+259.0, 0.0453409833355, 0.0506801187398, 0.068163078962, 0.00810087222001, -0.0167044412604, 0.00463594334778, -0.0765355858888, 0.0712099797536, 0.0324332257796, -0.0176461251598
+53.0, 0.0271782910804, 0.0506801187398, -0.0353068801306, 0.0322009670762, -0.0112006298276, 0.00150445872989, -0.0102661054152, -0.00259226199818, -0.0149564750249, -0.0507829804785
+190.0, -0.0563700932931, -0.044641636507, -0.0115950145052, -0.0332135761048, -0.0469754041408, -0.0476598497711, 0.00446044580111, -0.0394933828741, -0.00797939755454, -0.088061942712
+142.0, -0.0781653239992, -0.044641636507, -0.0730303027164, -0.057313670961, -0.0841261313123, -0.0742774690232, -0.0249926566316, -0.0394933828741, -0.0181182673079, -0.0839198357972
+75.0, 0.0671362140416, 0.0506801187398, -0.0417737525739, 0.0115437429137, 0.00255889875439, 0.00588853719494, 0.041276823842, -0.0394933828741, -0.0594726974107, -0.0217882320746
+142.0, -0.041839939489, 0.0506801187398, 0.0142724752679, -0.00567061055493, -0.0125765826858, 0.00620168565673, -0.0728539480847, 0.0712099797536, 0.0354619386608, -0.013504018245
+155.0, 0.0344433679824, -0.044641636507, -0.00728376620969, 0.0149866136075, -0.0442234984244, -0.037325950532, -0.00290282980707, -0.0394933828741, -0.0213936809404, 0.0072065163292
+225.0, 0.0598711371395, 0.0506801187398, 0.0164280994157, 0.0287580963824, -0.041471592708, -0.0291840905255, -0.0286742944357, -0.00259226199818, -0.00239668149341, -0.0217882320746
+59.0, -0.0527375548421, -0.044641636507, -0.00943939035745, -0.00567061055493, 0.0397096259258, 0.0447189464568, 0.0265502726256, -0.00259226199818, -0.0181182673079, -0.013504018245
+104.0, -0.00914709342983, -0.044641636507, -0.0159062628007, 0.0700725447073, 0.0121905687618, 0.022172257208, 0.0155053592134, -0.00259226199818, -0.0332487872476, 0.0486275854776
+182.0, -0.049105016391, -0.044641636507, 0.0250505960067, 0.00810087222001, 0.020446285911, 0.0177881787429, 0.0523217372542, -0.0394933828741, -0.041180385188, 0.0072065163292
+128.0, -0.041839939489, -0.044641636507, -0.049318437091, -0.0366564467986, -0.00707277125302, -0.0226079728279, 0.085456477491, -0.0394933828741, -0.0664881482228, 0.0072065163292
+52.0, -0.041839939489, -0.044641636507, 0.041217777115, -0.0263278347174, -0.0318399227006, -0.0304366843726, -0.0360375700439, 0.0029429061332, 0.0336568129024, -0.0176461251598
+37.0, -0.0273097856849, -0.044641636507, -0.0633299940515, -0.0504279295735, -0.0896299427451, -0.104339721355, 0.0523217372542, -0.07639450375, -0.056157573095, -0.0673514081378
+170.0, 0.0417084448844, -0.044641636507, -0.0644078061254, 0.0356438377699, 0.0121905687618, -0.0579937490101, 0.181179060397, -0.07639450375, -0.000609254186102, -0.0507829804785
+170.0, 0.0635036755906, 0.0506801187398, -0.0256065714657, 0.0115437429137, 0.0644767773734, 0.0484767279983, 0.0302319104297, -0.00259226199818, 0.0383932482117, 0.0196328370737
+61.0, -0.0709002470972, -0.044641636507, -0.00405032998805, -0.0400993174923, -0.0662387441557, -0.0786615474882, 0.0523217372542, -0.07639450375, -0.0514005352606, -0.0342145528191
+144.0, -0.041839939489, 0.0506801187398, 0.004572166603, -0.0538708002672, -0.0442234984244, -0.0273051997547, -0.0802172236929, 0.0712099797536, 0.0366457977934, 0.0196328370737
+52.0, -0.0273097856849, 0.0506801187398, -0.00728376620969, -0.0400993174923, -0.0112006298276, -0.0138398158978, 0.0596850128624, -0.0394933828741, -0.0823814832581, -0.0259303389895
+128.0, -0.034574862587, -0.044641636507, -0.0374625042784, -0.0607565416547, 0.020446285911, 0.0434663526097, -0.0139477432193, -0.00259226199818, -0.0307512098646, -0.0714935150527
+71.0, 0.0671362140416, 0.0506801187398, -0.0256065714657, -0.0400993174923, -0.0634868384393, -0.0598726397809, -0.00290282980707, -0.0394933828741, -0.0191970476139, 0.011348623244
+163.0, -0.04547247794, 0.0506801187398, -0.0245287593918, 0.0597439326261, 0.00531080447079, 0.0149698425868, -0.0544457590643, 0.0712099797536, 0.0423448954496, 0.0154907301589
+150.0, -0.00914709342983, 0.0506801187398, -0.0180618869485, -0.0332135761048, -0.020832299835, 0.0121515064307, -0.0728539480847, 0.0712099797536, 0.000271485727907, 0.0196328370737
+97.0, 0.0417084448844, 0.0506801187398, -0.0148284507269, -0.0171468461892, -0.00569681839481, 0.00839372488926, -0.0139477432193, -0.00185423958066, -0.0119006848015, 0.00306440941437
+160.0, 0.0380759064334, 0.0506801187398, -0.0299178197612, -0.0400993174923, -0.0332158755588, -0.0241737151369, -0.0102661054152, -0.00259226199818, -0.0129079422542, 0.00306440941437
+178.0, 0.0162806757273, -0.044641636507, -0.0460850008694, -0.00567061055493, -0.0758704141631, -0.0614383820898, -0.0139477432193, -0.0394933828741, -0.0514005352606, 0.0196328370737
+48.0, -0.00188201652779, -0.044641636507, -0.0697968664948, -0.0125563519424, -0.00019300696201, -0.00914258897096, 0.0707299262747, -0.0394933828741, -0.0629129499163, 0.0403433716479
+270.0, -0.00188201652779, -0.044641636507, 0.0336730925978, 0.125158475807, 0.0245741444856, 0.0262431872113, -0.0102661054152, -0.00259226199818, 0.0267142576335, 0.0610539062221
+202.0, 0.0635036755906, 0.0506801187398, -0.00405032998805, -0.0125563519424, 0.103003457403, 0.0487898764601, 0.0560033750583, -0.00259226199818, 0.0844952822124, -0.0176461251598
+111.0, 0.0126481372763, 0.0506801187398, -0.0202175110963, -0.0022277398612, 0.0383336730676, 0.0531739549252, -0.00658446761116, 0.0343088588777, -0.00514530798026, -0.00936191133014
+85.0, 0.0126481372763, 0.0506801187398, 0.00241654245524, 0.0563010619323, 0.027326050202, 0.0171618818194, 0.041276823842, -0.0394933828741, 0.00371173823344, 0.0734802269666
+42.0, -0.00914709342983, 0.0506801187398, -0.0309956318351, -0.0263278347174, -0.0112006298276, -0.00100072896443, -0.0213110188275, -0.00259226199818, 0.00620931561651, 0.0279170509034
+170.0, -0.0309423241359, 0.0506801187398, 0.0282840322284, 0.0700725447073, -0.126780669917, -0.106844909049, -0.0544457590643, -0.0479806406756, -0.0307512098646, 0.0154907301589
+200.0, -0.0963280162543, -0.044641636507, -0.0363846922045, -0.0745280244297, -0.0387196869916, -0.0276183482165, 0.0155053592134, -0.0394933828741, -0.0740888714915, -0.00107769750047
+252.0, 0.00538306037425, -0.044641636507, -0.0579409336821, -0.0228849640236, -0.0676146970139, -0.0683276482492, -0.0544457590643, -0.00259226199818, 0.0428956878925, -0.0839198357972
+113.0, -0.103593093156, -0.044641636507, -0.0374625042784, -0.0263278347174, 0.00255889875439, 0.0199802179755, 0.0118237214093, -0.00259226199818, -0.0683297436244, -0.0259303389895
+143.0, 0.0707687524926, -0.044641636507, 0.0121168511202, 0.0425295791574, 0.0713565416644, 0.0534871033869, 0.0523217372542, -0.00259226199818, 0.0253931349154, -0.0052198044153
+51.0, 0.0126481372763, 0.0506801187398, -0.022373135244, -0.0297707054111, 0.0108146159036, 0.0284352264438, -0.0213110188275, 0.0343088588777, -0.00608024819631, -0.00107769750047
+52.0, -0.0164121703319, -0.044641636507, -0.0353068801306, -0.0263278347174, 0.0328298616348, 0.0171618818194, 0.100183028707, -0.0394933828741, -0.0702093127287, -0.0797777288823
+210.0, -0.038207401038, -0.044641636507, 0.00996122697241, -0.0469850588798, -0.0593589798647, -0.0529833736215, -0.0102661054152, -0.0394933828741, -0.0159982677581, -0.0424987666488
+65.0, 0.00175052192323, -0.044641636507, -0.0396181284261, -0.100923366426, -0.0290880169842, -0.0301235359109, 0.0449584616461, -0.0501947079281, -0.0683297436244, -0.12948301186
+141.0, 0.0453409833355, -0.044641636507, 0.0713965151836, 0.00121513083254, -0.00982467696942, -0.00100072896443, 0.0155053592134, -0.0394933828741, -0.041180385188, -0.0714935150527
+55.0, -0.0709002470972, 0.0506801187398, -0.0751859268642, -0.0400993174923, -0.0511032627155, -0.015092409745, -0.0397192078479, -0.00259226199818, -0.0964332228918, -0.0342145528191
+134.0, 0.0453409833355, -0.044641636507, -0.00620595413581, 0.0115437429137, 0.0631008245152, 0.016222436434, 0.0965013909033, -0.0394933828741, 0.0428956878925, -0.038356659734
+42.0, -0.0527375548421, 0.0506801187398, -0.0406959405, -0.0676422830422, -0.0318399227006, -0.0370128020702, 0.0375951860379, -0.0394933828741, -0.0345237153303, 0.0693381200517
+111.0, -0.04547247794, -0.044641636507, -0.0482406250172, -0.0194420933299, -0.00019300696201, -0.0160318551303, 0.0670482884706, -0.0394933828741, -0.0247911874325, 0.0196328370737
+98.0, 0.0126481372763, -0.044641636507, -0.0256065714657, -0.0400993174923, -0.0304639698424, -0.0451546620768, 0.0780932018828, -0.07639450375, -0.072128454602, 0.011348623244
+164.0, 0.0453409833355, -0.044641636507, 0.0519958978538, -0.0538708002672, 0.0631008245152, 0.0647604480114, -0.0102661054152, 0.0343088588777, 0.037232011209, 0.0196328370737
+48.0, -0.0200447087829, -0.044641636507, 0.004572166603, 0.0976155102572, 0.00531080447079, -0.0207290820572, 0.0633666506665, -0.0394933828741, 0.0125531528134, 0.011348623244
+96.0, -0.049105016391, -0.044641636507, -0.0644078061254, -0.10207098998, -0.00294491267841, -0.0154055582067, 0.0633666506665, -0.047242618258, -0.0332487872476, -0.0549250873933
+90.0, -0.0781653239992, -0.044641636507, -0.0169840748746, -0.0125563519424, -0.00019300696201, -0.013526667436, 0.0707299262747, -0.0394933828741, -0.041180385188, -0.0922040496268
+162.0, -0.0709002470972, -0.044641636507, -0.0579409336821, -0.0814137658171, -0.0455994512826, -0.0288709420637, -0.043400845652, -0.00259226199818, 0.00114379737951, -0.0052198044153
+150.0, 0.0562385986885, 0.0506801187398, 0.00996122697241, 0.0494153205448, -0.00432086553661, -0.0122740735889, -0.043400845652, 0.0343088588777, 0.0607877541507, 0.0320591578182
+279.0, -0.0273097856849, -0.044641636507, 0.0886415083657, -0.0251802111642, 0.0218222387692, 0.0425269072243, -0.0323559322398, 0.0343088588777, 0.00286377051894, 0.0776223338814
+92.0, 0.00175052192323, 0.0506801187398, -0.00512814206193, -0.0125563519424, -0.0153284884022, -0.0138398158978, 0.00814208360519, -0.0394933828741, -0.00608024819631, -0.0673514081378
+83.0, -0.00188201652779, -0.044641636507, -0.0644078061254, 0.0115437429137, 0.027326050202, 0.0375165318357, -0.0139477432193, 0.0343088588777, 0.0117839003836, -0.0549250873933
+128.0, 0.0162806757273, -0.044641636507, 0.0175059114896, -0.0228849640236, 0.0603489187988, 0.0444057979951, 0.0302319104297, -0.00259226199818, 0.037232011209, -0.00107769750047
+102.0, 0.0162806757273, 0.0506801187398, -0.0450071887955, 0.0631868033198, 0.0108146159036, -0.00037443204085, 0.0633666506665, -0.0394933828741, -0.0307512098646, 0.036201264733
+302.0, -0.0926954778033, -0.044641636507, 0.0282840322284, -0.0159992226361, 0.0369577202094, 0.0249905933641, 0.0560033750583, -0.0394933828741, -0.00514530798026, -0.00107769750047
+198.0, 0.0598711371395, 0.0506801187398, 0.041217777115, 0.0115437429137, 0.041085578784, 0.0707102687854, -0.0360375700439, 0.0343088588777, -0.0109044358474, -0.0300724459043
+95.0, -0.0273097856849, -0.044641636507, 0.0649296427403, -0.0022277398612, -0.0249601584096, -0.0172844489775, 0.0228686348215, -0.0394933828741, -0.0611765950943, -0.063209301223
+53.0, 0.0235457526293, 0.0506801187398, -0.0320734439089, -0.0400993174923, -0.0318399227006, -0.0216685274425, -0.0139477432193, -0.00259226199818, -0.0109044358474, 0.0196328370737
+134.0, -0.0963280162543, -0.044641636507, -0.0762637389381, -0.043542188186, -0.0455994512826, -0.0348207628377, 0.00814208360519, -0.0394933828741, -0.0594726974107, -0.0839198357972
+144.0, 0.0271782910804, -0.044641636507, 0.049840273706, -0.0550184238203, -0.00294491267841, 0.0406480164536, -0.0581273968684, 0.0527594193157, -0.0529587932392, -0.0052198044153
+232.0, 0.0199132141783, 0.0506801187398, 0.0455290254105, 0.0299057198322, -0.0621108855811, -0.0558017097776, -0.0728539480847, 0.0269286347025, 0.0456008084141, 0.0403433716479
+81.0, 0.0380759064334, 0.0506801187398, -0.00943939035745, 0.00236275438564, 0.00118294589619, 0.0375165318357, -0.0544457590643, 0.0501763408544, -0.0259524244352, 0.106617082285
+104.0, 0.0417084448844, 0.0506801187398, -0.0320734439089, -0.0228849640236, -0.0497273098573, -0.0401442866881, 0.0302319104297, -0.0394933828741, -0.12609738556, 0.0154907301589
+59.0, 0.0199132141783, -0.044641636507, 0.004572166603, -0.0263278347174, 0.0231981916274, 0.01027261566, 0.0670482884706, -0.0394933828741, -0.0236445575721, -0.0466408735636
+246.0, -0.0854304009012, -0.044641636507, 0.0207393477112, -0.0263278347174, 0.00531080447079, 0.0196670695137, -0.00290282980707, -0.00259226199818, -0.0236445575721, 0.00306440941437
+297.0, 0.0199132141783, 0.0506801187398, 0.0142724752679, 0.0631868033198, 0.0149424744782, 0.0202933664373, -0.0470824834561, 0.0343088588777, 0.0466607723568, 0.0900486546259
+258.0, 0.0235457526293, -0.044641636507, 0.110197749843, 0.0631868033198, 0.01356652162, -0.032941872067, -0.0249926566316, 0.0206554441536, 0.099240225734, 0.0237749439885
+229.0, -0.0309423241359, 0.0506801187398, 0.00133873038136, -0.00567061055493, 0.0644767773734, 0.0494161733837, -0.0470824834561, 0.10811110063, 0.0837967663655, 0.00306440941437
+275.0, 0.0489735217865, 0.0506801187398, 0.058462770297, 0.0700725447073, 0.01356652162, 0.020606514899, -0.0213110188275, 0.0343088588777, 0.0220040504562, 0.0279170509034
+281.0, 0.0598711371395, -0.044641636507, -0.0212953231701, 0.0872868981759, 0.0452134373586, 0.0315667110617, -0.0470824834561, 0.0712099797536, 0.0791210813897, 0.135611830689
+179.0, -0.0563700932931, 0.0506801187398, -0.0105172024313, 0.0253152256887, 0.0231981916274, 0.04002171953, -0.0397192078479, 0.0343088588777, 0.0206123307214, 0.0569117993072
+200.0, 0.0162806757273, -0.044641636507, -0.0471628129433, -0.0022277398612, -0.0194563469768, -0.0429626228442, 0.0339135482338, -0.0394933828741, 0.0273677075426, 0.0279170509034
+200.0, -0.049105016391, -0.044641636507, 0.004572166603, 0.0115437429137, -0.0373437341334, -0.0185370428246, -0.0176293810234, -0.00259226199818, -0.0398095943643, -0.0217882320746
+173.0, 0.0635036755906, -0.044641636507, 0.0175059114896, 0.021872354995, 0.0080627101872, 0.0215459602844, -0.0360375700439, 0.0343088588777, 0.0199084208763, 0.011348623244
+180.0, 0.0489735217865, 0.0506801187398, 0.0810968238485, 0.021872354995, 0.0438374845004, 0.0641341510878, -0.0544457590643, 0.0712099797536, 0.0324332257796, 0.0486275854776
+84.0, 0.00538306037425, 0.0506801187398, 0.0347509046717, -0.0010801163081, 0.152537760298, 0.198787989657, -0.0618090346725, 0.18523444326, 0.0155668445407, 0.0734802269666
+121.0, -0.00551455497881, -0.044641636507, 0.0239727839329, 0.00810087222001, -0.034591828417, -0.038891692841, 0.0228686348215, -0.0394933828741, -0.0159982677581, -0.013504018245
+161.0, -0.00551455497881, 0.0506801187398, -0.00836157828357, -0.0022277398612, -0.0332158755588, -0.0636304213223, -0.0360375700439, -0.00259226199818, 0.0805854642387, 0.0072065163292
+99.0, -0.0890629393523, -0.044641636507, -0.0611743699037, -0.0263278347174, -0.0552311212901, -0.0545491159304, 0.041276823842, -0.07639450375, -0.0939356455087, -0.0549250873933
+109.0, 0.0344433679824, 0.0506801187398, -0.00189470584028, -0.0125563519424, 0.0383336730676, 0.0137172487397, 0.0780932018828, -0.0394933828741, 0.00455189046613, -0.0963461565417
+115.0, -0.0527375548421, -0.044641636507, -0.0622521819776, -0.0263278347174, -0.00569681839481, -0.00507165896769, 0.0302319104297, -0.0394933828741, -0.0307512098646, -0.0714935150527
+268.0, 0.00901559882527, -0.044641636507, 0.0164280994157, 0.00465800152627, 0.0094386630454, 0.0105857641218, -0.0286742944357, 0.0343088588777, 0.0389683660309, 0.11904340303
+274.0, -0.0636351701951, 0.0506801187398, 0.0961861928829, 0.104501251645, -0.00294491267841, -0.0047585105059, -0.00658446761116, -0.00259226199818, 0.0226920225667, 0.0734802269666
+158.0, -0.0963280162543, -0.044641636507, -0.0697968664948, -0.0676422830422, -0.0194563469768, -0.0107083312799, 0.0155053592134, -0.0394933828741, -0.0468794828442, -0.0797777288823
+107.0, 0.0162806757273, 0.0506801187398, -0.0212953231701, -0.00911348124867, 0.034205814493, 0.0478504310747, 0.000778807997018, -0.00259226199818, -0.0129079422542, 0.0237749439885
+83.0, -0.041839939489, 0.0506801187398, -0.0536296853866, -0.0400993174923, -0.0841261313123, -0.0717722813289, -0.00290282980707, -0.0394933828741, -0.072128454602, -0.0300724459043
+103.0, -0.0745327855482, -0.044641636507, 0.0433734012627, -0.0332135761048, 0.0121905687618, 0.000251864882729, 0.0633666506665, -0.0394933828741, -0.0271286455543, -0.0466408735636
+272.0, -0.00551455497881, -0.044641636507, 0.0563071461493, -0.0366564467986, -0.048351356999, -0.0429626228442, -0.0728539480847, 0.0379989709653, 0.050781513363, 0.0569117993072
+85.0, -0.0926954778033, -0.044641636507, -0.0816527993075, -0.057313670961, -0.0607349327229, -0.0680144997874, 0.0486400994501, -0.07639450375, -0.0664881482228, -0.0217882320746
+280.0, 0.00538306037425, -0.044641636507, 0.049840273706, 0.0976155102572, -0.0153284884022, -0.0163450035921, -0.00658446761116, -0.00259226199818, 0.0170371324148, -0.013504018245
+336.0, 0.0344433679824, 0.0506801187398, 0.111275561917, 0.0769582860947, -0.0318399227006, -0.0338813174523, -0.0213110188275, -0.00259226199818, 0.0280165065233, 0.0734802269666
+281.0, 0.0235457526293, -0.044641636507, 0.0616962065187, 0.0528581912386, -0.034591828417, -0.0489124436182, -0.0286742944357, -0.00259226199818, 0.0547240033482, -0.0052198044153
+118.0, 0.0417084448844, 0.0506801187398, 0.0142724752679, 0.0425295791574, -0.0304639698424, -0.00131387742622, -0.043400845652, -0.00259226199818, -0.0332487872476, 0.0154907301589
+317.0, -0.0273097856849, -0.044641636507, 0.0476846495582, -0.0469850588798, 0.034205814493, 0.0572448849284, -0.0802172236929, 0.130251773155, 0.0450661683363, 0.131469723774
+235.0, 0.0417084448844, 0.0506801187398, 0.0121168511202, 0.0390867084636, 0.054845107366, 0.0444057979951, 0.00446044580111, -0.00259226199818, 0.0456008084141, -0.00107769750047
+60.0, -0.0309423241359, -0.044641636507, 0.00564997867688, -0.00911348124867, 0.0190703330528, 0.00682798258031, 0.0744115640788, -0.0394933828741, -0.041180385188, -0.0424987666488
+174.0, 0.0308108295314, 0.0506801187398, 0.0466068374844, -0.0159992226361, 0.020446285911, 0.0506687672308, -0.0581273968684, 0.0712099797536, 0.00620931561651, 0.0072065163292
+259.0, -0.041839939489, -0.044641636507, 0.128520555099, 0.0631868033198, -0.0332158755588, -0.0326287236052, 0.0118237214093, -0.0394933828741, -0.0159982677581, -0.0507829804785
+178.0, -0.0309423241359, 0.0506801187398, 0.0595405823709, 0.00121513083254, 0.0121905687618, 0.0315667110617, -0.043400845652, 0.0343088588777, 0.0148227108413, 0.0072065163292
+128.0, -0.0563700932931, -0.044641636507, 0.0929527566612, -0.0194420933299, 0.0149424744782, 0.0234248510552, -0.0286742944357, 0.0254525898675, 0.0260560896337, 0.0403433716479
+96.0, -0.0600026317441, 0.0506801187398, 0.0153502873418, -0.0194420933299, 0.0369577202094, 0.0481635795365, 0.0191869970175, -0.00259226199818, -0.0307512098646, -0.00107769750047
+126.0, -0.049105016391, 0.0506801187398, -0.00512814206193, -0.0469850588798, -0.020832299835, -0.0204159335954, -0.0691723102806, 0.0712099797536, 0.0612379075197, -0.038356659734
+288.0, 0.0235457526293, -0.044641636507, 0.0703187031097, 0.0253152256887, -0.034591828417, -0.0144661128214, -0.0323559322398, -0.00259226199818, -0.0191970476139, -0.00936191133014
+88.0, 0.00175052192323, -0.044641636507, -0.00405032998805, -0.00567061055493, -0.00844872411122, -0.0238605666751, 0.0523217372542, -0.0394933828741, -0.0089440189578, -0.013504018245
+292.0, -0.034574862587, 0.0506801187398, -0.000816893766404, 0.0700725447073, 0.0397096259258, 0.0669524872439, -0.0654906724765, 0.10811110063, 0.0267142576335, 0.0734802269666
+71.0, 0.0417084448844, 0.0506801187398, -0.0439293767216, 0.0631868033198, -0.00432086553661, 0.016222436434, -0.0139477432193, -0.00259226199818, -0.0345237153303, 0.011348623244
+197.0, 0.0671362140416, 0.0506801187398, 0.0207393477112, -0.00567061055493, 0.020446285911, 0.0262431872113, -0.00290282980707, -0.00259226199818, 0.00864028293306, 0.00306440941437
+186.0, -0.0273097856849, 0.0506801187398, 0.0606183944448, 0.0494153205448, 0.0851160702465, 0.0863676918749, -0.00290282980707, 0.0343088588777, 0.0378144788263, 0.0486275854776
+25.0, -0.0164121703319, -0.044641636507, -0.0105172024313, 0.00121513083254, -0.0373437341334, -0.0357602082231, 0.0118237214093, -0.0394933828741, -0.0213936809404, -0.0342145528191
+84.0, -0.00188201652779, 0.0506801187398, -0.0331512559828, -0.0182944697768, 0.0314539087766, 0.0428400556861, -0.0139477432193, 0.0199174217361, 0.010225642405, 0.0279170509034
+96.0, -0.0127796318808, -0.044641636507, -0.0654856181993, -0.0699375301828, 0.00118294589619, 0.0168487333576, -0.00290282980707, -0.00702039650329, -0.0307512098646, -0.0507829804785
+195.0, -0.00551455497881, -0.044641636507, 0.0433734012627, 0.0872868981759, 0.01356652162, 0.0071411310421, -0.0139477432193, -0.00259226199818, 0.0423448954496, -0.0176461251598
+53.0, -0.00914709342983, -0.044641636507, -0.0622521819776, -0.0745280244297, -0.0235842055514, -0.0132135189742, 0.00446044580111, -0.0394933828741, -0.0358167281015, -0.0466408735636
+217.0, -0.04547247794, 0.0506801187398, 0.0638518306665, 0.0700725447073, 0.133274420283, 0.131461070373, -0.0397192078479, 0.10811110063, 0.0757375884575, 0.0859065477111
+172.0, -0.0527375548421, -0.044641636507, 0.0304396563761, -0.0745280244297, -0.0235842055514, -0.0113346282035, -0.00290282980707, -0.00259226199818, -0.0307512098646, -0.00107769750047
+131.0, 0.0162806757273, 0.0506801187398, 0.0724743272575, 0.0769582860947, -0.00844872411122, 0.00557538873315, -0.00658446761116, -0.00259226199818, -0.0236445575721, 0.0610539062221
+214.0, 0.0453409833355, -0.044641636507, -0.0191396990224, 0.021872354995, 0.027326050202, -0.013526667436, 0.100183028707, -0.0394933828741, 0.0177634778671, -0.013504018245
+59.0, -0.041839939489, -0.044641636507, -0.0665634302731, -0.0469850588798, -0.0373437341334, -0.043275771306, 0.0486400994501, -0.0394933828741, -0.056157573095, -0.013504018245
+70.0, -0.0563700932931, 0.0506801187398, -0.0600965578299, -0.0366564467986, -0.0882539898869, -0.0708328359435, -0.0139477432193, -0.0394933828741, -0.0781409106691, -0.104630370371
+220.0, 0.0707687524926, -0.044641636507, 0.0692408910359, 0.0379390850138, 0.0218222387692, 0.00150445872989, -0.0360375700439, 0.0391060045916, 0.0776327891956, 0.106617082285
+268.0, 0.00175052192323, 0.0506801187398, 0.0595405823709, -0.0022277398612, 0.061724871657, 0.0631947057024, -0.0581273968684, 0.10811110063, 0.0689822116363, 0.127327616859
+152.0, -0.00188201652779, -0.044641636507, -0.0266843835395, 0.0494153205448, 0.0589729659406, -0.0160318551303, -0.0470824834561, 0.0712099797536, 0.133598980013, 0.0196328370737
+47.0, 0.0235457526293, 0.0506801187398, -0.0202175110963, -0.0366564467986, -0.013952535544, -0.015092409745, 0.0596850128624, -0.0394933828741, -0.0964332228918, -0.0176461251598
+74.0, -0.0200447087829, -0.044641636507, -0.0460850008694, -0.0986281192858, -0.0758704141631, -0.0598726397809, -0.0176293810234, -0.0394933828741, -0.0514005352606, -0.0466408735636
+295.0, 0.0417084448844, 0.0506801187398, 0.0713965151836, 0.00810087222001, 0.0383336730676, 0.0159092879722, -0.0176293810234, 0.0343088588777, 0.0734100780491, 0.0859065477111
+101.0, -0.0636351701951, 0.0506801187398, -0.0794971751597, -0.00567061055493, -0.0717425555885, -0.0664487574784, -0.0102661054152, -0.0394933828741, -0.0181182673079, -0.0549250873933
+151.0, 0.0162806757273, 0.0506801187398, 0.00996122697241, -0.043542188186, -0.0965097070361, -0.0946321190395, -0.0397192078479, -0.0394933828741, 0.0170371324148, 0.0072065163292
+127.0, 0.0671362140416, -0.044641636507, -0.0385403163522, -0.0263278347174, -0.0318399227006, -0.0263657543694, 0.00814208360519, -0.0394933828741, -0.0271286455543, 0.00306440941437
+237.0, 0.0453409833355, 0.0506801187398, 0.0196615356373, 0.0390867084636, 0.020446285911, 0.0259300387495, 0.00814208360519, -0.00259226199818, -0.00330371257868, 0.0196328370737
+225.0, 0.0489735217865, -0.044641636507, 0.0272062201545, -0.0251802111642, 0.0231981916274, 0.0184144756665, -0.0618090346725, 0.0800662487639, 0.0722236508199, 0.0320591578182
+81.0, 0.0417084448844, -0.044641636507, -0.00836157828357, -0.0263278347174, 0.0245741444856, 0.016222436434, 0.0707299262747, -0.0394933828741, -0.0483617248029, -0.0300724459043
+151.0, -0.0236772472339, -0.044641636507, -0.0159062628007, -0.0125563519424, 0.020446285911, 0.0412743133772, -0.043400845652, 0.0343088588777, 0.0140724525158, -0.00936191133014
+107.0, -0.038207401038, 0.0506801187398, 0.004572166603, 0.0356438377699, -0.0112006298276, 0.00588853719494, -0.0470824834561, 0.0343088588777, 0.0163049527999, -0.00107769750047
+64.0, 0.0489735217865, -0.044641636507, -0.0428515646478, -0.0538708002672, 0.0452134373586, 0.0500424703073, 0.0339135482338, -0.00259226199818, -0.0259524244352, -0.063209301223
+138.0, 0.0453409833355, 0.0506801187398, 0.00564997867688, 0.0563010619323, 0.0644767773734, 0.089186028031, -0.0397192078479, 0.0712099797536, 0.0155668445407, -0.00936191133014
+185.0, 0.0453409833355, 0.0506801187398, -0.0353068801306, 0.0631868033198, -0.00432086553661, -0.00162702588801, -0.0102661054152, -0.00259226199818, 0.0155668445407, 0.0569117993072
+265.0, 0.0162806757273, -0.044641636507, 0.0239727839329, -0.0228849640236, -0.0249601584096, -0.0260526059076, -0.0323559322398, -0.00259226199818, 0.037232011209, 0.0320591578182
+101.0, -0.0745327855482, 0.0506801187398, -0.0180618869485, 0.00810087222001, -0.0194563469768, -0.0248000120604, -0.0654906724765, 0.0343088588777, 0.0673172179147, -0.0176461251598
+137.0, -0.0817978624502, 0.0506801187398, 0.0422955891888, -0.0194420933299, 0.0397096259258, 0.0575580333902, -0.0691723102806, 0.10811110063, 0.047186167886, -0.038356659734
+143.0, -0.0672677086461, -0.044641636507, -0.0547074974604, -0.0263278347174, -0.0758704141631, -0.0821061805679, 0.0486400994501, -0.07639450375, -0.0868289932163, -0.104630370371
+141.0, 0.00538306037425, -0.044641636507, -0.00297251791417, 0.0494153205448, 0.0741084473809, 0.0707102687854, 0.0449584616461, -0.00259226199818, -0.00149858682029, -0.00936191133014
+79.0, -0.00188201652779, -0.044641636507, -0.0665634302731, 0.00121513083254, -0.00294491267841, 0.00307020103883, 0.0118237214093, -0.00259226199818, -0.0202887477516, -0.0259303389895
+292.0, 0.00901559882527, -0.044641636507, -0.0126728265791, 0.0287580963824, -0.0180803941186, -0.00507165896769, -0.0470824834561, 0.0343088588777, 0.0233748412798, -0.0052198044153
+178.0, -0.00551455497881, 0.0506801187398, -0.0417737525739, -0.043542188186, -0.0799982727377, -0.0761563597939, -0.0323559322398, -0.0394933828741, 0.010225642405, -0.00936191133014
+91.0, 0.0562385986885, 0.0506801187398, -0.0309956318351, 0.00810087222001, 0.0190703330528, 0.0212328118226, 0.0339135482338, -0.0394933828741, -0.0295276227418, -0.0590671943082
+116.0, 0.00901559882527, 0.0506801187398, -0.00512814206193, -0.0641994123485, 0.0699805888062, 0.0838625041805, -0.0397192078479, 0.0712099797536, 0.039539878072, 0.0196328370737
+86.0, -0.0672677086461, -0.044641636507, -0.059018745756, 0.0322009670762, -0.0511032627155, -0.0495387405418, -0.0102661054152, -0.0394933828741, 0.00200784054982, 0.0237749439885
+122.0, 0.0271782910804, 0.0506801187398, 0.0250505960067, 0.0149866136075, 0.0259500973438, 0.0484767279983, -0.0397192078479, 0.0343088588777, 0.00783714230182, 0.0237749439885
+72.0, -0.0236772472339, -0.044641636507, -0.0460850008694, -0.0332135761048, 0.0328298616348, 0.0362639379885, 0.0375951860379, -0.00259226199818, -0.0332487872476, 0.011348623244
+129.0, 0.0489735217865, 0.0506801187398, 0.00349435452912, 0.0700725447073, -0.00844872411122, 0.0134041002779, -0.0544457590643, 0.0343088588777, 0.0133159679089, 0.036201264733
+142.0, -0.0527375548421, -0.044641636507, 0.0541515220015, -0.0263278347174, -0.0552311212901, -0.0338813174523, -0.0139477432193, -0.0394933828741, -0.0740888714915, -0.0590671943082
+90.0, 0.0417084448844, -0.044641636507, -0.0450071887955, 0.0344962143201, 0.0438374845004, -0.0157187066685, 0.0375951860379, -0.0144006206785, 0.0898986932777, 0.0072065163292
+158.0, 0.0562385986885, -0.044641636507, -0.0579409336821, -0.00796585769557, 0.0520932016496, 0.0491030249219, 0.0560033750583, -0.0214118336449, -0.028320242548, 0.0444854785627
+39.0, -0.034574862587, 0.0506801187398, -0.0557853095343, -0.0159992226361, -0.00982467696942, -0.0078899951238, 0.0375951860379, -0.0394933828741, -0.0529587932392, 0.0279170509034
+196.0, 0.0816663678457, 0.0506801187398, 0.00133873038136, 0.0356438377699, 0.126394655992, 0.0910649188017, 0.0191869970175, 0.0343088588777, 0.0844952822124, -0.0300724459043
+222.0, -0.00188201652779, 0.0506801187398, 0.0304396563761, 0.0528581912386, 0.0397096259258, 0.0566185880048, -0.0397192078479, 0.0712099797536, 0.0253931349154, 0.0279170509034
+277.0, 0.110726675454, 0.0506801187398, 0.00672779075076, 0.0287580963824, -0.027712064126, -0.00726369820022, -0.0470824834561, 0.0343088588777, 0.00200784054982, 0.0776223338814
+99.0, -0.0309423241359, -0.044641636507, 0.0466068374844, 0.0149866136075, -0.0167044412604, -0.0470335528475, 0.000778807997018, -0.00259226199818, 0.0634559213721, -0.0259303389895
+196.0, 0.00175052192323, 0.0506801187398, 0.0261284080806, -0.00911348124867, 0.0245741444856, 0.0384559772211, -0.0213110188275, 0.0343088588777, 0.00943640914608, 0.00306440941437
+202.0, 0.00901559882527, -0.044641636507, 0.0455290254105, 0.0287580963824, 0.0121905687618, -0.0138398158978, 0.0265502726256, -0.0394933828741, 0.0461323310394, 0.036201264733
+155.0, 0.0308108295314, -0.044641636507, 0.0401399650411, 0.0769582860947, 0.0176943801946, 0.0378296802975, -0.0286742944357, 0.0343088588777, -0.00149858682029, 0.11904340303
+77.0, 0.0380759064334, 0.0506801187398, -0.0180618869485, 0.0666296740135, -0.0511032627155, -0.0166581520539, -0.0765355858888, 0.0343088588777, -0.0119006848015, -0.013504018245
+191.0, 0.00901559882527, -0.044641636507, 0.0142724752679, 0.0149866136075, 0.054845107366, 0.0472241341512, 0.0707299262747, -0.0394933828741, -0.0332487872476, -0.0590671943082
+70.0, 0.0925639831987, -0.044641636507, 0.0369065288194, 0.021872354995, -0.0249601584096, -0.0166581520539, 0.000778807997018, -0.0394933828741, -0.0225121719297, -0.0217882320746
+73.0, 0.0671362140416, -0.044641636507, 0.00349435452912, 0.0356438377699, 0.0493412959332, 0.0312535625999, 0.0707299262747, -0.0394933828741, -0.000609254186102, 0.0196328370737
+49.0, 0.00175052192323, -0.044641636507, -0.0708746785687, -0.0228849640236, -0.00156895982021, -0.00100072896443, 0.0265502726256, -0.0394933828741, -0.0225121719297, 0.0072065163292
+65.0, 0.0308108295314, -0.044641636507, -0.0331512559828, -0.0228849640236, -0.0469754041408, -0.0811667351825, 0.103864666511, -0.07639450375, -0.0398095943643, -0.0549250873933
+263.0, 0.0271782910804, 0.0506801187398, 0.0940305687351, 0.0976155102572, -0.034591828417, -0.0320024266816, -0.043400845652, -0.00259226199818, 0.0366457977934, 0.106617082285
+248.0, 0.0126481372763, 0.0506801187398, 0.0358287167455, 0.0494153205448, 0.0534691545078, 0.0741549018651, -0.0691723102806, 0.145012221505, 0.0456008084141, 0.0486275854776
+296.0, 0.0744012909436, -0.044641636507, 0.03151746845, 0.101058380951, 0.0465893902168, 0.0368902349121, 0.0155053592134, -0.00259226199818, 0.0336568129024, 0.0444854785627
+214.0, -0.041839939489, -0.044641636507, -0.0654856181993, -0.0400993174923, -0.00569681839481, 0.0143435456633, -0.043400845652, 0.0343088588777, 0.00702686254915, -0.013504018245
+185.0, -0.0890629393523, -0.044641636507, -0.0417737525739, -0.0194420933299, -0.0662387441557, -0.0742774690232, 0.00814208360519, -0.0394933828741, 0.00114379737951, -0.0300724459043
+78.0, 0.0235457526293, 0.0506801187398, -0.0396181284261, -0.00567061055493, -0.048351356999, -0.0332550205288, 0.0118237214093, -0.0394933828741, -0.101643547946, -0.0673514081378
+93.0, -0.04547247794, -0.044641636507, -0.0385403163522, -0.0263278347174, -0.0153284884022, 0.000878161806308, -0.0323559322398, -0.00259226199818, 0.00114379737951, -0.038356659734
+252.0, -0.0236772472339, 0.0506801187398, -0.0256065714657, 0.0425295791574, -0.0538551684319, -0.0476598497711, -0.0213110188275, -0.0394933828741, 0.00114379737951, 0.0196328370737
+150.0, -0.0999605547053, -0.044641636507, -0.0234509473179, -0.0641994123485, -0.0579830270065, -0.0601857882427, 0.0118237214093, -0.0394933828741, -0.0181182673079, -0.0507829804785
+77.0, -0.0273097856849, -0.044641636507, -0.0665634302731, -0.112399602061, -0.0497273098573, -0.0413968805353, 0.000778807997018, -0.0394933828741, -0.0358167281015, -0.00936191133014
+208.0, 0.0308108295314, 0.0506801187398, 0.0325952805239, 0.0494153205448, -0.0400956398498, -0.0435889197678, -0.0691723102806, 0.0343088588777, 0.0630166151147, 0.00306440941437
+77.0, -0.103593093156, 0.0506801187398, -0.0460850008694, -0.0263278347174, -0.0249601584096, -0.0248000120604, 0.0302319104297, -0.0394933828741, -0.0398095943643, -0.0549250873933
+108.0, 0.0671362140416, 0.0506801187398, -0.0299178197612, 0.0574486853821, -0.00019300696201, -0.0157187066685, 0.0744115640788, -0.0505637191369, -0.0384591123014, 0.0072065163292
+160.0, -0.0527375548421, -0.044641636507, -0.0126728265791, -0.0607565416547, -0.00019300696201, 0.00808057642747, 0.0118237214093, -0.00259226199818, -0.0271286455543, -0.0507829804785
+53.0, -0.0273097856849, 0.0506801187398, -0.0159062628007, -0.0297707054111, 0.00393485161259, -0.00068758050264, 0.041276823842, -0.0394933828741, -0.0236445575721, 0.011348623244
+220.0, -0.038207401038, 0.0506801187398, 0.0713965151836, -0.057313670961, 0.153913713157, 0.155886650392, 0.000778807997018, 0.0719480021712, 0.05027649339, 0.0693381200517
+154.0, 0.00901559882527, -0.044641636507, -0.0309956318351, 0.021872354995, 0.0080627101872, 0.00870687335105, 0.00446044580111, -0.00259226199818, 0.00943640914608, 0.011348623244
+259.0, 0.0126481372763, 0.0506801187398, 0.000260918307477, -0.0114087283893, 0.0397096259258, 0.0572448849284, -0.0397192078479, 0.0560805201945, 0.0240525832269, 0.0320591578182
+90.0, 0.0671362140416, -0.044641636507, 0.0369065288194, -0.0504279295735, -0.0235842055514, -0.0345076143759, 0.0486400994501, -0.0394933828741, -0.0259524244352, -0.038356659734
+246.0, 0.0453409833355, -0.044641636507, 0.0390621529672, 0.0459724498511, 0.006686757329, -0.0241737151369, 0.00814208360519, -0.0125555646347, 0.0643282330237, 0.0569117993072
+124.0, 0.0671362140416, 0.0506801187398, -0.0148284507269, 0.0585963091762, -0.0593589798647, -0.0345076143759, -0.0618090346725, 0.0129062087697, -0.00514530798026, 0.0486275854776
+67.0, 0.0271782910804, -0.044641636507, 0.00672779075076, 0.0356438377699, 0.0796122588137, 0.0707102687854, 0.0155053592134, 0.0343088588777, 0.0406722637145, 0.011348623244
+72.0, 0.0562385986885, -0.044641636507, -0.0687190544209, -0.0687899065953, -0.00019300696201, -0.00100072896443, 0.0449584616461, -0.0376483268303, -0.0483617248029, -0.00107769750047
+257.0, 0.0344433679824, 0.0506801187398, -0.00943939035745, 0.0597439326261, -0.0359677812752, -0.00757684666201, -0.0765355858888, 0.0712099797536, 0.0110081010459, -0.0217882320746
+262.0, 0.0235457526293, -0.044641636507, 0.0196615356373, -0.0125563519424, 0.0837401173883, 0.0387691256828, 0.0633666506665, -0.00259226199818, 0.0660482061631, 0.0486275854776
+275.0, 0.0489735217865, 0.0506801187398, 0.0746299514053, 0.0666296740135, -0.00982467696942, -0.00225332281159, -0.043400845652, 0.0343088588777, 0.0336568129024, 0.0196328370737
+177.0, 0.0308108295314, 0.0506801187398, -0.00836157828357, 0.00465800152627, 0.0149424744782, 0.0274957810584, 0.00814208360519, -0.00812743012957, -0.0295276227418, 0.0569117993072
+71.0, -0.103593093156, 0.0506801187398, -0.0234509473179, -0.0228849640236, -0.0868780370287, -0.0677013513256, -0.0176293810234, -0.0394933828741, -0.0781409106691, -0.0714935150527
+47.0, 0.0162806757273, 0.0506801187398, -0.0460850008694, 0.0115437429137, -0.0332158755588, -0.0160318551303, -0.0102661054152, -0.00259226199818, -0.0439854025656, -0.0424987666488
+187.0, -0.0600026317441, 0.0506801187398, 0.0541515220015, -0.0194420933299, -0.0497273098573, -0.0489124436182, 0.0228686348215, -0.0394933828741, -0.0439854025656, -0.0052198044153
+125.0, -0.0273097856849, -0.044641636507, -0.0353068801306, -0.0297707054111, -0.0566070741483, -0.0586200459337, 0.0302319104297, -0.0394933828741, -0.0498684677352, -0.12948301186
+78.0, 0.0417084448844, -0.044641636507, -0.0320734439089, -0.0619041652078, 0.0796122588137, 0.0509819156926, 0.0560033750583, -0.00997248617336, 0.0450661683363, -0.0590671943082
+51.0, -0.0817978624502, -0.044641636507, -0.0816527993075, -0.0400993174923, 0.00255889875439, -0.0185370428246, 0.0707299262747, -0.0394933828741, -0.0109044358474, -0.0922040496268
+258.0, -0.041839939489, -0.044641636507, 0.0476846495582, 0.0597439326261, 0.127770608851, 0.128016437293, -0.0249926566316, 0.10811110063, 0.0638931206368, 0.0403433716479
+215.0, -0.0127796318808, -0.044641636507, 0.0606183944448, 0.0528581912386, 0.047965343075, 0.0293746718292, -0.0176293810234, 0.0343088588777, 0.0702112981933, 0.0072065163292
+303.0, 0.0671362140416, -0.044641636507, 0.0563071461493, 0.073515415401, -0.013952535544, -0.0392048413028, -0.0323559322398, -0.00259226199818, 0.0757375884575, 0.036201264733
+243.0, -0.0527375548421, 0.0506801187398, 0.0983418170306, 0.0872868981759, 0.0603489187988, 0.0487898764601, -0.0581273968684, 0.10811110063, 0.0844952822124, 0.0403433716479
+91.0, 0.00538306037425, -0.044641636507, 0.0595405823709, -0.0561660474079, 0.0245741444856, 0.0528608064634, -0.043400845652, 0.0509143632719, -0.00421985970695, -0.0300724459043
+150.0, 0.0816663678457, -0.044641636507, 0.0336730925978, 0.00810087222001, 0.0520932016496, 0.0566185880048, -0.0176293810234, 0.0343088588777, 0.0348641930962, 0.0693381200517
+310.0, 0.0308108295314, 0.0506801187398, 0.0563071461493, 0.0769582860947, 0.0493412959332, -0.0122740735889, -0.0360375700439, 0.0712099797536, 0.120053382002, 0.0900486546259
+153.0, 0.00175052192323, -0.044641636507, -0.0654856181993, -0.00567061055493, -0.00707277125302, -0.01947648821, 0.041276823842, -0.0394933828741, -0.00330371257868, 0.0072065163292
+346.0, -0.049105016391, -0.044641636507, 0.160854917316, -0.0469850588798, -0.0290880169842, -0.0197896366718, -0.0470824834561, 0.0343088588777, 0.0280165065233, 0.011348623244
+63.0, -0.0273097856849, 0.0506801187398, -0.0557853095343, 0.0253152256887, -0.00707277125302, -0.0235474182133, 0.0523217372542, -0.0394933828741, -0.00514530798026, -0.0507829804785
+89.0, 0.0780338293946, 0.0506801187398, -0.0245287593918, -0.0423945646329, 0.006686757329, 0.0528608064634, -0.0691723102806, 0.0808042711814, -0.0371283460105, 0.0569117993072
+50.0, 0.0126481372763, -0.044641636507, -0.0363846922045, 0.0425295791574, -0.013952535544, 0.0129343775852, -0.0268334755336, 0.00515697338576, -0.0439854025656, 0.0072065163292
+39.0, 0.0417084448844, -0.044641636507, -0.00836157828357, -0.057313670961, 0.0080627101872, -0.031376129758, 0.151725957965, -0.07639450375, -0.0802365402489, -0.0176461251598
+103.0, 0.0489735217865, -0.044641636507, -0.0417737525739, 0.104501251645, 0.0355817673512, -0.0257394574458, 0.177497422593, -0.07639450375, -0.0129079422542, 0.0154907301589
+308.0, -0.0164121703319, 0.0506801187398, 0.127442743025, 0.0976155102572, 0.0163184273364, 0.0174750302812, -0.0213110188275, 0.0343088588777, 0.0348641930962, 0.00306440941437
+116.0, -0.0745327855482, 0.0506801187398, -0.0773415510119, -0.0469850588798, -0.0469754041408, -0.0326287236052, 0.00446044580111, -0.0394933828741, -0.072128454602, -0.0176461251598
+145.0, 0.0344433679824, 0.0506801187398, 0.0282840322284, -0.0332135761048, -0.0455994512826, -0.00976888589454, -0.0507641212602, -0.00259226199818, -0.0594726974107, -0.0217882320746
+74.0, -0.034574862587, 0.0506801187398, -0.0256065714657, -0.0171468461892, 0.00118294589619, -0.00287961973517, 0.00814208360519, -0.0155076543048, 0.0148227108413, 0.0403433716479
+45.0, -0.0527375548421, 0.0506801187398, -0.0622521819776, 0.0115437429137, -0.00844872411122, -0.0366996536084, 0.122272855532, -0.07639450375, -0.0868289932163, 0.00306440941437
+115.0, 0.0598711371395, -0.044641636507, -0.000816893766404, -0.0848566365109, 0.0754844002391, 0.0794784257155, 0.00446044580111, 0.0343088588777, 0.0233748412798, 0.0279170509034
+264.0, 0.0635036755906, 0.0506801187398, 0.0886415083657, 0.0700725447073, 0.020446285911, 0.0375165318357, -0.0507641212602, 0.0712099797536, 0.0293004132686, 0.0734802269666
+87.0, 0.00901559882527, -0.044641636507, -0.0320734439089, -0.0263278347174, 0.0424615316422, -0.0103951828181, 0.159089233573, -0.07639450375, -0.0119006848015, -0.038356659734
+202.0, 0.00538306037425, 0.0506801187398, 0.0304396563761, 0.0838440274822, -0.0373437341334, -0.0473467013093, 0.0155053592134, -0.0394933828741, 0.00864028293306, 0.0154907301589
+127.0, 0.0380759064334, 0.0506801187398, 0.00888341489852, 0.0425295791574, -0.0428475455662, -0.021042230519, -0.0397192078479, -0.00259226199818, -0.0181182673079, 0.0072065163292
+182.0, 0.0126481372763, -0.044641636507, 0.00672779075076, -0.0561660474079, -0.0758704141631, -0.0664487574784, -0.0213110188275, -0.0376483268303, -0.0181182673079, -0.0922040496268
+241.0, 0.0744012909436, 0.0506801187398, -0.0202175110963, 0.0459724498511, 0.0741084473809, 0.0328193049088, -0.0360375700439, 0.0712099797536, 0.106354276742, 0.036201264733
+66.0, 0.0162806757273, -0.044641636507, -0.0245287593918, 0.0356438377699, -0.00707277125302, -0.00319276819696, -0.0139477432193, -0.00259226199818, 0.0155668445407, 0.0154907301589
+94.0, -0.00551455497881, 0.0506801187398, -0.0115950145052, 0.0115437429137, -0.0222082526932, -0.0154055582067, -0.0213110188275, -0.00259226199818, 0.0110081010459, 0.0693381200517
+283.0, 0.0126481372763, -0.044641636507, 0.0261284080806, 0.0631868033198, 0.125018703134, 0.0916912157253, 0.0633666506665, -0.00259226199818, 0.0575728562024, -0.0217882320746
+64.0, -0.034574862587, -0.044641636507, -0.059018745756, 0.00121513083254, -0.0538551684319, -0.0780352505647, 0.0670482884706, -0.07639450375, -0.0213936809404, 0.0154907301589
+102.0, 0.0671362140416, 0.0506801187398, -0.0363846922045, -0.0848566365109, -0.00707277125302, 0.0196670695137, -0.0544457590643, 0.0343088588777, 0.00114379737951, 0.0320591578182
+200.0, 0.0380759064334, 0.0506801187398, -0.0245287593918, 0.00465800152627, -0.0263361112678, -0.0263657543694, 0.0155053592134, -0.0394933828741, -0.0159982677581, -0.0259303389895
+265.0, 0.00901559882527, 0.0506801187398, 0.0185837235635, 0.0390867084636, 0.0176943801946, 0.0105857641218, 0.0191869970175, -0.00259226199818, 0.0163049527999, -0.0176461251598
+94.0, -0.0926954778033, 0.0506801187398, -0.0902752958985, -0.057313670961, -0.0249601584096, -0.0304366843726, -0.00658446761116, -0.00259226199818, 0.0240525832269, 0.00306440941437
+230.0, 0.0707687524926, -0.044641636507, -0.00512814206193, -0.00567061055493, 0.0878679759629, 0.10296456035, 0.0118237214093, 0.0343088588777, -0.0089440189578, 0.0279170509034
+181.0, -0.0164121703319, -0.044641636507, -0.0525518733127, -0.0332135761048, -0.0442234984244, -0.0363865051466, 0.0191869970175, -0.0394933828741, -0.0683297436244, -0.0300724459043
+156.0, 0.0417084448844, 0.0506801187398, -0.022373135244, 0.0287580963824, -0.0662387441557, -0.0451546620768, -0.0618090346725, -0.00259226199818, 0.00286377051894, -0.0549250873933
+233.0, 0.0126481372763, -0.044641636507, -0.0202175110963, -0.0159992226361, 0.0121905687618, 0.0212328118226, -0.0765355858888, 0.10811110063, 0.0598807230655, -0.0217882320746
+60.0, -0.038207401038, -0.044641636507, -0.0547074974604, -0.0779708951234, -0.0332158755588, -0.086490259033, 0.140681044552, -0.07639450375, -0.0191970476139, -0.0052198044153
+219.0, 0.0453409833355, -0.044641636507, -0.00620595413581, -0.0159992226361, 0.125018703134, 0.125198101137, 0.0191869970175, 0.0343088588777, 0.0324332257796, -0.0052198044153
+80.0, 0.0707687524926, 0.0506801187398, -0.0169840748746, 0.021872354995, 0.0438374845004, 0.0563054395431, 0.0375951860379, -0.00259226199818, -0.0702093127287, -0.0176461251598
+68.0, -0.0745327855482, 0.0506801187398, 0.0552293340754, -0.0400993174923, 0.0534691545078, 0.0531739549252, -0.043400845652, 0.0712099797536, 0.0612379075197, -0.0342145528191
+332.0, 0.0598711371395, 0.0506801187398, 0.076785575553, 0.0253152256887, 0.00118294589619, 0.0168487333576, -0.0544457590643, 0.0343088588777, 0.0299356483965, 0.0444854785627
+248.0, 0.0744012909436, -0.044641636507, 0.0185837235635, 0.0631868033198, 0.061724871657, 0.0428400556861, 0.00814208360519, -0.00259226199818, 0.0580391276639, -0.0590671943082
+84.0, 0.00901559882527, -0.044641636507, -0.022373135244, -0.0320659525517, -0.0497273098573, -0.068640796711, 0.0780932018828, -0.0708593356186, -0.0629129499163, -0.038356659734
+200.0, -0.0709002470972, -0.044641636507, 0.0929527566612, 0.0126913664668, 0.020446285911, 0.0425269072243, 0.000778807997018, 0.00035982767189, -0.0545441527111, -0.00107769750047
+55.0, 0.0235457526293, 0.0506801187398, -0.0309956318351, -0.00567061055493, -0.0167044412604, 0.0177881787429, -0.0323559322398, -0.00259226199818, -0.0740888714915, -0.0342145528191
+85.0, -0.0527375548421, 0.0506801187398, 0.0390621529672, -0.0400993174923, -0.00569681839481, -0.0129003705124, 0.0118237214093, -0.0394933828741, 0.0163049527999, 0.00306440941437
+89.0, 0.0671362140416, -0.044641636507, -0.0611743699037, -0.0400993174923, -0.0263361112678, -0.0244868635986, 0.0339135482338, -0.0394933828741, -0.056157573095, -0.0590671943082
+31.0, 0.00175052192323, -0.044641636507, -0.00836157828357, -0.0641994123485, -0.0387196869916, -0.0244868635986, 0.00446044580111, -0.0394933828741, -0.0646830224645, -0.0549250873933
+129.0, 0.0235457526293, 0.0506801187398, -0.0374625042784, -0.0469850588798, -0.0910058956033, -0.0755300628703, -0.0323559322398, -0.0394933828741, -0.0307512098646, -0.013504018245
+83.0, 0.0380759064334, 0.0506801187398, -0.013750638653, -0.0159992226361, -0.0359677812752, -0.0219816759043, -0.0139477432193, -0.00259226199818, -0.0259524244352, -0.00107769750047
+275.0, 0.0162806757273, -0.044641636507, 0.0735521393314, -0.0412469410454, -0.00432086553661, -0.013526667436, -0.0139477432193, -0.00111621716315, 0.0428956878925, 0.0444854785627
+65.0, -0.00188201652779, 0.0506801187398, -0.0245287593918, 0.0528581912386, 0.027326050202, 0.0300009687527, 0.0302319104297, -0.00259226199818, -0.0213936809404, 0.036201264733
+198.0, 0.0126481372763, -0.044641636507, 0.0336730925978, 0.033348590526, 0.0300779559184, 0.0271826325966, -0.00290282980707, 0.00884708547335, 0.0311929907028, 0.0279170509034
+236.0, 0.0744012909436, -0.044641636507, 0.0347509046717, 0.0941726395634, 0.0575970130824, 0.0202933664373, 0.0228686348215, -0.00259226199818, 0.07380214692, -0.0217882320746
+253.0, 0.0417084448844, 0.0506801187398, -0.0385403163522, 0.0528581912386, 0.0768603530973, 0.116429944207, -0.0397192078479, 0.0712099797536, -0.0225121719297, -0.013504018245
+124.0, -0.00914709342983, 0.0506801187398, -0.0396181284261, -0.0400993174923, -0.00844872411122, 0.016222436434, -0.0654906724765, 0.0712099797536, 0.0177634778671, -0.0673514081378
+44.0, 0.00901559882527, 0.0506801187398, -0.00189470584028, 0.021872354995, -0.0387196869916, -0.0248000120604, -0.00658446761116, -0.0394933828741, -0.0398095943643, -0.013504018245
+172.0, 0.0671362140416, 0.0506801187398, -0.0309956318351, 0.00465800152627, 0.0245741444856, 0.0356376410649, -0.0286742944357, 0.0343088588777, 0.0233748412798, 0.0817644407962
+114.0, 0.00175052192323, -0.044641636507, -0.0460850008694, -0.0332135761048, -0.0731185084467, -0.0814798836443, 0.0449584616461, -0.0693832907836, -0.0611765950943, -0.0797777288823
+142.0, -0.00914709342983, 0.0506801187398, 0.00133873038136, -0.0022277398612, 0.0796122588137, 0.0700839718618, 0.0339135482338, -0.00259226199818, 0.0267142576335, 0.0817644407962
+109.0, -0.00551455497881, -0.044641636507, 0.0649296427403, 0.0356438377699, -0.00156895982021, 0.0149698425868, -0.0139477432193, 0.000728838880649, -0.0181182673079, 0.0320591578182
+180.0, 0.0961965216497, -0.044641636507, 0.0401399650411, -0.057313670961, 0.0452134373586, 0.0606895180081, -0.0213110188275, 0.0361539149215, 0.0125531528134, 0.0237749439885
+144.0, -0.0745327855482, -0.044641636507, -0.0234509473179, -0.00567061055493, -0.020832299835, -0.0141529643596, 0.0155053592134, -0.0394933828741, -0.0384591123014, -0.0300724459043
+163.0, 0.0598711371395, 0.0506801187398, 0.0530737099276, 0.0528581912386, 0.0328298616348, 0.0196670695137, -0.0102661054152, 0.0343088588777, 0.0552050380896, -0.00107769750047
+147.0, -0.0236772472339, -0.044641636507, 0.0401399650411, -0.0125563519424, -0.00982467696942, -0.00100072896443, -0.00290282980707, -0.00259226199818, -0.0119006848015, -0.038356659734
+97.0, 0.00901559882527, -0.044641636507, -0.0202175110963, -0.0538708002672, 0.0314539087766, 0.020606514899, 0.0560033750583, -0.0394933828741, -0.0109044358474, -0.00107769750047
+220.0, 0.0162806757273, 0.0506801187398, 0.0142724752679, 0.00121513083254, 0.00118294589619, -0.0213553789807, -0.0323559322398, 0.0343088588777, 0.0749683360277, 0.0403433716479
+190.0, 0.0199132141783, -0.044641636507, -0.0342290680567, 0.0551534384825, 0.0672286830898, 0.0741549018651, -0.00658446761116, 0.0328328140427, 0.0247253233428, 0.0693381200517
+109.0, 0.0889314447477, -0.044641636507, 0.00672779075076, 0.0253152256887, 0.0300779559184, 0.00870687335105, 0.0633666506665, -0.0394933828741, 0.00943640914608, 0.0320591578182
+191.0, 0.0199132141783, -0.044641636507, 0.004572166603, 0.0459724498511, -0.0180803941186, -0.0545491159304, 0.0633666506665, -0.0394933828741, 0.0286607203138, 0.0610539062221
+122.0, -0.0236772472339, -0.044641636507, 0.0304396563761, -0.00567061055493, 0.0823641645301, 0.0920043641871, -0.0176293810234, 0.0712099797536, 0.0330470723549, 0.00306440941437
+230.0, 0.0961965216497, -0.044641636507, 0.0519958978538, 0.0792535333387, 0.054845107366, 0.0365770864503, -0.0765355858888, 0.141322109418, 0.0986463743049, 0.0610539062221
+242.0, 0.0235457526293, 0.0506801187398, 0.0616962065187, 0.06203917987, 0.0245741444856, -0.0360733566849, -0.0912621371052, 0.155344535351, 0.133395733837, 0.0817644407962
+248.0, 0.0707687524926, 0.0506801187398, -0.00728376620969, 0.0494153205448, 0.0603489187988, -0.00444536204411, -0.0544457590643, 0.10811110063, 0.1290194116, 0.0569117993072
+249.0, 0.0308108295314, -0.044641636507, 0.00564997867688, 0.0115437429137, 0.0782363059555, 0.0779126834065, -0.043400845652, 0.10811110063, 0.0660482061631, 0.0196328370737
+192.0, -0.00188201652779, -0.044641636507, 0.0541515220015, -0.0664946594891, 0.0727324945226, 0.0566185880048, -0.043400845652, 0.0848633944777, 0.0844952822124, 0.0486275854776
+131.0, 0.0453409833355, 0.0506801187398, -0.00836157828357, -0.0332135761048, -0.00707277125302, 0.0011913102681, -0.0397192078479, 0.0343088588777, 0.0299356483965, 0.0279170509034
+237.0, 0.0744012909436, -0.044641636507, 0.114508998139, 0.0287580963824, 0.0245741444856, 0.0249905933641, 0.0191869970175, -0.00259226199818, -0.000609254186102, -0.0052198044153
+78.0, -0.038207401038, -0.044641636507, 0.0670852668881, -0.0607565416547, -0.0290880169842, -0.0232342697515, -0.0102661054152, -0.00259226199818, -0.00149858682029, 0.0196328370737
+135.0, -0.0127796318808, 0.0506801187398, -0.0557853095343, -0.0022277398612, -0.027712064126, -0.0291840905255, 0.0191869970175, -0.0394933828741, -0.0170521046047, 0.0444854785627
+244.0, 0.00901559882527, 0.0506801187398, 0.0304396563761, 0.0425295791574, -0.00294491267841, 0.0368902349121, -0.0654906724765, 0.0712099797536, -0.0236445575721, 0.0154907301589
+199.0, 0.0816663678457, 0.0506801187398, -0.0256065714657, -0.0366564467986, -0.0703666027303, -0.0464072559239, -0.0397192078479, -0.00259226199818, -0.041180385188, -0.0052198044153
+270.0, 0.0308108295314, -0.044641636507, 0.104808689474, 0.0769582860947, -0.0112006298276, -0.0113346282035, -0.0581273968684, 0.0343088588777, 0.0571041874478, 0.036201264733
+164.0, 0.0271782910804, 0.0506801187398, -0.00620595413581, 0.0287580963824, -0.0167044412604, -0.00162702588801, -0.0581273968684, 0.0343088588777, 0.0293004132686, 0.0320591578182
+72.0, -0.0600026317441, 0.0506801187398, -0.0471628129433, -0.0228849640236, -0.0717425555885, -0.0576806005483, -0.00658446761116, -0.0394933828741, -0.0629129499163, -0.0549250873933
+96.0, 0.00538306037425, -0.044641636507, -0.0482406250172, -0.0125563519424, 0.00118294589619, -0.00663740127664, 0.0633666506665, -0.0394933828741, -0.0514005352606, -0.0590671943082
+306.0, -0.0200447087829, -0.044641636507, 0.0854080721441, -0.0366564467986, 0.0919958345375, 0.0894991764927, -0.0618090346725, 0.145012221505, 0.0809479135113, 0.0527696923924
+91.0, 0.0199132141783, 0.0506801187398, -0.0126728265791, 0.0700725447073, -0.0112006298276, 0.0071411310421, -0.0397192078479, 0.0343088588777, 0.00538436996855, 0.00306440941437
+214.0, -0.0636351701951, -0.044641636507, -0.0331512559828, -0.0332135761048, 0.00118294589619, 0.0240511479787, -0.0249926566316, -0.00259226199818, -0.0225121719297, -0.0590671943082
+95.0, 0.0271782910804, -0.044641636507, -0.00728376620969, -0.0504279295735, 0.0754844002391, 0.0566185880048, 0.0339135482338, -0.00259226199818, 0.0434431722528, 0.0154907301589
+216.0, -0.0164121703319, -0.044641636507, -0.013750638653, 0.132044217195, -0.00982467696942, -0.00381906512053, 0.0191869970175, -0.0394933828741, -0.0358167281015, -0.0300724459043
+263.0, 0.0308108295314, 0.0506801187398, 0.0595405823709, 0.0563010619323, -0.0222082526932, 0.0011913102681, -0.0323559322398, -0.00259226199818, -0.0247911874325, -0.0176461251598
+178.0, 0.0562385986885, 0.0506801187398, 0.0218171597851, 0.0563010619323, -0.00707277125302, 0.0181013272047, -0.0323559322398, -0.00259226199818, -0.0236445575721, 0.0237749439885
+113.0, -0.0200447087829, -0.044641636507, 0.0185837235635, 0.0907297688697, 0.00393485161259, 0.00870687335105, 0.0375951860379, -0.0394933828741, -0.0578000656756, 0.0072065163292
+200.0, -0.107225631607, -0.044641636507, -0.0115950145052, -0.0400993174923, 0.0493412959332, 0.0644472995496, -0.0139477432193, 0.0343088588777, 0.00702686254915, -0.0300724459043
+139.0, 0.0816663678457, 0.0506801187398, -0.00297251791417, -0.0332135761048, 0.0424615316422, 0.057871181852, -0.0102661054152, 0.0343088588777, -0.000609254186102, -0.00107769750047
+139.0, 0.00538306037425, 0.0506801187398, 0.0175059114896, 0.0322009670762, 0.127770608851, 0.127390140369, -0.0213110188275, 0.0712099797536, 0.0625751814581, 0.0154907301589
+88.0, 0.0380759064334, 0.0506801187398, -0.0299178197612, -0.0745280244297, -0.0125765826858, -0.0125872220506, 0.00446044580111, -0.00259226199818, 0.00371173823344, -0.0300724459043
+148.0, 0.0308108295314, -0.044641636507, -0.0202175110963, -0.00567061055493, -0.00432086553661, -0.0294972389873, 0.0780932018828, -0.0394933828741, -0.0109044358474, -0.00107769750047
+88.0, 0.00175052192323, 0.0506801187398, -0.0579409336821, -0.043542188186, -0.0965097070361, -0.0470335528475, -0.0986254127133, 0.0343088588777, -0.0611765950943, -0.0714935150527
+243.0, -0.0273097856849, 0.0506801187398, 0.0606183944448, 0.107944122338, 0.0121905687618, -0.0175975974393, -0.00290282980707, -0.00259226199818, 0.0702112981933, 0.135611830689
+71.0, -0.0854304009012, 0.0506801187398, -0.0406959405, -0.0332135761048, -0.0813742255959, -0.0695802420963, -0.00658446761116, -0.0394933828741, -0.0578000656756, -0.0424987666488
+77.0, 0.0126481372763, 0.0506801187398, -0.0719524906425, -0.0469850588798, -0.0511032627155, -0.0971373067338, 0.118591217728, -0.07639450375, -0.0202887477516, -0.038356659734
+109.0, -0.0527375548421, -0.044641636507, -0.0557853095343, -0.0366564467986, 0.0892439288211, -0.00319276819696, 0.00814208360519, 0.0343088588777, 0.132372649339, 0.00306440941437
+272.0, -0.0236772472339, 0.0506801187398, 0.0455290254105, 0.021872354995, 0.109883221694, 0.0888728795692, 0.000778807997018, 0.0343088588777, 0.07419253669, 0.0610539062221
+60.0, -0.0745327855482, 0.0506801187398, -0.00943939035745, 0.0149866136075, -0.0373437341334, -0.0216685274425, -0.0139477432193, -0.00259226199818, -0.0332487872476, 0.011348623244
+54.0, -0.00551455497881, 0.0506801187398, -0.0331512559828, -0.0159992226361, 0.0080627101872, 0.016222436434, 0.0155053592134, -0.00259226199818, -0.028320242548, -0.0756356219675
+221.0, -0.0600026317441, 0.0506801187398, 0.049840273706, 0.0184294843012, -0.0167044412604, -0.0301235359109, -0.0176293810234, -0.00259226199818, 0.0497686599207, -0.0590671943082
+90.0, -0.0200447087829, -0.044641636507, -0.0848862355291, -0.0263278347174, -0.0359677812752, -0.0341944659141, 0.041276823842, -0.0516707527631, -0.0823814832581, -0.0466408735636
+311.0, 0.0380759064334, 0.0506801187398, 0.00564997867688, 0.0322009670762, 0.006686757329, 0.0174750302812, -0.0249926566316, 0.0343088588777, 0.0148227108413, 0.0610539062221
+281.0, 0.0162806757273, -0.044641636507, 0.0207393477112, 0.021872354995, -0.013952535544, -0.0132135189742, -0.00658446761116, -0.00259226199818, 0.0133159679089, 0.0403433716479
+182.0, 0.0417084448844, -0.044641636507, -0.00728376620969, 0.0287580963824, -0.0428475455662, -0.0482861466946, 0.0523217372542, -0.07639450375, -0.072128454602, 0.0237749439885
+321.0, 0.0199132141783, 0.0506801187398, 0.104808689474, 0.0700725447073, -0.0359677812752, -0.0266789028312, -0.0249926566316, -0.00259226199818, 0.00371173823344, 0.0403433716479
+58.0, -0.049105016391, 0.0506801187398, -0.0245287593918, 6.75072794357e-05, -0.0469754041408, -0.0282446451401, -0.0654906724765, 0.0284046795376, 0.0191990330786, 0.011348623244
+262.0, 0.00175052192323, 0.0506801187398, -0.00620595413581, -0.0194420933299, -0.00982467696942, 0.00494909180957, -0.0397192078479, 0.0343088588777, 0.0148227108413, 0.0983328684556
+206.0, 0.0344433679824, -0.044641636507, -0.0385403163522, -0.0125563519424, 0.0094386630454, 0.00526224027136, -0.00658446761116, -0.00259226199818, 0.0311929907028, 0.0983328684556
+233.0, -0.04547247794, 0.0506801187398, 0.13714305169, -0.0159992226361, 0.041085578784, 0.0318798595235, -0.043400845652, 0.0712099797536, 0.071021577946, 0.0486275854776
+242.0, -0.00914709342983, 0.0506801187398, 0.170555225981, 0.0149866136075, 0.0300779559184, 0.0337587502942, -0.0213110188275, 0.0343088588777, 0.0336568129024, 0.0320591578182
+123.0, -0.0164121703319, 0.0506801187398, 0.00241654245524, 0.0149866136075, 0.0218222387692, -0.0100820343563, -0.0249926566316, 0.0343088588777, 0.0855331211874, 0.0817644407962
+167.0, -0.00914709342983, -0.044641636507, 0.0379843408933, -0.0400993174923, -0.0249601584096, -0.00381906512053, -0.043400845652, 0.0158582984398, -0.00514530798026, 0.0279170509034
+63.0, 0.0199132141783, -0.044641636507, -0.0579409336821, -0.057313670961, -0.00156895982021, -0.0125872220506, 0.0744115640788, -0.0394933828741, -0.0611765950943, -0.0756356219675
+197.0, 0.0526060602375, 0.0506801187398, -0.00943939035745, 0.0494153205448, 0.0507172487914, -0.0191633397482, -0.0139477432193, 0.0343088588777, 0.119343994204, -0.0176461251598
+71.0, -0.0273097856849, 0.0506801187398, -0.0234509473179, -0.0159992226361, 0.01356652162, 0.0127778033543, 0.0265502726256, -0.00259226199818, -0.0109044358474, -0.0217882320746
+168.0, -0.0745327855482, -0.044641636507, -0.0105172024313, -0.00567061055493, -0.0662387441557, -0.0570543036248, -0.00290282980707, -0.0394933828741, -0.0425721049228, -0.00107769750047
+140.0, -0.107225631607, -0.044641636507, -0.0342290680567, -0.0676422830422, -0.0634868384393, -0.0705196874817, 0.00814208360519, -0.0394933828741, -0.000609254186102, -0.0797777288823
+217.0, 0.0453409833355, 0.0506801187398, -0.00297251791417, 0.107944122338, 0.0355817673512, 0.0224854056698, 0.0265502726256, -0.00259226199818, 0.0280165065233, 0.0196328370737
+121.0, -0.00188201652779, -0.044641636507, 0.068163078962, -0.00567061055493, 0.119514891701, 0.130208476525, -0.0249926566316, 0.0867084505215, 0.0461323310394, -0.00107769750047
+235.0, 0.0199132141783, 0.0506801187398, 0.00996122697241, 0.0184294843012, 0.0149424744782, 0.0447189464568, -0.0618090346725, 0.0712099797536, 0.00943640914608, -0.063209301223
+245.0, 0.0162806757273, 0.0506801187398, 0.00241654245524, -0.00567061055493, -0.00569681839481, 0.0108989125836, -0.0507641212602, 0.0343088588777, 0.0226920225667, -0.038356659734
+40.0, -0.00188201652779, -0.044641636507, -0.0385403163522, 0.021872354995, -0.10889328276, -0.115613065979, 0.0228686348215, -0.07639450375, -0.0468794828442, 0.0237749439885
+52.0, 0.0162806757273, -0.044641636507, 0.0261284080806, 0.0585963091762, -0.0607349327229, -0.0442152166914, -0.0139477432193, -0.0339582147427, -0.0514005352606, -0.0259303389895
+104.0, -0.0709002470972, 0.0506801187398, -0.0891974838246, -0.0745280244297, -0.0428475455662, -0.0257394574458, -0.0323559322398, -0.00259226199818, -0.0129079422542, -0.0549250873933
+132.0, 0.0489735217865, -0.044641636507, 0.0606183944448, -0.0228849640236, -0.0235842055514, -0.0727117267142, -0.043400845652, -0.00259226199818, 0.104137611359, 0.036201264733
+88.0, 0.00538306037425, 0.0506801187398, -0.0288400076873, -0.00911348124867, -0.0318399227006, -0.0288709420637, 0.00814208360519, -0.0394933828741, -0.0181182673079, 0.0072065163292
+69.0, 0.0344433679824, 0.0506801187398, -0.0299178197612, 0.00465800152627, 0.0933717873957, 0.0869939887984, 0.0339135482338, -0.00259226199818, 0.0240525832269, -0.038356659734
+219.0, 0.0235457526293, 0.0506801187398, -0.0191396990224, 0.0494153205448, -0.0634868384393, -0.061125233628, 0.00446044580111, -0.0394933828741, -0.0259524244352, -0.013504018245
+72.0, 0.0199132141783, -0.044641636507, -0.0406959405, -0.0159992226361, -0.00844872411122, -0.0175975974393, 0.0523217372542, -0.0394933828741, -0.0307512098646, 0.00306440941437
+201.0, -0.04547247794, -0.044641636507, 0.0153502873418, -0.0745280244297, -0.0497273098573, -0.0172844489775, -0.0286742944357, -0.00259226199818, -0.104364820832, -0.0756356219675
+110.0, 0.0526060602375, 0.0506801187398, -0.0245287593918, 0.0563010619323, -0.00707277125302, -0.00507165896769, -0.0213110188275, -0.00259226199818, 0.0267142576335, -0.038356659734
+51.0, -0.00551455497881, 0.0506801187398, 0.00133873038136, -0.0848566365109, -0.0112006298276, -0.0166581520539, 0.0486400994501, -0.0394933828741, -0.041180385188, -0.088061942712
+277.0, 0.00901559882527, 0.0506801187398, 0.0692408910359, 0.0597439326261, 0.0176943801946, -0.0232342697515, -0.0470824834561, 0.0343088588777, 0.103292264912, 0.0734802269666
+63.0, -0.0236772472339, -0.044641636507, -0.0697968664948, -0.0641994123485, -0.0593589798647, -0.0504781859272, 0.0191869970175, -0.0394933828741, -0.0891368600793, -0.0507829804785
+118.0, -0.041839939489, 0.0506801187398, -0.0299178197612, -0.0022277398612, 0.0218222387692, 0.0365770864503, 0.0118237214093, -0.00259226199818, -0.041180385188, 0.0651960131369
+69.0, -0.0745327855482, -0.044641636507, -0.0460850008694, -0.043542188186, -0.0290880169842, -0.0232342697515, 0.0155053592134, -0.0394933828741, -0.0398095943643, -0.0217882320746
+273.0, 0.0344433679824, -0.044641636507, 0.0185837235635, 0.0563010619323, 0.0121905687618, -0.0545491159304, -0.0691723102806, 0.0712099797536, 0.130080609522, 0.0072065163292
+258.0, -0.0600026317441, -0.044641636507, 0.00133873038136, -0.0297707054111, -0.00707277125302, -0.0216685274425, 0.0118237214093, -0.00259226199818, 0.0318152175008, -0.0549250873933
+43.0, -0.0854304009012, 0.0506801187398, -0.0309956318351, -0.0228849640236, -0.0634868384393, -0.0542359674686, 0.0191869970175, -0.0394933828741, -0.0964332228918, -0.0342145528191
+198.0, 0.0526060602375, -0.044641636507, -0.00405032998805, -0.0309183289642, -0.0469754041408, -0.0583068974719, -0.0139477432193, -0.02583996815, 0.0360557900898, 0.0237749439885
+242.0, 0.0126481372763, -0.044641636507, 0.0153502873418, -0.0332135761048, 0.041085578784, 0.0321930079853, -0.00290282980707, -0.00259226199818, 0.0450661683363, -0.0673514081378
+232.0, 0.0598711371395, 0.0506801187398, 0.022894971859, 0.0494153205448, 0.0163184273364, 0.0118383579689, -0.0139477432193, -0.00259226199818, 0.039539878072, 0.0196328370737
+175.0, -0.0236772472339, -0.044641636507, 0.0455290254105, 0.0907297688697, -0.0180803941186, -0.0354470597613, 0.0707299262747, -0.0394933828741, -0.0345237153303, -0.00936191133014
+93.0, 0.0162806757273, -0.044641636507, -0.0450071887955, -0.057313670961, -0.034591828417, -0.0539228190069, 0.0744115640788, -0.07639450375, -0.0425721049228, 0.0403433716479
+168.0, 0.110726675454, 0.0506801187398, -0.0331512559828, -0.0228849640236, -0.00432086553661, 0.0202933664373, -0.0618090346725, 0.0712099797536, 0.0155668445407, 0.0444854785627
+275.0, -0.0200447087829, -0.044641636507, 0.0972640049568, -0.00567061055493, -0.00569681839481, -0.0238605666751, -0.0213110188275, -0.00259226199818, 0.0616858488239, 0.0403433716479
+293.0, -0.0164121703319, -0.044641636507, 0.0541515220015, 0.0700725447073, -0.0332158755588, -0.0279314966783, 0.00814208360519, -0.0394933828741, -0.0271286455543, -0.00936191133014
+281.0, 0.0489735217865, 0.0506801187398, 0.12313149473, 0.0838440274822, -0.104765424185, -0.100895088275, -0.0691723102806, -0.00259226199818, 0.0366457977934, -0.0300724459043
+72.0, -0.0563700932931, -0.044641636507, -0.0805749872336, -0.0848566365109, -0.0373437341334, -0.0370128020702, 0.0339135482338, -0.0394933828741, -0.056157573095, -0.13776722569
+140.0, 0.0271782910804, -0.044641636507, 0.0929527566612, -0.0527231767141, 0.0080627101872, 0.0397085710682, -0.0286742944357, 0.0210244553624, -0.0483617248029, 0.0196328370737
+189.0, 0.0635036755906, -0.044641636507, -0.0503962491649, 0.107944122338, 0.0314539087766, 0.0193539210519, -0.0176293810234, 0.0236075338237, 0.0580391276639, 0.0403433716479
+181.0, -0.0527375548421, 0.0506801187398, -0.0115950145052, 0.0563010619323, 0.0562210602242, 0.0729023080179, -0.0397192078479, 0.0712099797536, 0.0305664873984, -0.0052198044153
+209.0, -0.00914709342983, 0.0506801187398, -0.0277621956134, 0.00810087222001, 0.047965343075, 0.0372033833739, -0.0286742944357, 0.0343088588777, 0.0660482061631, -0.0424987666488
+136.0, 0.00538306037425, -0.044641636507, 0.058462770297, -0.043542188186, -0.0731185084467, -0.0723985782524, 0.0191869970175, -0.07639450375, -0.0514005352606, -0.0259303389895
+261.0, 0.0744012909436, -0.044641636507, 0.0854080721441, 0.0631868033198, 0.0149424744782, 0.0130909518161, 0.0155053592134, -0.00259226199818, 0.00620931561651, 0.0859065477111
+113.0, -0.0527375548421, -0.044641636507, -0.000816893766404, -0.0263278347174, 0.0108146159036, 0.0071411310421, 0.0486400994501, -0.0394933828741, -0.0358167281015, 0.0196328370737
+131.0, 0.0816663678457, 0.0506801187398, 0.00672779075076, -0.00452298700183, 0.109883221694, 0.11705624113, -0.0323559322398, 0.0918746074441, 0.0547240033482, 0.0072065163292
+174.0, -0.00551455497881, -0.044641636507, 0.00888341489852, -0.0504279295735, 0.0259500973438, 0.0472241341512, -0.043400845652, 0.0712099797536, 0.0148227108413, 0.00306440941437
+257.0, -0.0273097856849, -0.044641636507, 0.0800190117747, 0.098763133707, -0.00294491267841, 0.0181013272047, -0.0176293810234, 0.00331191734196, -0.0295276227418, 0.036201264733
+55.0, -0.0527375548421, -0.044641636507, 0.0713965151836, -0.0745280244297, -0.0153284884022, -0.00131387742622, 0.00446044580111, -0.0214118336449, -0.0468794828442, 0.00306440941437
+84.0, 0.00901559882527, -0.044641636507, -0.0245287593918, -0.0263278347174, 0.0988755988285, 0.0941964034196, 0.0707299262747, -0.00259226199818, -0.0213936809404, 0.0072065163292
+42.0, -0.0200447087829, -0.044641636507, -0.0547074974604, -0.0538708002672, -0.0662387441557, -0.0573674520865, 0.0118237214093, -0.0394933828741, -0.0740888714915, -0.0052198044153
+146.0, 0.0235457526293, -0.044641636507, -0.0363846922045, 6.75072794357e-05, 0.00118294589619, 0.0346981956796, -0.043400845652, 0.0343088588777, -0.0332487872476, 0.0610539062221
+212.0, 0.0380759064334, 0.0506801187398, 0.0164280994157, 0.021872354995, 0.0397096259258, 0.0450320949186, -0.043400845652, 0.0712099797536, 0.0497686599207, 0.0154907301589
+233.0, -0.0781653239992, 0.0506801187398, 0.0778633876269, 0.0528581912386, 0.0782363059555, 0.0644472995496, 0.0265502726256, -0.00259226199818, 0.0406722637145, -0.00936191133014
+91.0, 0.00901559882527, 0.0506801187398, -0.0396181284261, 0.0287580963824, 0.0383336730676, 0.0735286049415, -0.0728539480847, 0.10811110063, 0.0155668445407, -0.0466408735636
+111.0, 0.00175052192323, 0.0506801187398, 0.0110390390463, -0.0194420933299, -0.0167044412604, -0.00381906512053, -0.0470824834561, 0.0343088588777, 0.0240525832269, 0.0237749439885
+152.0, -0.0781653239992, -0.044641636507, -0.0406959405, -0.0814137658171, -0.100637565611, -0.112794729823, 0.0228686348215, -0.07639450375, -0.0202887477516, -0.0507829804785
+120.0, 0.0308108295314, 0.0506801187398, -0.0342290680567, 0.0436772026072, 0.0575970130824, 0.0688313780146, -0.0323559322398, 0.0575565650295, 0.0354619386608, 0.0859065477111
+67.0, -0.034574862587, 0.0506801187398, 0.00564997867688, -0.00567061055493, -0.0731185084467, -0.062690975937, -0.00658446761116, -0.0394933828741, -0.045420957777, 0.0320591578182
+310.0, 0.0489735217865, 0.0506801187398, 0.0886415083657, 0.0872868981759, 0.0355817673512, 0.0215459602844, -0.0249926566316, 0.0343088588777, 0.0660482061631, 0.131469723774
+94.0, -0.041839939489, -0.044641636507, -0.0331512559828, -0.0228849640236, 0.0465893902168, 0.0415874618389, 0.0560033750583, -0.0247329345237, -0.0259524244352, -0.038356659734
+183.0, -0.00914709342983, -0.044641636507, -0.0568631216082, -0.0504279295735, 0.0218222387692, 0.0453452433804, -0.0286742944357, 0.0343088588777, -0.00991895736315, -0.0176461251598
+66.0, 0.0707687524926, 0.0506801187398, -0.0309956318351, 0.021872354995, -0.0373437341334, -0.0470335528475, 0.0339135482338, -0.0394933828741, -0.0149564750249, -0.00107769750047
+173.0, 0.00901559882527, -0.044641636507, 0.0552293340754, -0.00567061055493, 0.0575970130824, 0.0447189464568, -0.00290282980707, 0.023238522615, 0.0556835477027, 0.106617082285
+72.0, -0.0273097856849, -0.044641636507, -0.0600965578299, -0.0297707054111, 0.0465893902168, 0.0199802179755, 0.122272855532, -0.0394933828741, -0.0514005352606, -0.00936191133014
+49.0, 0.0162806757273, -0.044641636507, 0.00133873038136, 0.00810087222001, 0.00531080447079, 0.0108989125836, 0.0302319104297, -0.0394933828741, -0.045420957777, 0.0320591578182
+64.0, -0.0127796318808, -0.044641636507, -0.0234509473179, -0.0400993174923, -0.0167044412604, 0.00463594334778, -0.0176293810234, -0.00259226199818, -0.0384591123014, -0.038356659734
+48.0, -0.0563700932931, -0.044641636507, -0.0741081147903, -0.0504279295735, -0.0249601584096, -0.0470335528475, 0.0928197530992, -0.07639450375, -0.0611765950943, -0.0466408735636
+178.0, 0.0417084448844, 0.0506801187398, 0.0196615356373, 0.0597439326261, -0.00569681839481, -0.00256647127338, -0.0286742944357, -0.00259226199818, 0.0311929907028, 0.0072065163292
+104.0, -0.00551455497881, 0.0506801187398, -0.0159062628007, -0.0676422830422, 0.0493412959332, 0.0791652772537, -0.0286742944357, 0.0343088588777, -0.0181182673079, 0.0444854785627
+132.0, 0.0417084448844, 0.0506801187398, -0.0159062628007, 0.0172818607481, -0.0373437341334, -0.0138398158978, -0.0249926566316, -0.0110795197996, -0.0468794828442, 0.0154907301589
+220.0, -0.04547247794, -0.044641636507, 0.0390621529672, 0.00121513083254, 0.0163184273364, 0.0152829910486, -0.0286742944357, 0.0265596234938, 0.0445283740214, -0.0259303389895
+57.0, -0.04547247794, -0.044641636507, -0.0730303027164, -0.0814137658171, 0.0837401173883, 0.0278089295202, 0.173815784789, -0.0394933828741, -0.00421985970695, 0.00306440941437
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/jdbc/JdbcPutIndexedValue8Benchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/jdbc/JdbcPutIndexedValue8Benchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/jdbc/JdbcPutIndexedValue8Benchmark.java
index f8bef94..3a41c08 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/jdbc/JdbcPutIndexedValue8Benchmark.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/jdbc/JdbcPutIndexedValue8Benchmark.java
@@ -20,11 +20,8 @@ package org.apache.ignite.yardstick.cache.jdbc;
 import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.util.Map;
-
 import org.apache.ignite.IgniteException;
 
-import static org.apache.ignite.yardstick.IgniteAbstractBenchmark.nextRandom;
-
 /**
  * JDBC benchmark that performs raw SQL insert into a table with 8 btree indexed columns
  */

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/yardstick/src/main/ml/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/ml/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java b/modules/yardstick/src/main/ml/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
index 9b36a4a..89b5471 100644
--- a/modules/yardstick/src/main/ml/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
+++ b/modules/yardstick/src/main/ml/org/apache/ignite/yardstick/ml/regression/IgniteOLSMultipleLinearRegressionBenchmark.java
@@ -19,9 +19,9 @@ 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.OLSMultipleLinearRegression;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionQRTrainer;
 import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
-import org.apache.ignite.yardstick.ml.DataChanger;
 
 /**
  * Ignite benchmark that performs ML Grid operations.
@@ -41,52 +41,29 @@ public class IgniteOLSMultipleLinearRegressionBenchmark extends IgniteAbstractBe
     private void runLongly() {
         // Y values are first, then independent vars
         // Each row is one observation
-        double[] design = new DataChanger.Scale().mutate(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
-        });
+        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;
 
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from NIST
-        mdl.estimateRegressionParameters();
-
-        // Check expected residuals from R
-        mdl.estimateResiduals();
-
-        // Check standard errors from NIST
-        mdl.estimateRegressionParametersStandardErrors();
-
-        // Estimate model without intercept
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from R
-        mdl.estimateRegressionParameters();
-
-        // Check standard errors from R
-        mdl.estimateRegressionParametersStandardErrors();
-
-        // Check expected residuals from R
-        mdl.estimateResiduals();
+        LinearRegressionQRTrainer trainer = new LinearRegressionQRTrainer();
+        LinearRegressionModel model = trainer.train(new DenseLocalOnHeapMatrix(data));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/parent/pom.xml
----------------------------------------------------------------------
diff --git a/parent/pom.xml b/parent/pom.xml
index 9b2386f..ec5a938 100644
--- a/parent/pom.xml
+++ b/parent/pom.xml
@@ -839,6 +839,7 @@
                                         <exclude>examples/src/main/java/org/apache/ignite/examples/streaming/wordcount/*.txt</exclude><!--books examples-->
                                         <exclude>src/main/resources/person.csv</exclude><!--CacheLoadOnlyStoreExample csv-->
                                         <exclude>**/resources/datasets/knn/*</exclude><!--Dataset examples in ml module-->
+                                        <exclude>**/resources/datasets/regression/*</exclude><!--Dataset examples in ml module-->
                                         <exclude>examples/src/main/resources/person.csv</exclude><!--CacheLoadOnlyStoreExample csv-->
                                         <exclude>src/main/java/org/jetbrains/annotations/*.java</exclude><!--copyright-->
                                         <exclude>dev-tools/IGNITE-*.patch</exclude>


[09/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..1a60b80
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GridAwareAbstractLinearRegressionTrainerTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+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/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModelTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModelTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModelTest.java
new file mode 100644
index 0000000..aac24f4
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModelTest.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.TestUtils;
+import org.apache.ignite.ml.math.Vector;
+import org.apache.ignite.ml.math.exceptions.CardinalityException;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.junit.Test;
+
+/**
+ * Tests for {@link LinearRegressionModel}.
+ */
+public class LinearRegressionModelTest {
+    /** */
+    private static final double PRECISION = 1e-6;
+
+    /** */
+    @Test
+    public void testPredict() {
+        Vector weights = new DenseLocalOnHeapVector(new double[]{2.0, 3.0});
+        LinearRegressionModel mdl = new LinearRegressionModel(weights, 1.0);
+
+        Vector observation = new DenseLocalOnHeapVector(new double[]{1.0, 1.0});
+        TestUtils.assertEquals(1.0 + 2.0 * 1.0 + 3.0 * 1.0, mdl.apply(observation), PRECISION);
+
+        observation = new DenseLocalOnHeapVector(new double[]{2.0, 1.0});
+        TestUtils.assertEquals(1.0 + 2.0 * 2.0 + 3.0 * 1.0, mdl.apply(observation), PRECISION);
+
+        observation = new DenseLocalOnHeapVector(new double[]{1.0, 2.0});
+        TestUtils.assertEquals(1.0 + 2.0 * 1.0 + 3.0 * 2.0, mdl.apply(observation), PRECISION);
+
+        observation = new DenseLocalOnHeapVector(new double[]{-2.0, 1.0});
+        TestUtils.assertEquals(1.0 - 2.0 * 2.0 + 3.0 * 1.0, mdl.apply(observation), PRECISION);
+
+        observation = new DenseLocalOnHeapVector(new double[]{1.0, -2.0});
+        TestUtils.assertEquals(1.0 + 2.0 * 1.0 - 3.0 * 2.0, mdl.apply(observation), PRECISION);
+    }
+
+    /** */
+    @Test(expected = CardinalityException.class)
+    public void testPredictOnAnObservationWithWrongCardinality() {
+        Vector weights = new DenseLocalOnHeapVector(new double[]{2.0, 3.0});
+
+        LinearRegressionModel mdl = new LinearRegressionModel(weights, 1.0);
+
+        Vector observation = new DenseLocalOnHeapVector(new double[]{1.0});
+
+        mdl.apply(observation);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..f37d71d
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionQRTrainerTest.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..bea164d
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/LocalLinearRegressionSGDTrainerTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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/b2060855/modules/ml/src/test/resources/datasets/regression/README.md
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/resources/datasets/regression/README.md b/modules/ml/src/test/resources/datasets/regression/README.md
new file mode 100644
index 0000000..b4d57cf
--- /dev/null
+++ b/modules/ml/src/test/resources/datasets/regression/README.md
@@ -0,0 +1,98 @@
+This package contains two datasets:
+
+Boston House Prices dataset
+===========================
+
+Notes
+------
+Data Set Characteristics:
+
+    :Number of Instances: 506
+
+    :Number of Attributes: 13 numeric/categorical predictive
+
+    :Median Value (attribute 14) is usually the target
+
+    :Attribute Information (in order):
+        - CRIM     per capita crime rate by town
+        - ZN       proportion of residential land zoned for lots over 25,000 sq.ft.
+        - INDUS    proportion of non-retail business acres per town
+        - CHAS     Charles River dummy variable (= 1 if tract bounds river; 0 otherwise)
+        - NOX      nitric oxides concentration (parts per 10 million)
+        - RM       average number of rooms per dwelling
+        - AGE      proportion of owner-occupied units built prior to 1940
+        - DIS      weighted distances to five Boston employment centres
+        - RAD      index of accessibility to radial highways
+        - TAX      full-value property-tax rate per $10,000
+        - PTRATIO  pupil-teacher ratio by town
+        - B        1000(Bk - 0.63)^2 where Bk is the proportion of blacks by town
+        - LSTAT    % lower status of the population
+        - MEDV     Median value of owner-occupied homes in $1000's
+
+    :Missing Attribute Values: None
+
+    :Creator: Harrison, D. and Rubinfeld, D.L.
+
+This is a copy of UCI ML housing dataset.
+http://archive.ics.uci.edu/ml/datasets/Housing
+
+
+This dataset was taken from the StatLib library which is maintained at Carnegie Mellon University.
+
+The Boston house-price data of Harrison, D. and Rubinfeld, D.L. 'Hedonic
+prices and the demand for clean air', J. Environ. Economics & Management,
+vol.5, 81-102, 1978.   Used in Belsley, Kuh & Welsch, 'Regression diagnostics
+...', Wiley, 1980.   N.B. Various transformations are used in the table on
+pages 244-261 of the latter.
+
+The Boston house-price data has been used in many machine learning papers that address regression
+problems.
+
+**References**
+
+   - Belsley, Kuh & Welsch, 'Regression diagnostics: Identifying Influential Data and Sources of Collinearity', Wiley, 1980. 244-261.
+   - Quinlan,R. (1993). Combining Instance-Based and Model-Based Learning. In Proceedings on the Tenth International Conference of Machine Learning, 236-243, University of Massachusetts, Amherst. Morgan Kaufmann.
+   - many more! (see http://archive.ics.uci.edu/ml/datasets/Housing)
+
+
+Diabetes dataset
+================
+
+Notes
+-----
+
+Ten baseline variables, age, sex, body mass index, average blood
+pressure, and six blood serum measurements were obtained for each of n =
+442 diabetes patients, as well as the response of interest, a
+quantitative measure of disease progression one year after baseline.
+
+Data Set Characteristics:
+
+  :Number of Instances: 442
+
+  :Number of Attributes: First 10 columns are numeric predictive values
+
+  :Target: Column 11 is a quantitative measure of disease progression one year after baseline
+
+  :Attributes:
+    :Age:
+    :Sex:
+    :Body mass index:
+    :Average blood pressure:
+    :S1:
+    :S2:
+    :S3:
+    :S4:
+    :S5:
+    :S6:
+
+Note: Each of these 10 feature variables have been mean centered and scaled by the standard deviation times
+n_samples
+ (i.e. the sum of squares of each column totals 1).
+
+Source URL:
+http://www4.stat.ncsu.edu/~boos/var.select/diabetes.html
+
+For more information see:
+Bradley Efron, Trevor Hastie, Iain Johnstone and Robert Tibshirani (2004) "Least Angle Regression," Annals of Statistics (with discussion), 407-499.
+(http://web.stanford.edu/~hastie/Papers/LARS/LeastAngle_2002.pdf)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/resources/datasets/regression/boston.csv
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/resources/datasets/regression/boston.csv b/modules/ml/src/test/resources/datasets/regression/boston.csv
new file mode 100644
index 0000000..b43afa9
--- /dev/null
+++ b/modules/ml/src/test/resources/datasets/regression/boston.csv
@@ -0,0 +1,506 @@
+24.0, 0.00632, 18.0, 2.31, 0.0, 0.538, 6.575, 65.2, 4.09, 1.0, 296.0, 15.3, 396.9, 4.98
+21.6, 0.02731, 0.0, 7.07, 0.0, 0.469, 6.421, 78.9, 4.9671, 2.0, 242.0, 17.8, 396.9, 9.14
+34.7, 0.02729, 0.0, 7.07, 0.0, 0.469, 7.185, 61.1, 4.9671, 2.0, 242.0, 17.8, 392.83, 4.03
+33.4, 0.03237, 0.0, 2.18, 0.0, 0.458, 6.998, 45.8, 6.0622, 3.0, 222.0, 18.7, 394.63, 2.94
+36.2, 0.06905, 0.0, 2.18, 0.0, 0.458, 7.147, 54.2, 6.0622, 3.0, 222.0, 18.7, 396.9, 5.33
+28.7, 0.02985, 0.0, 2.18, 0.0, 0.458, 6.43, 58.7, 6.0622, 3.0, 222.0, 18.7, 394.12, 5.21
+22.9, 0.08829, 12.5, 7.87, 0.0, 0.524, 6.012, 66.6, 5.5605, 5.0, 311.0, 15.2, 395.6, 12.43
+27.1, 0.14455, 12.5, 7.87, 0.0, 0.524, 6.172, 96.1, 5.9505, 5.0, 311.0, 15.2, 396.9, 19.15
+16.5, 0.21124, 12.5, 7.87, 0.0, 0.524, 5.631, 100.0, 6.0821, 5.0, 311.0, 15.2, 386.63, 29.93
+18.9, 0.17004, 12.5, 7.87, 0.0, 0.524, 6.004, 85.9, 6.5921, 5.0, 311.0, 15.2, 386.71, 17.1
+15.0, 0.22489, 12.5, 7.87, 0.0, 0.524, 6.377, 94.3, 6.3467, 5.0, 311.0, 15.2, 392.52, 20.45
+18.9, 0.11747, 12.5, 7.87, 0.0, 0.524, 6.009, 82.9, 6.2267, 5.0, 311.0, 15.2, 396.9, 13.27
+21.7, 0.09378, 12.5, 7.87, 0.0, 0.524, 5.889, 39.0, 5.4509, 5.0, 311.0, 15.2, 390.5, 15.71
+20.4, 0.62976, 0.0, 8.14, 0.0, 0.538, 5.949, 61.8, 4.7075, 4.0, 307.0, 21.0, 396.9, 8.26
+18.2, 0.63796, 0.0, 8.14, 0.0, 0.538, 6.096, 84.5, 4.4619, 4.0, 307.0, 21.0, 380.02, 10.26
+19.9, 0.62739, 0.0, 8.14, 0.0, 0.538, 5.834, 56.5, 4.4986, 4.0, 307.0, 21.0, 395.62, 8.47
+23.1, 1.05393, 0.0, 8.14, 0.0, 0.538, 5.935, 29.3, 4.4986, 4.0, 307.0, 21.0, 386.85, 6.58
+17.5, 0.7842, 0.0, 8.14, 0.0, 0.538, 5.99, 81.7, 4.2579, 4.0, 307.0, 21.0, 386.75, 14.67
+20.2, 0.80271, 0.0, 8.14, 0.0, 0.538, 5.456, 36.6, 3.7965, 4.0, 307.0, 21.0, 288.99, 11.69
+18.2, 0.7258, 0.0, 8.14, 0.0, 0.538, 5.727, 69.5, 3.7965, 4.0, 307.0, 21.0, 390.95, 11.28
+13.6, 1.25179, 0.0, 8.14, 0.0, 0.538, 5.57, 98.1, 3.7979, 4.0, 307.0, 21.0, 376.57, 21.02
+19.6, 0.85204, 0.0, 8.14, 0.0, 0.538, 5.965, 89.2, 4.0123, 4.0, 307.0, 21.0, 392.53, 13.83
+15.2, 1.23247, 0.0, 8.14, 0.0, 0.538, 6.142, 91.7, 3.9769, 4.0, 307.0, 21.0, 396.9, 18.72
+14.5, 0.98843, 0.0, 8.14, 0.0, 0.538, 5.813, 100.0, 4.0952, 4.0, 307.0, 21.0, 394.54, 19.88
+15.6, 0.75026, 0.0, 8.14, 0.0, 0.538, 5.924, 94.1, 4.3996, 4.0, 307.0, 21.0, 394.33, 16.3
+13.9, 0.84054, 0.0, 8.14, 0.0, 0.538, 5.599, 85.7, 4.4546, 4.0, 307.0, 21.0, 303.42, 16.51
+16.6, 0.67191, 0.0, 8.14, 0.0, 0.538, 5.813, 90.3, 4.682, 4.0, 307.0, 21.0, 376.88, 14.81
+14.8, 0.95577, 0.0, 8.14, 0.0, 0.538, 6.047, 88.8, 4.4534, 4.0, 307.0, 21.0, 306.38, 17.28
+18.4, 0.77299, 0.0, 8.14, 0.0, 0.538, 6.495, 94.4, 4.4547, 4.0, 307.0, 21.0, 387.94, 12.8
+21.0, 1.00245, 0.0, 8.14, 0.0, 0.538, 6.674, 87.3, 4.239, 4.0, 307.0, 21.0, 380.23, 11.98
+12.7, 1.13081, 0.0, 8.14, 0.0, 0.538, 5.713, 94.1, 4.233, 4.0, 307.0, 21.0, 360.17, 22.6
+14.5, 1.35472, 0.0, 8.14, 0.0, 0.538, 6.072, 100.0, 4.175, 4.0, 307.0, 21.0, 376.73, 13.04
+13.2, 1.38799, 0.0, 8.14, 0.0, 0.538, 5.95, 82.0, 3.99, 4.0, 307.0, 21.0, 232.6, 27.71
+13.1, 1.15172, 0.0, 8.14, 0.0, 0.538, 5.701, 95.0, 3.7872, 4.0, 307.0, 21.0, 358.77, 18.35
+13.5, 1.61282, 0.0, 8.14, 0.0, 0.538, 6.096, 96.9, 3.7598, 4.0, 307.0, 21.0, 248.31, 20.34
+18.9, 0.06417, 0.0, 5.96, 0.0, 0.499, 5.933, 68.2, 3.3603, 5.0, 279.0, 19.2, 396.9, 9.68
+20.0, 0.09744, 0.0, 5.96, 0.0, 0.499, 5.841, 61.4, 3.3779, 5.0, 279.0, 19.2, 377.56, 11.41
+21.0, 0.08014, 0.0, 5.96, 0.0, 0.499, 5.85, 41.5, 3.9342, 5.0, 279.0, 19.2, 396.9, 8.77
+24.7, 0.17505, 0.0, 5.96, 0.0, 0.499, 5.966, 30.2, 3.8473, 5.0, 279.0, 19.2, 393.43, 10.13
+30.8, 0.02763, 75.0, 2.95, 0.0, 0.428, 6.595, 21.8, 5.4011, 3.0, 252.0, 18.3, 395.63, 4.32
+34.9, 0.03359, 75.0, 2.95, 0.0, 0.428, 7.024, 15.8, 5.4011, 3.0, 252.0, 18.3, 395.62, 1.98
+26.6, 0.12744, 0.0, 6.91, 0.0, 0.448, 6.77, 2.9, 5.7209, 3.0, 233.0, 17.9, 385.41, 4.84
+25.3, 0.1415, 0.0, 6.91, 0.0, 0.448, 6.169, 6.6, 5.7209, 3.0, 233.0, 17.9, 383.37, 5.81
+24.7, 0.15936, 0.0, 6.91, 0.0, 0.448, 6.211, 6.5, 5.7209, 3.0, 233.0, 17.9, 394.46, 7.44
+21.2, 0.12269, 0.0, 6.91, 0.0, 0.448, 6.069, 40.0, 5.7209, 3.0, 233.0, 17.9, 389.39, 9.55
+19.3, 0.17142, 0.0, 6.91, 0.0, 0.448, 5.682, 33.8, 5.1004, 3.0, 233.0, 17.9, 396.9, 10.21
+20.0, 0.18836, 0.0, 6.91, 0.0, 0.448, 5.786, 33.3, 5.1004, 3.0, 233.0, 17.9, 396.9, 14.15
+16.6, 0.22927, 0.0, 6.91, 0.0, 0.448, 6.03, 85.5, 5.6894, 3.0, 233.0, 17.9, 392.74, 18.8
+14.4, 0.25387, 0.0, 6.91, 0.0, 0.448, 5.399, 95.3, 5.87, 3.0, 233.0, 17.9, 396.9, 30.81
+19.4, 0.21977, 0.0, 6.91, 0.0, 0.448, 5.602, 62.0, 6.0877, 3.0, 233.0, 17.9, 396.9, 16.2
+19.7, 0.08873, 21.0, 5.64, 0.0, 0.439, 5.963, 45.7, 6.8147, 4.0, 243.0, 16.8, 395.56, 13.45
+20.5, 0.04337, 21.0, 5.64, 0.0, 0.439, 6.115, 63.0, 6.8147, 4.0, 243.0, 16.8, 393.97, 9.43
+25.0, 0.0536, 21.0, 5.64, 0.0, 0.439, 6.511, 21.1, 6.8147, 4.0, 243.0, 16.8, 396.9, 5.28
+23.4, 0.04981, 21.0, 5.64, 0.0, 0.439, 5.998, 21.4, 6.8147, 4.0, 243.0, 16.8, 396.9, 8.43
+18.9, 0.0136, 75.0, 4.0, 0.0, 0.41, 5.888, 47.6, 7.3197, 3.0, 469.0, 21.1, 396.9, 14.8
+35.4, 0.01311, 90.0, 1.22, 0.0, 0.403, 7.249, 21.9, 8.6966, 5.0, 226.0, 17.9, 395.93, 4.81
+24.7, 0.02055, 85.0, 0.74, 0.0, 0.41, 6.383, 35.7, 9.1876, 2.0, 313.0, 17.3, 396.9, 5.77
+31.6, 0.01432, 100.0, 1.32, 0.0, 0.411, 6.816, 40.5, 8.3248, 5.0, 256.0, 15.1, 392.9, 3.95
+23.3, 0.15445, 25.0, 5.13, 0.0, 0.453, 6.145, 29.2, 7.8148, 8.0, 284.0, 19.7, 390.68, 6.86
+19.6, 0.10328, 25.0, 5.13, 0.0, 0.453, 5.927, 47.2, 6.932, 8.0, 284.0, 19.7, 396.9, 9.22
+18.7, 0.14932, 25.0, 5.13, 0.0, 0.453, 5.741, 66.2, 7.2254, 8.0, 284.0, 19.7, 395.11, 13.15
+16.0, 0.17171, 25.0, 5.13, 0.0, 0.453, 5.966, 93.4, 6.8185, 8.0, 284.0, 19.7, 378.08, 14.44
+22.2, 0.11027, 25.0, 5.13, 0.0, 0.453, 6.456, 67.8, 7.2255, 8.0, 284.0, 19.7, 396.9, 6.73
+25.0, 0.1265, 25.0, 5.13, 0.0, 0.453, 6.762, 43.4, 7.9809, 8.0, 284.0, 19.7, 395.58, 9.5
+33.0, 0.01951, 17.5, 1.38, 0.0, 0.4161, 7.104, 59.5, 9.2229, 3.0, 216.0, 18.6, 393.24, 8.05
+23.5, 0.03584, 80.0, 3.37, 0.0, 0.398, 6.29, 17.8, 6.6115, 4.0, 337.0, 16.1, 396.9, 4.67
+19.4, 0.04379, 80.0, 3.37, 0.0, 0.398, 5.787, 31.1, 6.6115, 4.0, 337.0, 16.1, 396.9, 10.24
+22.0, 0.05789, 12.5, 6.07, 0.0, 0.409, 5.878, 21.4, 6.498, 4.0, 345.0, 18.9, 396.21, 8.1
+17.4, 0.13554, 12.5, 6.07, 0.0, 0.409, 5.594, 36.8, 6.498, 4.0, 345.0, 18.9, 396.9, 13.09
+20.9, 0.12816, 12.5, 6.07, 0.0, 0.409, 5.885, 33.0, 6.498, 4.0, 345.0, 18.9, 396.9, 8.79
+24.2, 0.08826, 0.0, 10.81, 0.0, 0.413, 6.417, 6.6, 5.2873, 4.0, 305.0, 19.2, 383.73, 6.72
+21.7, 0.15876, 0.0, 10.81, 0.0, 0.413, 5.961, 17.5, 5.2873, 4.0, 305.0, 19.2, 376.94, 9.88
+22.8, 0.09164, 0.0, 10.81, 0.0, 0.413, 6.065, 7.8, 5.2873, 4.0, 305.0, 19.2, 390.91, 5.52
+23.4, 0.19539, 0.0, 10.81, 0.0, 0.413, 6.245, 6.2, 5.2873, 4.0, 305.0, 19.2, 377.17, 7.54
+24.1, 0.07896, 0.0, 12.83, 0.0, 0.437, 6.273, 6.0, 4.2515, 5.0, 398.0, 18.7, 394.92, 6.78
+21.4, 0.09512, 0.0, 12.83, 0.0, 0.437, 6.286, 45.0, 4.5026, 5.0, 398.0, 18.7, 383.23, 8.94
+20.0, 0.10153, 0.0, 12.83, 0.0, 0.437, 6.279, 74.5, 4.0522, 5.0, 398.0, 18.7, 373.66, 11.97
+20.8, 0.08707, 0.0, 12.83, 0.0, 0.437, 6.14, 45.8, 4.0905, 5.0, 398.0, 18.7, 386.96, 10.27
+21.2, 0.05646, 0.0, 12.83, 0.0, 0.437, 6.232, 53.7, 5.0141, 5.0, 398.0, 18.7, 386.4, 12.34
+20.3, 0.08387, 0.0, 12.83, 0.0, 0.437, 5.874, 36.6, 4.5026, 5.0, 398.0, 18.7, 396.06, 9.1
+28.0, 0.04113, 25.0, 4.86, 0.0, 0.426, 6.727, 33.5, 5.4007, 4.0, 281.0, 19.0, 396.9, 5.29
+23.9, 0.04462, 25.0, 4.86, 0.0, 0.426, 6.619, 70.4, 5.4007, 4.0, 281.0, 19.0, 395.63, 7.22
+24.8, 0.03659, 25.0, 4.86, 0.0, 0.426, 6.302, 32.2, 5.4007, 4.0, 281.0, 19.0, 396.9, 6.72
+22.9, 0.03551, 25.0, 4.86, 0.0, 0.426, 6.167, 46.7, 5.4007, 4.0, 281.0, 19.0, 390.64, 7.51
+23.9, 0.05059, 0.0, 4.49, 0.0, 0.449, 6.389, 48.0, 4.7794, 3.0, 247.0, 18.5, 396.9, 9.62
+26.6, 0.05735, 0.0, 4.49, 0.0, 0.449, 6.63, 56.1, 4.4377, 3.0, 247.0, 18.5, 392.3, 6.53
+22.5, 0.05188, 0.0, 4.49, 0.0, 0.449, 6.015, 45.1, 4.4272, 3.0, 247.0, 18.5, 395.99, 12.86
+22.2, 0.07151, 0.0, 4.49, 0.0, 0.449, 6.121, 56.8, 3.7476, 3.0, 247.0, 18.5, 395.15, 8.44
+23.6, 0.0566, 0.0, 3.41, 0.0, 0.489, 7.007, 86.3, 3.4217, 2.0, 270.0, 17.8, 396.9, 5.5
+28.7, 0.05302, 0.0, 3.41, 0.0, 0.489, 7.079, 63.1, 3.4145, 2.0, 270.0, 17.8, 396.06, 5.7
+22.6, 0.04684, 0.0, 3.41, 0.0, 0.489, 6.417, 66.1, 3.0923, 2.0, 270.0, 17.8, 392.18, 8.81
+22.0, 0.03932, 0.0, 3.41, 0.0, 0.489, 6.405, 73.9, 3.0921, 2.0, 270.0, 17.8, 393.55, 8.2
+22.9, 0.04203, 28.0, 15.04, 0.0, 0.464, 6.442, 53.6, 3.6659, 4.0, 270.0, 18.2, 395.01, 8.16
+25.0, 0.02875, 28.0, 15.04, 0.0, 0.464, 6.211, 28.9, 3.6659, 4.0, 270.0, 18.2, 396.33, 6.21
+20.6, 0.04294, 28.0, 15.04, 0.0, 0.464, 6.249, 77.3, 3.615, 4.0, 270.0, 18.2, 396.9, 10.59
+28.4, 0.12204, 0.0, 2.89, 0.0, 0.445, 6.625, 57.8, 3.4952, 2.0, 276.0, 18.0, 357.98, 6.65
+21.4, 0.11504, 0.0, 2.89, 0.0, 0.445, 6.163, 69.6, 3.4952, 2.0, 276.0, 18.0, 391.83, 11.34
+38.7, 0.12083, 0.0, 2.89, 0.0, 0.445, 8.069, 76.0, 3.4952, 2.0, 276.0, 18.0, 396.9, 4.21
+43.8, 0.08187, 0.0, 2.89, 0.0, 0.445, 7.82, 36.9, 3.4952, 2.0, 276.0, 18.0, 393.53, 3.57
+33.2, 0.0686, 0.0, 2.89, 0.0, 0.445, 7.416, 62.5, 3.4952, 2.0, 276.0, 18.0, 396.9, 6.19
+27.5, 0.14866, 0.0, 8.56, 0.0, 0.52, 6.727, 79.9, 2.7778, 5.0, 384.0, 20.9, 394.76, 9.42
+26.5, 0.11432, 0.0, 8.56, 0.0, 0.52, 6.781, 71.3, 2.8561, 5.0, 384.0, 20.9, 395.58, 7.67
+18.6, 0.22876, 0.0, 8.56, 0.0, 0.52, 6.405, 85.4, 2.7147, 5.0, 384.0, 20.9, 70.8, 10.63
+19.3, 0.21161, 0.0, 8.56, 0.0, 0.52, 6.137, 87.4, 2.7147, 5.0, 384.0, 20.9, 394.47, 13.44
+20.1, 0.1396, 0.0, 8.56, 0.0, 0.52, 6.167, 90.0, 2.421, 5.0, 384.0, 20.9, 392.69, 12.33
+19.5, 0.13262, 0.0, 8.56, 0.0, 0.52, 5.851, 96.7, 2.1069, 5.0, 384.0, 20.9, 394.05, 16.47
+19.5, 0.1712, 0.0, 8.56, 0.0, 0.52, 5.836, 91.9, 2.211, 5.0, 384.0, 20.9, 395.67, 18.66
+20.4, 0.13117, 0.0, 8.56, 0.0, 0.52, 6.127, 85.2, 2.1224, 5.0, 384.0, 20.9, 387.69, 14.09
+19.8, 0.12802, 0.0, 8.56, 0.0, 0.52, 6.474, 97.1, 2.4329, 5.0, 384.0, 20.9, 395.24, 12.27
+19.4, 0.26363, 0.0, 8.56, 0.0, 0.52, 6.229, 91.2, 2.5451, 5.0, 384.0, 20.9, 391.23, 15.55
+21.7, 0.10793, 0.0, 8.56, 0.0, 0.52, 6.195, 54.4, 2.7778, 5.0, 384.0, 20.9, 393.49, 13.0
+22.8, 0.10084, 0.0, 10.01, 0.0, 0.547, 6.715, 81.6, 2.6775, 6.0, 432.0, 17.8, 395.59, 10.16
+18.8, 0.12329, 0.0, 10.01, 0.0, 0.547, 5.913, 92.9, 2.3534, 6.0, 432.0, 17.8, 394.95, 16.21
+18.7, 0.22212, 0.0, 10.01, 0.0, 0.547, 6.092, 95.4, 2.548, 6.0, 432.0, 17.8, 396.9, 17.09
+18.5, 0.14231, 0.0, 10.01, 0.0, 0.547, 6.254, 84.2, 2.2565, 6.0, 432.0, 17.8, 388.74, 10.45
+18.3, 0.17134, 0.0, 10.01, 0.0, 0.547, 5.928, 88.2, 2.4631, 6.0, 432.0, 17.8, 344.91, 15.76
+21.2, 0.13158, 0.0, 10.01, 0.0, 0.547, 6.176, 72.5, 2.7301, 6.0, 432.0, 17.8, 393.3, 12.04
+19.2, 0.15098, 0.0, 10.01, 0.0, 0.547, 6.021, 82.6, 2.7474, 6.0, 432.0, 17.8, 394.51, 10.3
+20.4, 0.13058, 0.0, 10.01, 0.0, 0.547, 5.872, 73.1, 2.4775, 6.0, 432.0, 17.8, 338.63, 15.37
+19.3, 0.14476, 0.0, 10.01, 0.0, 0.547, 5.731, 65.2, 2.7592, 6.0, 432.0, 17.8, 391.5, 13.61
+22.0, 0.06899, 0.0, 25.65, 0.0, 0.581, 5.87, 69.7, 2.2577, 2.0, 188.0, 19.1, 389.15, 14.37
+20.3, 0.07165, 0.0, 25.65, 0.0, 0.581, 6.004, 84.1, 2.1974, 2.0, 188.0, 19.1, 377.67, 14.27
+20.5, 0.09299, 0.0, 25.65, 0.0, 0.581, 5.961, 92.9, 2.0869, 2.0, 188.0, 19.1, 378.09, 17.93
+17.3, 0.15038, 0.0, 25.65, 0.0, 0.581, 5.856, 97.0, 1.9444, 2.0, 188.0, 19.1, 370.31, 25.41
+18.8, 0.09849, 0.0, 25.65, 0.0, 0.581, 5.879, 95.8, 2.0063, 2.0, 188.0, 19.1, 379.38, 17.58
+21.4, 0.16902, 0.0, 25.65, 0.0, 0.581, 5.986, 88.4, 1.9929, 2.0, 188.0, 19.1, 385.02, 14.81
+15.7, 0.38735, 0.0, 25.65, 0.0, 0.581, 5.613, 95.6, 1.7572, 2.0, 188.0, 19.1, 359.29, 27.26
+16.2, 0.25915, 0.0, 21.89, 0.0, 0.624, 5.693, 96.0, 1.7883, 4.0, 437.0, 21.2, 392.11, 17.19
+18.0, 0.32543, 0.0, 21.89, 0.0, 0.624, 6.431, 98.8, 1.8125, 4.0, 437.0, 21.2, 396.9, 15.39
+14.3, 0.88125, 0.0, 21.89, 0.0, 0.624, 5.637, 94.7, 1.9799, 4.0, 437.0, 21.2, 396.9, 18.34
+19.2, 0.34006, 0.0, 21.89, 0.0, 0.624, 6.458, 98.9, 2.1185, 4.0, 437.0, 21.2, 395.04, 12.6
+19.6, 1.19294, 0.0, 21.89, 0.0, 0.624, 6.326, 97.7, 2.271, 4.0, 437.0, 21.2, 396.9, 12.26
+23.0, 0.59005, 0.0, 21.89, 0.0, 0.624, 6.372, 97.9, 2.3274, 4.0, 437.0, 21.2, 385.76, 11.12
+18.4, 0.32982, 0.0, 21.89, 0.0, 0.624, 5.822, 95.4, 2.4699, 4.0, 437.0, 21.2, 388.69, 15.03
+15.6, 0.97617, 0.0, 21.89, 0.0, 0.624, 5.757, 98.4, 2.346, 4.0, 437.0, 21.2, 262.76, 17.31
+18.1, 0.55778, 0.0, 21.89, 0.0, 0.624, 6.335, 98.2, 2.1107, 4.0, 437.0, 21.2, 394.67, 16.96
+17.4, 0.32264, 0.0, 21.89, 0.0, 0.624, 5.942, 93.5, 1.9669, 4.0, 437.0, 21.2, 378.25, 16.9
+17.1, 0.35233, 0.0, 21.89, 0.0, 0.624, 6.454, 98.4, 1.8498, 4.0, 437.0, 21.2, 394.08, 14.59
+13.3, 0.2498, 0.0, 21.89, 0.0, 0.624, 5.857, 98.2, 1.6686, 4.0, 437.0, 21.2, 392.04, 21.32
+17.8, 0.54452, 0.0, 21.89, 0.0, 0.624, 6.151, 97.9, 1.6687, 4.0, 437.0, 21.2, 396.9, 18.46
+14.0, 0.2909, 0.0, 21.89, 0.0, 0.624, 6.174, 93.6, 1.6119, 4.0, 437.0, 21.2, 388.08, 24.16
+14.4, 1.62864, 0.0, 21.89, 0.0, 0.624, 5.019, 100.0, 1.4394, 4.0, 437.0, 21.2, 396.9, 34.41
+13.4, 3.32105, 0.0, 19.58, 1.0, 0.871, 5.403, 100.0, 1.3216, 5.0, 403.0, 14.7, 396.9, 26.82
+15.6, 4.0974, 0.0, 19.58, 0.0, 0.871, 5.468, 100.0, 1.4118, 5.0, 403.0, 14.7, 396.9, 26.42
+11.8, 2.77974, 0.0, 19.58, 0.0, 0.871, 4.903, 97.8, 1.3459, 5.0, 403.0, 14.7, 396.9, 29.29
+13.8, 2.37934, 0.0, 19.58, 0.0, 0.871, 6.13, 100.0, 1.4191, 5.0, 403.0, 14.7, 172.91, 27.8
+15.6, 2.15505, 0.0, 19.58, 0.0, 0.871, 5.628, 100.0, 1.5166, 5.0, 403.0, 14.7, 169.27, 16.65
+14.6, 2.36862, 0.0, 19.58, 0.0, 0.871, 4.926, 95.7, 1.4608, 5.0, 403.0, 14.7, 391.71, 29.53
+17.8, 2.33099, 0.0, 19.58, 0.0, 0.871, 5.186, 93.8, 1.5296, 5.0, 403.0, 14.7, 356.99, 28.32
+15.4, 2.73397, 0.0, 19.58, 0.0, 0.871, 5.597, 94.9, 1.5257, 5.0, 403.0, 14.7, 351.85, 21.45
+21.5, 1.6566, 0.0, 19.58, 0.0, 0.871, 6.122, 97.3, 1.618, 5.0, 403.0, 14.7, 372.8, 14.1
+19.6, 1.49632, 0.0, 19.58, 0.0, 0.871, 5.404, 100.0, 1.5916, 5.0, 403.0, 14.7, 341.6, 13.28
+15.3, 1.12658, 0.0, 19.58, 1.0, 0.871, 5.012, 88.0, 1.6102, 5.0, 403.0, 14.7, 343.28, 12.12
+19.4, 2.14918, 0.0, 19.58, 0.0, 0.871, 5.709, 98.5, 1.6232, 5.0, 403.0, 14.7, 261.95, 15.79
+17.0, 1.41385, 0.0, 19.58, 1.0, 0.871, 6.129, 96.0, 1.7494, 5.0, 403.0, 14.7, 321.02, 15.12
+15.6, 3.53501, 0.0, 19.58, 1.0, 0.871, 6.152, 82.6, 1.7455, 5.0, 403.0, 14.7, 88.01, 15.02
+13.1, 2.44668, 0.0, 19.58, 0.0, 0.871, 5.272, 94.0, 1.7364, 5.0, 403.0, 14.7, 88.63, 16.14
+41.3, 1.22358, 0.0, 19.58, 0.0, 0.605, 6.943, 97.4, 1.8773, 5.0, 403.0, 14.7, 363.43, 4.59
+24.3, 1.34284, 0.0, 19.58, 0.0, 0.605, 6.066, 100.0, 1.7573, 5.0, 403.0, 14.7, 353.89, 6.43
+23.3, 1.42502, 0.0, 19.58, 0.0, 0.871, 6.51, 100.0, 1.7659, 5.0, 403.0, 14.7, 364.31, 7.39
+27.0, 1.27346, 0.0, 19.58, 1.0, 0.605, 6.25, 92.6, 1.7984, 5.0, 403.0, 14.7, 338.92, 5.5
+50.0, 1.46336, 0.0, 19.58, 0.0, 0.605, 7.489, 90.8, 1.9709, 5.0, 403.0, 14.7, 374.43, 1.73
+50.0, 1.83377, 0.0, 19.58, 1.0, 0.605, 7.802, 98.2, 2.0407, 5.0, 403.0, 14.7, 389.61, 1.92
+50.0, 1.51902, 0.0, 19.58, 1.0, 0.605, 8.375, 93.9, 2.162, 5.0, 403.0, 14.7, 388.45, 3.32
+22.7, 2.24236, 0.0, 19.58, 0.0, 0.605, 5.854, 91.8, 2.422, 5.0, 403.0, 14.7, 395.11, 11.64
+25.0, 2.924, 0.0, 19.58, 0.0, 0.605, 6.101, 93.0, 2.2834, 5.0, 403.0, 14.7, 240.16, 9.81
+50.0, 2.01019, 0.0, 19.58, 0.0, 0.605, 7.929, 96.2, 2.0459, 5.0, 403.0, 14.7, 369.3, 3.7
+23.8, 1.80028, 0.0, 19.58, 0.0, 0.605, 5.877, 79.2, 2.4259, 5.0, 403.0, 14.7, 227.61, 12.14
+23.8, 2.3004, 0.0, 19.58, 0.0, 0.605, 6.319, 96.1, 2.1, 5.0, 403.0, 14.7, 297.09, 11.1
+22.3, 2.44953, 0.0, 19.58, 0.0, 0.605, 6.402, 95.2, 2.2625, 5.0, 403.0, 14.7, 330.04, 11.32
+17.4, 1.20742, 0.0, 19.58, 0.0, 0.605, 5.875, 94.6, 2.4259, 5.0, 403.0, 14.7, 292.29, 14.43
+19.1, 2.3139, 0.0, 19.58, 0.0, 0.605, 5.88, 97.3, 2.3887, 5.0, 403.0, 14.7, 348.13, 12.03
+23.1, 0.13914, 0.0, 4.05, 0.0, 0.51, 5.572, 88.5, 2.5961, 5.0, 296.0, 16.6, 396.9, 14.69
+23.6, 0.09178, 0.0, 4.05, 0.0, 0.51, 6.416, 84.1, 2.6463, 5.0, 296.0, 16.6, 395.5, 9.04
+22.6, 0.08447, 0.0, 4.05, 0.0, 0.51, 5.859, 68.7, 2.7019, 5.0, 296.0, 16.6, 393.23, 9.64
+29.4, 0.06664, 0.0, 4.05, 0.0, 0.51, 6.546, 33.1, 3.1323, 5.0, 296.0, 16.6, 390.96, 5.33
+23.2, 0.07022, 0.0, 4.05, 0.0, 0.51, 6.02, 47.2, 3.5549, 5.0, 296.0, 16.6, 393.23, 10.11
+24.6, 0.05425, 0.0, 4.05, 0.0, 0.51, 6.315, 73.4, 3.3175, 5.0, 296.0, 16.6, 395.6, 6.29
+29.9, 0.06642, 0.0, 4.05, 0.0, 0.51, 6.86, 74.4, 2.9153, 5.0, 296.0, 16.6, 391.27, 6.92
+37.2, 0.0578, 0.0, 2.46, 0.0, 0.488, 6.98, 58.4, 2.829, 3.0, 193.0, 17.8, 396.9, 5.04
+39.8, 0.06588, 0.0, 2.46, 0.0, 0.488, 7.765, 83.3, 2.741, 3.0, 193.0, 17.8, 395.56, 7.56
+36.2, 0.06888, 0.0, 2.46, 0.0, 0.488, 6.144, 62.2, 2.5979, 3.0, 193.0, 17.8, 396.9, 9.45
+37.9, 0.09103, 0.0, 2.46, 0.0, 0.488, 7.155, 92.2, 2.7006, 3.0, 193.0, 17.8, 394.12, 4.82
+32.5, 0.10008, 0.0, 2.46, 0.0, 0.488, 6.563, 95.6, 2.847, 3.0, 193.0, 17.8, 396.9, 5.68
+26.4, 0.08308, 0.0, 2.46, 0.0, 0.488, 5.604, 89.8, 2.9879, 3.0, 193.0, 17.8, 391.0, 13.98
+29.6, 0.06047, 0.0, 2.46, 0.0, 0.488, 6.153, 68.8, 3.2797, 3.0, 193.0, 17.8, 387.11, 13.15
+50.0, 0.05602, 0.0, 2.46, 0.0, 0.488, 7.831, 53.6, 3.1992, 3.0, 193.0, 17.8, 392.63, 4.45
+32.0, 0.07875, 45.0, 3.44, 0.0, 0.437, 6.782, 41.1, 3.7886, 5.0, 398.0, 15.2, 393.87, 6.68
+29.8, 0.12579, 45.0, 3.44, 0.0, 0.437, 6.556, 29.1, 4.5667, 5.0, 398.0, 15.2, 382.84, 4.56
+34.9, 0.0837, 45.0, 3.44, 0.0, 0.437, 7.185, 38.9, 4.5667, 5.0, 398.0, 15.2, 396.9, 5.39
+37.0, 0.09068, 45.0, 3.44, 0.0, 0.437, 6.951, 21.5, 6.4798, 5.0, 398.0, 15.2, 377.68, 5.1
+30.5, 0.06911, 45.0, 3.44, 0.0, 0.437, 6.739, 30.8, 6.4798, 5.0, 398.0, 15.2, 389.71, 4.69
+36.4, 0.08664, 45.0, 3.44, 0.0, 0.437, 7.178, 26.3, 6.4798, 5.0, 398.0, 15.2, 390.49, 2.87
+31.1, 0.02187, 60.0, 2.93, 0.0, 0.401, 6.8, 9.9, 6.2196, 1.0, 265.0, 15.6, 393.37, 5.03
+29.1, 0.01439, 60.0, 2.93, 0.0, 0.401, 6.604, 18.8, 6.2196, 1.0, 265.0, 15.6, 376.7, 4.38
+50.0, 0.01381, 80.0, 0.46, 0.0, 0.422, 7.875, 32.0, 5.6484, 4.0, 255.0, 14.4, 394.23, 2.97
+33.3, 0.04011, 80.0, 1.52, 0.0, 0.404, 7.287, 34.1, 7.309, 2.0, 329.0, 12.6, 396.9, 4.08
+30.3, 0.04666, 80.0, 1.52, 0.0, 0.404, 7.107, 36.6, 7.309, 2.0, 329.0, 12.6, 354.31, 8.61
+34.6, 0.03768, 80.0, 1.52, 0.0, 0.404, 7.274, 38.3, 7.309, 2.0, 329.0, 12.6, 392.2, 6.62
+34.9, 0.0315, 95.0, 1.47, 0.0, 0.403, 6.975, 15.3, 7.6534, 3.0, 402.0, 17.0, 396.9, 4.56
+32.9, 0.01778, 95.0, 1.47, 0.0, 0.403, 7.135, 13.9, 7.6534, 3.0, 402.0, 17.0, 384.3, 4.45
+24.1, 0.03445, 82.5, 2.03, 0.0, 0.415, 6.162, 38.4, 6.27, 2.0, 348.0, 14.7, 393.77, 7.43
+42.3, 0.02177, 82.5, 2.03, 0.0, 0.415, 7.61, 15.7, 6.27, 2.0, 348.0, 14.7, 395.38, 3.11
+48.5, 0.0351, 95.0, 2.68, 0.0, 0.4161, 7.853, 33.2, 5.118, 4.0, 224.0, 14.7, 392.78, 3.81
+50.0, 0.02009, 95.0, 2.68, 0.0, 0.4161, 8.034, 31.9, 5.118, 4.0, 224.0, 14.7, 390.55, 2.88
+22.6, 0.13642, 0.0, 10.59, 0.0, 0.489, 5.891, 22.3, 3.9454, 4.0, 277.0, 18.6, 396.9, 10.87
+24.4, 0.22969, 0.0, 10.59, 0.0, 0.489, 6.326, 52.5, 4.3549, 4.0, 277.0, 18.6, 394.87, 10.97
+22.5, 0.25199, 0.0, 10.59, 0.0, 0.489, 5.783, 72.7, 4.3549, 4.0, 277.0, 18.6, 389.43, 18.06
+24.4, 0.13587, 0.0, 10.59, 1.0, 0.489, 6.064, 59.1, 4.2392, 4.0, 277.0, 18.6, 381.32, 14.66
+20.0, 0.43571, 0.0, 10.59, 1.0, 0.489, 5.344, 100.0, 3.875, 4.0, 277.0, 18.6, 396.9, 23.09
+21.7, 0.17446, 0.0, 10.59, 1.0, 0.489, 5.96, 92.1, 3.8771, 4.0, 277.0, 18.6, 393.25, 17.27
+19.3, 0.37578, 0.0, 10.59, 1.0, 0.489, 5.404, 88.6, 3.665, 4.0, 277.0, 18.6, 395.24, 23.98
+22.4, 0.21719, 0.0, 10.59, 1.0, 0.489, 5.807, 53.8, 3.6526, 4.0, 277.0, 18.6, 390.94, 16.03
+28.1, 0.14052, 0.0, 10.59, 0.0, 0.489, 6.375, 32.3, 3.9454, 4.0, 277.0, 18.6, 385.81, 9.38
+23.7, 0.28955, 0.0, 10.59, 0.0, 0.489, 5.412, 9.8, 3.5875, 4.0, 277.0, 18.6, 348.93, 29.55
+25.0, 0.19802, 0.0, 10.59, 0.0, 0.489, 6.182, 42.4, 3.9454, 4.0, 277.0, 18.6, 393.63, 9.47
+23.3, 0.0456, 0.0, 13.89, 1.0, 0.55, 5.888, 56.0, 3.1121, 5.0, 276.0, 16.4, 392.8, 13.51
+28.7, 0.07013, 0.0, 13.89, 0.0, 0.55, 6.642, 85.1, 3.4211, 5.0, 276.0, 16.4, 392.78, 9.69
+21.5, 0.11069, 0.0, 13.89, 1.0, 0.55, 5.951, 93.8, 2.8893, 5.0, 276.0, 16.4, 396.9, 17.92
+23.0, 0.11425, 0.0, 13.89, 1.0, 0.55, 6.373, 92.4, 3.3633, 5.0, 276.0, 16.4, 393.74, 10.5
+26.7, 0.35809, 0.0, 6.2, 1.0, 0.507, 6.951, 88.5, 2.8617, 8.0, 307.0, 17.4, 391.7, 9.71
+21.7, 0.40771, 0.0, 6.2, 1.0, 0.507, 6.164, 91.3, 3.048, 8.0, 307.0, 17.4, 395.24, 21.46
+27.5, 0.62356, 0.0, 6.2, 1.0, 0.507, 6.879, 77.7, 3.2721, 8.0, 307.0, 17.4, 390.39, 9.93
+30.1, 0.6147, 0.0, 6.2, 0.0, 0.507, 6.618, 80.8, 3.2721, 8.0, 307.0, 17.4, 396.9, 7.6
+44.8, 0.31533, 0.0, 6.2, 0.0, 0.504, 8.266, 78.3, 2.8944, 8.0, 307.0, 17.4, 385.05, 4.14
+50.0, 0.52693, 0.0, 6.2, 0.0, 0.504, 8.725, 83.0, 2.8944, 8.0, 307.0, 17.4, 382.0, 4.63
+37.6, 0.38214, 0.0, 6.2, 0.0, 0.504, 8.04, 86.5, 3.2157, 8.0, 307.0, 17.4, 387.38, 3.13
+31.6, 0.41238, 0.0, 6.2, 0.0, 0.504, 7.163, 79.9, 3.2157, 8.0, 307.0, 17.4, 372.08, 6.36
+46.7, 0.29819, 0.0, 6.2, 0.0, 0.504, 7.686, 17.0, 3.3751, 8.0, 307.0, 17.4, 377.51, 3.92
+31.5, 0.44178, 0.0, 6.2, 0.0, 0.504, 6.552, 21.4, 3.3751, 8.0, 307.0, 17.4, 380.34, 3.76
+24.3, 0.537, 0.0, 6.2, 0.0, 0.504, 5.981, 68.1, 3.6715, 8.0, 307.0, 17.4, 378.35, 11.65
+31.7, 0.46296, 0.0, 6.2, 0.0, 0.504, 7.412, 76.9, 3.6715, 8.0, 307.0, 17.4, 376.14, 5.25
+41.7, 0.57529, 0.0, 6.2, 0.0, 0.507, 8.337, 73.3, 3.8384, 8.0, 307.0, 17.4, 385.91, 2.47
+48.3, 0.33147, 0.0, 6.2, 0.0, 0.507, 8.247, 70.4, 3.6519, 8.0, 307.0, 17.4, 378.95, 3.95
+29.0, 0.44791, 0.0, 6.2, 1.0, 0.507, 6.726, 66.5, 3.6519, 8.0, 307.0, 17.4, 360.2, 8.05
+24.0, 0.33045, 0.0, 6.2, 0.0, 0.507, 6.086, 61.5, 3.6519, 8.0, 307.0, 17.4, 376.75, 10.88
+25.1, 0.52058, 0.0, 6.2, 1.0, 0.507, 6.631, 76.5, 4.148, 8.0, 307.0, 17.4, 388.45, 9.54
+31.5, 0.51183, 0.0, 6.2, 0.0, 0.507, 7.358, 71.6, 4.148, 8.0, 307.0, 17.4, 390.07, 4.73
+23.7, 0.08244, 30.0, 4.93, 0.0, 0.428, 6.481, 18.5, 6.1899, 6.0, 300.0, 16.6, 379.41, 6.36
+23.3, 0.09252, 30.0, 4.93, 0.0, 0.428, 6.606, 42.2, 6.1899, 6.0, 300.0, 16.6, 383.78, 7.37
+22.0, 0.11329, 30.0, 4.93, 0.0, 0.428, 6.897, 54.3, 6.3361, 6.0, 300.0, 16.6, 391.25, 11.38
+20.1, 0.10612, 30.0, 4.93, 0.0, 0.428, 6.095, 65.1, 6.3361, 6.0, 300.0, 16.6, 394.62, 12.4
+22.2, 0.1029, 30.0, 4.93, 0.0, 0.428, 6.358, 52.9, 7.0355, 6.0, 300.0, 16.6, 372.75, 11.22
+23.7, 0.12757, 30.0, 4.93, 0.0, 0.428, 6.393, 7.8, 7.0355, 6.0, 300.0, 16.6, 374.71, 5.19
+17.6, 0.20608, 22.0, 5.86, 0.0, 0.431, 5.593, 76.5, 7.9549, 7.0, 330.0, 19.1, 372.49, 12.5
+18.5, 0.19133, 22.0, 5.86, 0.0, 0.431, 5.605, 70.2, 7.9549, 7.0, 330.0, 19.1, 389.13, 18.46
+24.3, 0.33983, 22.0, 5.86, 0.0, 0.431, 6.108, 34.9, 8.0555, 7.0, 330.0, 19.1, 390.18, 9.16
+20.5, 0.19657, 22.0, 5.86, 0.0, 0.431, 6.226, 79.2, 8.0555, 7.0, 330.0, 19.1, 376.14, 10.15
+24.5, 0.16439, 22.0, 5.86, 0.0, 0.431, 6.433, 49.1, 7.8265, 7.0, 330.0, 19.1, 374.71, 9.52
+26.2, 0.19073, 22.0, 5.86, 0.0, 0.431, 6.718, 17.5, 7.8265, 7.0, 330.0, 19.1, 393.74, 6.56
+24.4, 0.1403, 22.0, 5.86, 0.0, 0.431, 6.487, 13.0, 7.3967, 7.0, 330.0, 19.1, 396.28, 5.9
+24.8, 0.21409, 22.0, 5.86, 0.0, 0.431, 6.438, 8.9, 7.3967, 7.0, 330.0, 19.1, 377.07, 3.59
+29.6, 0.08221, 22.0, 5.86, 0.0, 0.431, 6.957, 6.8, 8.9067, 7.0, 330.0, 19.1, 386.09, 3.53
+42.8, 0.36894, 22.0, 5.86, 0.0, 0.431, 8.259, 8.4, 8.9067, 7.0, 330.0, 19.1, 396.9, 3.54
+21.9, 0.04819, 80.0, 3.64, 0.0, 0.392, 6.108, 32.0, 9.2203, 1.0, 315.0, 16.4, 392.89, 6.57
+20.9, 0.03548, 80.0, 3.64, 0.0, 0.392, 5.876, 19.1, 9.2203, 1.0, 315.0, 16.4, 395.18, 9.25
+44.0, 0.01538, 90.0, 3.75, 0.0, 0.394, 7.454, 34.2, 6.3361, 3.0, 244.0, 15.9, 386.34, 3.11
+50.0, 0.61154, 20.0, 3.97, 0.0, 0.647, 8.704, 86.9, 1.801, 5.0, 264.0, 13.0, 389.7, 5.12
+36.0, 0.66351, 20.0, 3.97, 0.0, 0.647, 7.333, 100.0, 1.8946, 5.0, 264.0, 13.0, 383.29, 7.79
+30.1, 0.65665, 20.0, 3.97, 0.0, 0.647, 6.842, 100.0, 2.0107, 5.0, 264.0, 13.0, 391.93, 6.9
+33.8, 0.54011, 20.0, 3.97, 0.0, 0.647, 7.203, 81.8, 2.1121, 5.0, 264.0, 13.0, 392.8, 9.59
+43.1, 0.53412, 20.0, 3.97, 0.0, 0.647, 7.52, 89.4, 2.1398, 5.0, 264.0, 13.0, 388.37, 7.26
+48.8, 0.52014, 20.0, 3.97, 0.0, 0.647, 8.398, 91.5, 2.2885, 5.0, 264.0, 13.0, 386.86, 5.91
+31.0, 0.82526, 20.0, 3.97, 0.0, 0.647, 7.327, 94.5, 2.0788, 5.0, 264.0, 13.0, 393.42, 11.25
+36.5, 0.55007, 20.0, 3.97, 0.0, 0.647, 7.206, 91.6, 1.9301, 5.0, 264.0, 13.0, 387.89, 8.1
+22.8, 0.76162, 20.0, 3.97, 0.0, 0.647, 5.56, 62.8, 1.9865, 5.0, 264.0, 13.0, 392.4, 10.45
+30.7, 0.7857, 20.0, 3.97, 0.0, 0.647, 7.014, 84.6, 2.1329, 5.0, 264.0, 13.0, 384.07, 14.79
+50.0, 0.57834, 20.0, 3.97, 0.0, 0.575, 8.297, 67.0, 2.4216, 5.0, 264.0, 13.0, 384.54, 7.44
+43.5, 0.5405, 20.0, 3.97, 0.0, 0.575, 7.47, 52.6, 2.872, 5.0, 264.0, 13.0, 390.3, 3.16
+20.7, 0.09065, 20.0, 6.96, 1.0, 0.464, 5.92, 61.5, 3.9175, 3.0, 223.0, 18.6, 391.34, 13.65
+21.1, 0.29916, 20.0, 6.96, 0.0, 0.464, 5.856, 42.1, 4.429, 3.0, 223.0, 18.6, 388.65, 13.0
+25.2, 0.16211, 20.0, 6.96, 0.0, 0.464, 6.24, 16.3, 4.429, 3.0, 223.0, 18.6, 396.9, 6.59
+24.4, 0.1146, 20.0, 6.96, 0.0, 0.464, 6.538, 58.7, 3.9175, 3.0, 223.0, 18.6, 394.96, 7.73
+35.2, 0.22188, 20.0, 6.96, 1.0, 0.464, 7.691, 51.8, 4.3665, 3.0, 223.0, 18.6, 390.77, 6.58
+32.4, 0.05644, 40.0, 6.41, 1.0, 0.447, 6.758, 32.9, 4.0776, 4.0, 254.0, 17.6, 396.9, 3.53
+32.0, 0.09604, 40.0, 6.41, 0.0, 0.447, 6.854, 42.8, 4.2673, 4.0, 254.0, 17.6, 396.9, 2.98
+33.2, 0.10469, 40.0, 6.41, 1.0, 0.447, 7.267, 49.0, 4.7872, 4.0, 254.0, 17.6, 389.25, 6.05
+33.1, 0.06127, 40.0, 6.41, 1.0, 0.447, 6.826, 27.6, 4.8628, 4.0, 254.0, 17.6, 393.45, 4.16
+29.1, 0.07978, 40.0, 6.41, 0.0, 0.447, 6.482, 32.1, 4.1403, 4.0, 254.0, 17.6, 396.9, 7.19
+35.1, 0.21038, 20.0, 3.33, 0.0, 0.4429, 6.812, 32.2, 4.1007, 5.0, 216.0, 14.9, 396.9, 4.85
+45.4, 0.03578, 20.0, 3.33, 0.0, 0.4429, 7.82, 64.5, 4.6947, 5.0, 216.0, 14.9, 387.31, 3.76
+35.4, 0.03705, 20.0, 3.33, 0.0, 0.4429, 6.968, 37.2, 5.2447, 5.0, 216.0, 14.9, 392.23, 4.59
+46.0, 0.06129, 20.0, 3.33, 1.0, 0.4429, 7.645, 49.7, 5.2119, 5.0, 216.0, 14.9, 377.07, 3.01
+50.0, 0.01501, 90.0, 1.21, 1.0, 0.401, 7.923, 24.8, 5.885, 1.0, 198.0, 13.6, 395.52, 3.16
+32.2, 0.00906, 90.0, 2.97, 0.0, 0.4, 7.088, 20.8, 7.3073, 1.0, 285.0, 15.3, 394.72, 7.85
+22.0, 0.01096, 55.0, 2.25, 0.0, 0.389, 6.453, 31.9, 7.3073, 1.0, 300.0, 15.3, 394.72, 8.23
+20.1, 0.01965, 80.0, 1.76, 0.0, 0.385, 6.23, 31.5, 9.0892, 1.0, 241.0, 18.2, 341.6, 12.93
+23.2, 0.03871, 52.5, 5.32, 0.0, 0.405, 6.209, 31.3, 7.3172, 6.0, 293.0, 16.6, 396.9, 7.14
+22.3, 0.0459, 52.5, 5.32, 0.0, 0.405, 6.315, 45.6, 7.3172, 6.0, 293.0, 16.6, 396.9, 7.6
+24.8, 0.04297, 52.5, 5.32, 0.0, 0.405, 6.565, 22.9, 7.3172, 6.0, 293.0, 16.6, 371.72, 9.51
+28.5, 0.03502, 80.0, 4.95, 0.0, 0.411, 6.861, 27.9, 5.1167, 4.0, 245.0, 19.2, 396.9, 3.33
+37.3, 0.07886, 80.0, 4.95, 0.0, 0.411, 7.148, 27.7, 5.1167, 4.0, 245.0, 19.2, 396.9, 3.56
+27.9, 0.03615, 80.0, 4.95, 0.0, 0.411, 6.63, 23.4, 5.1167, 4.0, 245.0, 19.2, 396.9, 4.7
+23.9, 0.08265, 0.0, 13.92, 0.0, 0.437, 6.127, 18.4, 5.5027, 4.0, 289.0, 16.0, 396.9, 8.58
+21.7, 0.08199, 0.0, 13.92, 0.0, 0.437, 6.009, 42.3, 5.5027, 4.0, 289.0, 16.0, 396.9, 10.4
+28.6, 0.12932, 0.0, 13.92, 0.0, 0.437, 6.678, 31.1, 5.9604, 4.0, 289.0, 16.0, 396.9, 6.27
+27.1, 0.05372, 0.0, 13.92, 0.0, 0.437, 6.549, 51.0, 5.9604, 4.0, 289.0, 16.0, 392.85, 7.39
+20.3, 0.14103, 0.0, 13.92, 0.0, 0.437, 5.79, 58.0, 6.32, 4.0, 289.0, 16.0, 396.9, 15.84
+22.5, 0.06466, 70.0, 2.24, 0.0, 0.4, 6.345, 20.1, 7.8278, 5.0, 358.0, 14.8, 368.24, 4.97
+29.0, 0.05561, 70.0, 2.24, 0.0, 0.4, 7.041, 10.0, 7.8278, 5.0, 358.0, 14.8, 371.58, 4.74
+24.8, 0.04417, 70.0, 2.24, 0.0, 0.4, 6.871, 47.4, 7.8278, 5.0, 358.0, 14.8, 390.86, 6.07
+22.0, 0.03537, 34.0, 6.09, 0.0, 0.433, 6.59, 40.4, 5.4917, 7.0, 329.0, 16.1, 395.75, 9.5
+26.4, 0.09266, 34.0, 6.09, 0.0, 0.433, 6.495, 18.4, 5.4917, 7.0, 329.0, 16.1, 383.61, 8.67
+33.1, 0.1, 34.0, 6.09, 0.0, 0.433, 6.982, 17.7, 5.4917, 7.0, 329.0, 16.1, 390.43, 4.86
+36.1, 0.05515, 33.0, 2.18, 0.0, 0.472, 7.236, 41.1, 4.022, 7.0, 222.0, 18.4, 393.68, 6.93
+28.4, 0.05479, 33.0, 2.18, 0.0, 0.472, 6.616, 58.1, 3.37, 7.0, 222.0, 18.4, 393.36, 8.93
+33.4, 0.07503, 33.0, 2.18, 0.0, 0.472, 7.42, 71.9, 3.0992, 7.0, 222.0, 18.4, 396.9, 6.47
+28.2, 0.04932, 33.0, 2.18, 0.0, 0.472, 6.849, 70.3, 3.1827, 7.0, 222.0, 18.4, 396.9, 7.53
+22.8, 0.49298, 0.0, 9.9, 0.0, 0.544, 6.635, 82.5, 3.3175, 4.0, 304.0, 18.4, 396.9, 4.54
+20.3, 0.3494, 0.0, 9.9, 0.0, 0.544, 5.972, 76.7, 3.1025, 4.0, 304.0, 18.4, 396.24, 9.97
+16.1, 2.63548, 0.0, 9.9, 0.0, 0.544, 4.973, 37.8, 2.5194, 4.0, 304.0, 18.4, 350.45, 12.64
+22.1, 0.79041, 0.0, 9.9, 0.0, 0.544, 6.122, 52.8, 2.6403, 4.0, 304.0, 18.4, 396.9, 5.98
+19.4, 0.26169, 0.0, 9.9, 0.0, 0.544, 6.023, 90.4, 2.834, 4.0, 304.0, 18.4, 396.3, 11.72
+21.6, 0.26938, 0.0, 9.9, 0.0, 0.544, 6.266, 82.8, 3.2628, 4.0, 304.0, 18.4, 393.39, 7.9
+23.8, 0.3692, 0.0, 9.9, 0.0, 0.544, 6.567, 87.3, 3.6023, 4.0, 304.0, 18.4, 395.69, 9.28
+16.2, 0.25356, 0.0, 9.9, 0.0, 0.544, 5.705, 77.7, 3.945, 4.0, 304.0, 18.4, 396.42, 11.5
+17.8, 0.31827, 0.0, 9.9, 0.0, 0.544, 5.914, 83.2, 3.9986, 4.0, 304.0, 18.4, 390.7, 18.33
+19.8, 0.24522, 0.0, 9.9, 0.0, 0.544, 5.782, 71.7, 4.0317, 4.0, 304.0, 18.4, 396.9, 15.94
+23.1, 0.40202, 0.0, 9.9, 0.0, 0.544, 6.382, 67.2, 3.5325, 4.0, 304.0, 18.4, 395.21, 10.36
+21.0, 0.47547, 0.0, 9.9, 0.0, 0.544, 6.113, 58.8, 4.0019, 4.0, 304.0, 18.4, 396.23, 12.73
+23.8, 0.1676, 0.0, 7.38, 0.0, 0.493, 6.426, 52.3, 4.5404, 5.0, 287.0, 19.6, 396.9, 7.2
+23.1, 0.18159, 0.0, 7.38, 0.0, 0.493, 6.376, 54.3, 4.5404, 5.0, 287.0, 19.6, 396.9, 6.87
+20.4, 0.35114, 0.0, 7.38, 0.0, 0.493, 6.041, 49.9, 4.7211, 5.0, 287.0, 19.6, 396.9, 7.7
+18.5, 0.28392, 0.0, 7.38, 0.0, 0.493, 5.708, 74.3, 4.7211, 5.0, 287.0, 19.6, 391.13, 11.74
+25.0, 0.34109, 0.0, 7.38, 0.0, 0.493, 6.415, 40.1, 4.7211, 5.0, 287.0, 19.6, 396.9, 6.12
+24.6, 0.19186, 0.0, 7.38, 0.0, 0.493, 6.431, 14.7, 5.4159, 5.0, 287.0, 19.6, 393.68, 5.08
+23.0, 0.30347, 0.0, 7.38, 0.0, 0.493, 6.312, 28.9, 5.4159, 5.0, 287.0, 19.6, 396.9, 6.15
+22.2, 0.24103, 0.0, 7.38, 0.0, 0.493, 6.083, 43.7, 5.4159, 5.0, 287.0, 19.6, 396.9, 12.79
+19.3, 0.06617, 0.0, 3.24, 0.0, 0.46, 5.868, 25.8, 5.2146, 4.0, 430.0, 16.9, 382.44, 9.97
+22.6, 0.06724, 0.0, 3.24, 0.0, 0.46, 6.333, 17.2, 5.2146, 4.0, 430.0, 16.9, 375.21, 7.34
+19.8, 0.04544, 0.0, 3.24, 0.0, 0.46, 6.144, 32.2, 5.8736, 4.0, 430.0, 16.9, 368.57, 9.09
+17.1, 0.05023, 35.0, 6.06, 0.0, 0.4379, 5.706, 28.4, 6.6407, 1.0, 304.0, 16.9, 394.02, 12.43
+19.4, 0.03466, 35.0, 6.06, 0.0, 0.4379, 6.031, 23.3, 6.6407, 1.0, 304.0, 16.9, 362.25, 7.83
+22.2, 0.05083, 0.0, 5.19, 0.0, 0.515, 6.316, 38.1, 6.4584, 5.0, 224.0, 20.2, 389.71, 5.68
+20.7, 0.03738, 0.0, 5.19, 0.0, 0.515, 6.31, 38.5, 6.4584, 5.0, 224.0, 20.2, 389.4, 6.75
+21.1, 0.03961, 0.0, 5.19, 0.0, 0.515, 6.037, 34.5, 5.9853, 5.0, 224.0, 20.2, 396.9, 8.01
+19.5, 0.03427, 0.0, 5.19, 0.0, 0.515, 5.869, 46.3, 5.2311, 5.0, 224.0, 20.2, 396.9, 9.8
+18.5, 0.03041, 0.0, 5.19, 0.0, 0.515, 5.895, 59.6, 5.615, 5.0, 224.0, 20.2, 394.81, 10.56
+20.6, 0.03306, 0.0, 5.19, 0.0, 0.515, 6.059, 37.3, 4.8122, 5.0, 224.0, 20.2, 396.14, 8.51
+19.0, 0.05497, 0.0, 5.19, 0.0, 0.515, 5.985, 45.4, 4.8122, 5.0, 224.0, 20.2, 396.9, 9.74
+18.7, 0.06151, 0.0, 5.19, 0.0, 0.515, 5.968, 58.5, 4.8122, 5.0, 224.0, 20.2, 396.9, 9.29
+32.7, 0.01301, 35.0, 1.52, 0.0, 0.442, 7.241, 49.3, 7.0379, 1.0, 284.0, 15.5, 394.74, 5.49
+16.5, 0.02498, 0.0, 1.89, 0.0, 0.518, 6.54, 59.7, 6.2669, 1.0, 422.0, 15.9, 389.96, 8.65
+23.9, 0.02543, 55.0, 3.78, 0.0, 0.484, 6.696, 56.4, 5.7321, 5.0, 370.0, 17.6, 396.9, 7.18
+31.2, 0.03049, 55.0, 3.78, 0.0, 0.484, 6.874, 28.1, 6.4654, 5.0, 370.0, 17.6, 387.97, 4.61
+17.5, 0.03113, 0.0, 4.39, 0.0, 0.442, 6.014, 48.5, 8.0136, 3.0, 352.0, 18.8, 385.64, 10.53
+17.2, 0.06162, 0.0, 4.39, 0.0, 0.442, 5.898, 52.3, 8.0136, 3.0, 352.0, 18.8, 364.61, 12.67
+23.1, 0.0187, 85.0, 4.15, 0.0, 0.429, 6.516, 27.7, 8.5353, 4.0, 351.0, 17.9, 392.43, 6.36
+24.5, 0.01501, 80.0, 2.01, 0.0, 0.435, 6.635, 29.7, 8.344, 4.0, 280.0, 17.0, 390.94, 5.99
+26.6, 0.02899, 40.0, 1.25, 0.0, 0.429, 6.939, 34.5, 8.7921, 1.0, 335.0, 19.7, 389.85, 5.89
+22.9, 0.06211, 40.0, 1.25, 0.0, 0.429, 6.49, 44.4, 8.7921, 1.0, 335.0, 19.7, 396.9, 5.98
+24.1, 0.0795, 60.0, 1.69, 0.0, 0.411, 6.579, 35.9, 10.7103, 4.0, 411.0, 18.3, 370.78, 5.49
+18.6, 0.07244, 60.0, 1.69, 0.0, 0.411, 5.884, 18.5, 10.7103, 4.0, 411.0, 18.3, 392.33, 7.79
+30.1, 0.01709, 90.0, 2.02, 0.0, 0.41, 6.728, 36.1, 12.1265, 5.0, 187.0, 17.0, 384.46, 4.5
+18.2, 0.04301, 80.0, 1.91, 0.0, 0.413, 5.663, 21.9, 10.5857, 4.0, 334.0, 22.0, 382.8, 8.05
+20.6, 0.10659, 80.0, 1.91, 0.0, 0.413, 5.936, 19.5, 10.5857, 4.0, 334.0, 22.0, 376.04, 5.57
+17.8, 8.98296, 0.0, 18.1, 1.0, 0.77, 6.212, 97.4, 2.1222, 24.0, 666.0, 20.2, 377.73, 17.6
+21.7, 3.8497, 0.0, 18.1, 1.0, 0.77, 6.395, 91.0, 2.5052, 24.0, 666.0, 20.2, 391.34, 13.27
+22.7, 5.20177, 0.0, 18.1, 1.0, 0.77, 6.127, 83.4, 2.7227, 24.0, 666.0, 20.2, 395.43, 11.48
+22.6, 4.26131, 0.0, 18.1, 0.0, 0.77, 6.112, 81.3, 2.5091, 24.0, 666.0, 20.2, 390.74, 12.67
+25.0, 4.54192, 0.0, 18.1, 0.0, 0.77, 6.398, 88.0, 2.5182, 24.0, 666.0, 20.2, 374.56, 7.79
+19.9, 3.83684, 0.0, 18.1, 0.0, 0.77, 6.251, 91.1, 2.2955, 24.0, 666.0, 20.2, 350.65, 14.19
+20.8, 3.67822, 0.0, 18.1, 0.0, 0.77, 5.362, 96.2, 2.1036, 24.0, 666.0, 20.2, 380.79, 10.19
+16.8, 4.22239, 0.0, 18.1, 1.0, 0.77, 5.803, 89.0, 1.9047, 24.0, 666.0, 20.2, 353.04, 14.64
+21.9, 3.47428, 0.0, 18.1, 1.0, 0.718, 8.78, 82.9, 1.9047, 24.0, 666.0, 20.2, 354.55, 5.29
+27.5, 4.55587, 0.0, 18.1, 0.0, 0.718, 3.561, 87.9, 1.6132, 24.0, 666.0, 20.2, 354.7, 7.12
+21.9, 3.69695, 0.0, 18.1, 0.0, 0.718, 4.963, 91.4, 1.7523, 24.0, 666.0, 20.2, 316.03, 14.0
+23.1, 13.5222, 0.0, 18.1, 0.0, 0.631, 3.863, 100.0, 1.5106, 24.0, 666.0, 20.2, 131.42, 13.33
+50.0, 4.89822, 0.0, 18.1, 0.0, 0.631, 4.97, 100.0, 1.3325, 24.0, 666.0, 20.2, 375.52, 3.26
+50.0, 5.66998, 0.0, 18.1, 1.0, 0.631, 6.683, 96.8, 1.3567, 24.0, 666.0, 20.2, 375.33, 3.73
+50.0, 6.53876, 0.0, 18.1, 1.0, 0.631, 7.016, 97.5, 1.2024, 24.0, 666.0, 20.2, 392.05, 2.96
+50.0, 9.2323, 0.0, 18.1, 0.0, 0.631, 6.216, 100.0, 1.1691, 24.0, 666.0, 20.2, 366.15, 9.53
+50.0, 8.26725, 0.0, 18.1, 1.0, 0.668, 5.875, 89.6, 1.1296, 24.0, 666.0, 20.2, 347.88, 8.88
+13.8, 11.1081, 0.0, 18.1, 0.0, 0.668, 4.906, 100.0, 1.1742, 24.0, 666.0, 20.2, 396.9, 34.77
+13.8, 18.4982, 0.0, 18.1, 0.0, 0.668, 4.138, 100.0, 1.137, 24.0, 666.0, 20.2, 396.9, 37.97
+15.0, 19.6091, 0.0, 18.1, 0.0, 0.671, 7.313, 97.9, 1.3163, 24.0, 666.0, 20.2, 396.9, 13.44
+13.9, 15.288, 0.0, 18.1, 0.0, 0.671, 6.649, 93.3, 1.3449, 24.0, 666.0, 20.2, 363.02, 23.24
+13.3, 9.82349, 0.0, 18.1, 0.0, 0.671, 6.794, 98.8, 1.358, 24.0, 666.0, 20.2, 396.9, 21.24
+13.1, 23.6482, 0.0, 18.1, 0.0, 0.671, 6.38, 96.2, 1.3861, 24.0, 666.0, 20.2, 396.9, 23.69
+10.2, 17.8667, 0.0, 18.1, 0.0, 0.671, 6.223, 100.0, 1.3861, 24.0, 666.0, 20.2, 393.74, 21.78
+10.4, 88.9762, 0.0, 18.1, 0.0, 0.671, 6.968, 91.9, 1.4165, 24.0, 666.0, 20.2, 396.9, 17.21
+10.9, 15.8744, 0.0, 18.1, 0.0, 0.671, 6.545, 99.1, 1.5192, 24.0, 666.0, 20.2, 396.9, 21.08
+11.3, 9.18702, 0.0, 18.1, 0.0, 0.7, 5.536, 100.0, 1.5804, 24.0, 666.0, 20.2, 396.9, 23.6
+12.3, 7.99248, 0.0, 18.1, 0.0, 0.7, 5.52, 100.0, 1.5331, 24.0, 666.0, 20.2, 396.9, 24.56
+8.8, 20.0849, 0.0, 18.1, 0.0, 0.7, 4.368, 91.2, 1.4395, 24.0, 666.0, 20.2, 285.83, 30.63
+7.2, 16.8118, 0.0, 18.1, 0.0, 0.7, 5.277, 98.1, 1.4261, 24.0, 666.0, 20.2, 396.9, 30.81
+10.5, 24.3938, 0.0, 18.1, 0.0, 0.7, 4.652, 100.0, 1.4672, 24.0, 666.0, 20.2, 396.9, 28.28
+7.4, 22.5971, 0.0, 18.1, 0.0, 0.7, 5.0, 89.5, 1.5184, 24.0, 666.0, 20.2, 396.9, 31.99
+10.2, 14.3337, 0.0, 18.1, 0.0, 0.7, 4.88, 100.0, 1.5895, 24.0, 666.0, 20.2, 372.92, 30.62
+11.5, 8.15174, 0.0, 18.1, 0.0, 0.7, 5.39, 98.9, 1.7281, 24.0, 666.0, 20.2, 396.9, 20.85
+15.1, 6.96215, 0.0, 18.1, 0.0, 0.7, 5.713, 97.0, 1.9265, 24.0, 666.0, 20.2, 394.43, 17.11
+23.2, 5.29305, 0.0, 18.1, 0.0, 0.7, 6.051, 82.5, 2.1678, 24.0, 666.0, 20.2, 378.38, 18.76
+9.7, 11.5779, 0.0, 18.1, 0.0, 0.7, 5.036, 97.0, 1.77, 24.0, 666.0, 20.2, 396.9, 25.68
+13.8, 8.64476, 0.0, 18.1, 0.0, 0.693, 6.193, 92.6, 1.7912, 24.0, 666.0, 20.2, 396.9, 15.17
+12.7, 13.3598, 0.0, 18.1, 0.0, 0.693, 5.887, 94.7, 1.7821, 24.0, 666.0, 20.2, 396.9, 16.35
+13.1, 8.71675, 0.0, 18.1, 0.0, 0.693, 6.471, 98.8, 1.7257, 24.0, 666.0, 20.2, 391.98, 17.12
+12.5, 5.87205, 0.0, 18.1, 0.0, 0.693, 6.405, 96.0, 1.6768, 24.0, 666.0, 20.2, 396.9, 19.37
+8.5, 7.67202, 0.0, 18.1, 0.0, 0.693, 5.747, 98.9, 1.6334, 24.0, 666.0, 20.2, 393.1, 19.92
+5.0, 38.3518, 0.0, 18.1, 0.0, 0.693, 5.453, 100.0, 1.4896, 24.0, 666.0, 20.2, 396.9, 30.59
+6.3, 9.91655, 0.0, 18.1, 0.0, 0.693, 5.852, 77.8, 1.5004, 24.0, 666.0, 20.2, 338.16, 29.97
+5.6, 25.0461, 0.0, 18.1, 0.0, 0.693, 5.987, 100.0, 1.5888, 24.0, 666.0, 20.2, 396.9, 26.77
+7.2, 14.2362, 0.0, 18.1, 0.0, 0.693, 6.343, 100.0, 1.5741, 24.0, 666.0, 20.2, 396.9, 20.32
+12.1, 9.59571, 0.0, 18.1, 0.0, 0.693, 6.404, 100.0, 1.639, 24.0, 666.0, 20.2, 376.11, 20.31
+8.3, 24.8017, 0.0, 18.1, 0.0, 0.693, 5.349, 96.0, 1.7028, 24.0, 666.0, 20.2, 396.9, 19.77
+8.5, 41.5292, 0.0, 18.1, 0.0, 0.693, 5.531, 85.4, 1.6074, 24.0, 666.0, 20.2, 329.46, 27.38
+5.0, 67.9208, 0.0, 18.1, 0.0, 0.693, 5.683, 100.0, 1.4254, 24.0, 666.0, 20.2, 384.97, 22.98
+11.9, 20.7162, 0.0, 18.1, 0.0, 0.659, 4.138, 100.0, 1.1781, 24.0, 666.0, 20.2, 370.22, 23.34
+27.9, 11.9511, 0.0, 18.1, 0.0, 0.659, 5.608, 100.0, 1.2852, 24.0, 666.0, 20.2, 332.09, 12.13
+17.2, 7.40389, 0.0, 18.1, 0.0, 0.597, 5.617, 97.9, 1.4547, 24.0, 666.0, 20.2, 314.64, 26.4
+27.5, 14.4383, 0.0, 18.1, 0.0, 0.597, 6.852, 100.0, 1.4655, 24.0, 666.0, 20.2, 179.36, 19.78
+15.0, 51.1358, 0.0, 18.1, 0.0, 0.597, 5.757, 100.0, 1.413, 24.0, 666.0, 20.2, 2.6, 10.11
+17.2, 14.0507, 0.0, 18.1, 0.0, 0.597, 6.657, 100.0, 1.5275, 24.0, 666.0, 20.2, 35.05, 21.22
+17.9, 18.811, 0.0, 18.1, 0.0, 0.597, 4.628, 100.0, 1.5539, 24.0, 666.0, 20.2, 28.79, 34.37
+16.3, 28.6558, 0.0, 18.1, 0.0, 0.597, 5.155, 100.0, 1.5894, 24.0, 666.0, 20.2, 210.97, 20.08
+7.0, 45.7461, 0.0, 18.1, 0.0, 0.693, 4.519, 100.0, 1.6582, 24.0, 666.0, 20.2, 88.27, 36.98
+7.2, 18.0846, 0.0, 18.1, 0.0, 0.679, 6.434, 100.0, 1.8347, 24.0, 666.0, 20.2, 27.25, 29.05
+7.5, 10.8342, 0.0, 18.1, 0.0, 0.679, 6.782, 90.8, 1.8195, 24.0, 666.0, 20.2, 21.57, 25.79
+10.4, 25.9406, 0.0, 18.1, 0.0, 0.679, 5.304, 89.1, 1.6475, 24.0, 666.0, 20.2, 127.36, 26.64
+8.8, 73.5341, 0.0, 18.1, 0.0, 0.679, 5.957, 100.0, 1.8026, 24.0, 666.0, 20.2, 16.45, 20.62
+8.4, 11.8123, 0.0, 18.1, 0.0, 0.718, 6.824, 76.5, 1.794, 24.0, 666.0, 20.2, 48.45, 22.74
+16.7, 11.0874, 0.0, 18.1, 0.0, 0.718, 6.411, 100.0, 1.8589, 24.0, 666.0, 20.2, 318.75, 15.02
+14.2, 7.02259, 0.0, 18.1, 0.0, 0.718, 6.006, 95.3, 1.8746, 24.0, 666.0, 20.2, 319.98, 15.7
+20.8, 12.0482, 0.0, 18.1, 0.0, 0.614, 5.648, 87.6, 1.9512, 24.0, 666.0, 20.2, 291.55, 14.1
+13.4, 7.05042, 0.0, 18.1, 0.0, 0.614, 6.103, 85.1, 2.0218, 24.0, 666.0, 20.2, 2.52, 23.29
+11.7, 8.79212, 0.0, 18.1, 0.0, 0.584, 5.565, 70.6, 2.0635, 24.0, 666.0, 20.2, 3.65, 17.16
+8.3, 15.8603, 0.0, 18.1, 0.0, 0.679, 5.896, 95.4, 1.9096, 24.0, 666.0, 20.2, 7.68, 24.39
+10.2, 12.2472, 0.0, 18.1, 0.0, 0.584, 5.837, 59.7, 1.9976, 24.0, 666.0, 20.2, 24.65, 15.69
+10.9, 37.6619, 0.0, 18.1, 0.0, 0.679, 6.202, 78.7, 1.8629, 24.0, 666.0, 20.2, 18.82, 14.52
+11.0, 7.36711, 0.0, 18.1, 0.0, 0.679, 6.193, 78.1, 1.9356, 24.0, 666.0, 20.2, 96.73, 21.52
+9.5, 9.33889, 0.0, 18.1, 0.0, 0.679, 6.38, 95.6, 1.9682, 24.0, 666.0, 20.2, 60.72, 24.08
+14.5, 8.49213, 0.0, 18.1, 0.0, 0.584, 6.348, 86.1, 2.0527, 24.0, 666.0, 20.2, 83.45, 17.64
+14.1, 10.0623, 0.0, 18.1, 0.0, 0.584, 6.833, 94.3, 2.0882, 24.0, 666.0, 20.2, 81.33, 19.69
+16.1, 6.44405, 0.0, 18.1, 0.0, 0.584, 6.425, 74.8, 2.2004, 24.0, 666.0, 20.2, 97.95, 12.03
+14.3, 5.58107, 0.0, 18.1, 0.0, 0.713, 6.436, 87.9, 2.3158, 24.0, 666.0, 20.2, 100.19, 16.22
+11.7, 13.9134, 0.0, 18.1, 0.0, 0.713, 6.208, 95.0, 2.2222, 24.0, 666.0, 20.2, 100.63, 15.17
+13.4, 11.1604, 0.0, 18.1, 0.0, 0.74, 6.629, 94.6, 2.1247, 24.0, 666.0, 20.2, 109.85, 23.27
+9.6, 14.4208, 0.0, 18.1, 0.0, 0.74, 6.461, 93.3, 2.0026, 24.0, 666.0, 20.2, 27.49, 18.05
+8.7, 15.1772, 0.0, 18.1, 0.0, 0.74, 6.152, 100.0, 1.9142, 24.0, 666.0, 20.2, 9.32, 26.45
+8.4, 13.6781, 0.0, 18.1, 0.0, 0.74, 5.935, 87.9, 1.8206, 24.0, 666.0, 20.2, 68.95, 34.02
+12.8, 9.39063, 0.0, 18.1, 0.0, 0.74, 5.627, 93.9, 1.8172, 24.0, 666.0, 20.2, 396.9, 22.88
+10.5, 22.0511, 0.0, 18.1, 0.0, 0.74, 5.818, 92.4, 1.8662, 24.0, 666.0, 20.2, 391.45, 22.11
+17.1, 9.72418, 0.0, 18.1, 0.0, 0.74, 6.406, 97.2, 2.0651, 24.0, 666.0, 20.2, 385.96, 19.52
+18.4, 5.66637, 0.0, 18.1, 0.0, 0.74, 6.219, 100.0, 2.0048, 24.0, 666.0, 20.2, 395.69, 16.59
+15.4, 9.96654, 0.0, 18.1, 0.0, 0.74, 6.485, 100.0, 1.9784, 24.0, 666.0, 20.2, 386.73, 18.85
+10.8, 12.8023, 0.0, 18.1, 0.0, 0.74, 5.854, 96.6, 1.8956, 24.0, 666.0, 20.2, 240.52, 23.79
+11.8, 0.6718, 0.0, 18.1, 0.0, 0.74, 6.459, 94.8, 1.9879, 24.0, 666.0, 20.2, 43.06, 23.98
+14.9, 6.28807, 0.0, 18.1, 0.0, 0.74, 6.341, 96.4, 2.072, 24.0, 666.0, 20.2, 318.01, 17.79
+12.6, 9.92485, 0.0, 18.1, 0.0, 0.74, 6.251, 96.6, 2.198, 24.0, 666.0, 20.2, 388.52, 16.44
+14.1, 9.32909, 0.0, 18.1, 0.0, 0.713, 6.185, 98.7, 2.2616, 24.0, 666.0, 20.2, 396.9, 18.13
+13.0, 7.52601, 0.0, 18.1, 0.0, 0.713, 6.417, 98.3, 2.185, 24.0, 666.0, 20.2, 304.21, 19.31
+13.4, 6.71772, 0.0, 18.1, 0.0, 0.713, 6.749, 92.6, 2.3236, 24.0, 666.0, 20.2, 0.32, 17.44
+15.2, 5.44114, 0.0, 18.1, 0.0, 0.713, 6.655, 98.2, 2.3552, 24.0, 666.0, 20.2, 355.29, 17.73
+16.1, 5.09017, 0.0, 18.1, 0.0, 0.713, 6.297, 91.8, 2.3682, 24.0, 666.0, 20.2, 385.09, 17.27
+17.8, 8.24809, 0.0, 18.1, 0.0, 0.713, 7.393, 99.3, 2.4527, 24.0, 666.0, 20.2, 375.87, 16.74
+14.9, 9.51363, 0.0, 18.1, 0.0, 0.713, 6.728, 94.1, 2.4961, 24.0, 666.0, 20.2, 6.68, 18.71
+14.1, 4.75237, 0.0, 18.1, 0.0, 0.713, 6.525, 86.5, 2.4358, 24.0, 666.0, 20.2, 50.92, 18.13
+12.7, 4.66883, 0.0, 18.1, 0.0, 0.713, 5.976, 87.9, 2.5806, 24.0, 666.0, 20.2, 10.48, 19.01
+13.5, 8.20058, 0.0, 18.1, 0.0, 0.713, 5.936, 80.3, 2.7792, 24.0, 666.0, 20.2, 3.5, 16.94
+14.9, 7.75223, 0.0, 18.1, 0.0, 0.713, 6.301, 83.7, 2.7831, 24.0, 666.0, 20.2, 272.21, 16.23
+20.0, 6.80117, 0.0, 18.1, 0.0, 0.713, 6.081, 84.4, 2.7175, 24.0, 666.0, 20.2, 396.9, 14.7
+16.4, 4.81213, 0.0, 18.1, 0.0, 0.713, 6.701, 90.0, 2.5975, 24.0, 666.0, 20.2, 255.23, 16.42
+17.7, 3.69311, 0.0, 18.1, 0.0, 0.713, 6.376, 88.4, 2.5671, 24.0, 666.0, 20.2, 391.43, 14.65
+19.5, 6.65492, 0.0, 18.1, 0.0, 0.713, 6.317, 83.0, 2.7344, 24.0, 666.0, 20.2, 396.9, 13.99
+20.2, 5.82115, 0.0, 18.1, 0.0, 0.713, 6.513, 89.9, 2.8016, 24.0, 666.0, 20.2, 393.82, 10.29
+21.4, 7.83932, 0.0, 18.1, 0.0, 0.655, 6.209, 65.4, 2.9634, 24.0, 666.0, 20.2, 396.9, 13.22
+19.9, 3.1636, 0.0, 18.1, 0.0, 0.655, 5.759, 48.2, 3.0665, 24.0, 666.0, 20.2, 334.4, 14.13
+19.0, 3.77498, 0.0, 18.1, 0.0, 0.655, 5.952, 84.7, 2.8715, 24.0, 666.0, 20.2, 22.01, 17.15
+19.1, 4.42228, 0.0, 18.1, 0.0, 0.584, 6.003, 94.5, 2.5403, 24.0, 666.0, 20.2, 331.29, 21.32
+19.1, 15.5757, 0.0, 18.1, 0.0, 0.58, 5.926, 71.0, 2.9084, 24.0, 666.0, 20.2, 368.74, 18.13
+20.1, 13.0751, 0.0, 18.1, 0.0, 0.58, 5.713, 56.7, 2.8237, 24.0, 666.0, 20.2, 396.9, 14.76
+19.9, 4.34879, 0.0, 18.1, 0.0, 0.58, 6.167, 84.0, 3.0334, 24.0, 666.0, 20.2, 396.9, 16.29
+19.6, 4.03841, 0.0, 18.1, 0.0, 0.532, 6.229, 90.7, 3.0993, 24.0, 666.0, 20.2, 395.33, 12.87
+23.2, 3.56868, 0.0, 18.1, 0.0, 0.58, 6.437, 75.0, 2.8965, 24.0, 666.0, 20.2, 393.37, 14.36
+29.8, 4.64689, 0.0, 18.1, 0.0, 0.614, 6.98, 67.6, 2.5329, 24.0, 666.0, 20.2, 374.68, 11.66
+13.8, 8.05579, 0.0, 18.1, 0.0, 0.584, 5.427, 95.4, 2.4298, 24.0, 666.0, 20.2, 352.58, 18.14
+13.3, 6.39312, 0.0, 18.1, 0.0, 0.584, 6.162, 97.4, 2.206, 24.0, 666.0, 20.2, 302.76, 24.1
+16.7, 4.87141, 0.0, 18.1, 0.0, 0.614, 6.484, 93.6, 2.3053, 24.0, 666.0, 20.2, 396.21, 18.68
+12.0, 15.0234, 0.0, 18.1, 0.0, 0.614, 5.304, 97.3, 2.1007, 24.0, 666.0, 20.2, 349.48, 24.91
+14.6, 10.233, 0.0, 18.1, 0.0, 0.614, 6.185, 96.7, 2.1705, 24.0, 666.0, 20.2, 379.7, 18.03
+21.4, 14.3337, 0.0, 18.1, 0.0, 0.614, 6.229, 88.0, 1.9512, 24.0, 666.0, 20.2, 383.32, 13.11
+23.0, 5.82401, 0.0, 18.1, 0.0, 0.532, 6.242, 64.7, 3.4242, 24.0, 666.0, 20.2, 396.9, 10.74
+23.7, 5.70818, 0.0, 18.1, 0.0, 0.532, 6.75, 74.9, 3.3317, 24.0, 666.0, 20.2, 393.07, 7.74
+25.0, 5.73116, 0.0, 18.1, 0.0, 0.532, 7.061, 77.0, 3.4106, 24.0, 666.0, 20.2, 395.28, 7.01
+21.8, 2.81838, 0.0, 18.1, 0.0, 0.532, 5.762, 40.3, 4.0983, 24.0, 666.0, 20.2, 392.92, 10.42
+20.6, 2.37857, 0.0, 18.1, 0.0, 0.583, 5.871, 41.9, 3.724, 24.0, 666.0, 20.2, 370.73, 13.34
+21.2, 3.67367, 0.0, 18.1, 0.0, 0.583, 6.312, 51.9, 3.9917, 24.0, 666.0, 20.2, 388.62, 10.58
+19.1, 5.69175, 0.0, 18.1, 0.0, 0.583, 6.114, 79.8, 3.5459, 24.0, 666.0, 20.2, 392.68, 14.98
+20.6, 4.83567, 0.0, 18.1, 0.0, 0.583, 5.905, 53.2, 3.1523, 24.0, 666.0, 20.2, 388.22, 11.45
+15.2, 0.15086, 0.0, 27.74, 0.0, 0.609, 5.454, 92.7, 1.8209, 4.0, 711.0, 20.1, 395.09, 18.06
+7.0, 0.18337, 0.0, 27.74, 0.0, 0.609, 5.414, 98.3, 1.7554, 4.0, 711.0, 20.1, 344.05, 23.97
+8.1, 0.20746, 0.0, 27.74, 0.0, 0.609, 5.093, 98.0, 1.8226, 4.0, 711.0, 20.1, 318.43, 29.68
+13.6, 0.10574, 0.0, 27.74, 0.0, 0.609, 5.983, 98.8, 1.8681, 4.0, 711.0, 20.1, 390.11, 18.07
+20.1, 0.11132, 0.0, 27.74, 0.0, 0.609, 5.983, 83.5, 2.1099, 4.0, 711.0, 20.1, 396.9, 13.35
+21.8, 0.17331, 0.0, 9.69, 0.0, 0.585, 5.707, 54.0, 2.3817, 6.0, 391.0, 19.2, 396.9, 12.01
+24.5, 0.27957, 0.0, 9.69, 0.0, 0.585, 5.926, 42.6, 2.3817, 6.0, 391.0, 19.2, 396.9, 13.59
+23.1, 0.17899, 0.0, 9.69, 0.0, 0.585, 5.67, 28.8, 2.7986, 6.0, 391.0, 19.2, 393.29, 17.6
+19.7, 0.2896, 0.0, 9.69, 0.0, 0.585, 5.39, 72.9, 2.7986, 6.0, 391.0, 19.2, 396.9, 21.14
+18.3, 0.26838, 0.0, 9.69, 0.0, 0.585, 5.794, 70.6, 2.8927, 6.0, 391.0, 19.2, 396.9, 14.1
+21.2, 0.23912, 0.0, 9.69, 0.0, 0.585, 6.019, 65.3, 2.4091, 6.0, 391.0, 19.2, 396.9, 12.92
+17.5, 0.17783, 0.0, 9.69, 0.0, 0.585, 5.569, 73.5, 2.3999, 6.0, 391.0, 19.2, 395.77, 15.1
+16.8, 0.22438, 0.0, 9.69, 0.0, 0.585, 6.027, 79.7, 2.4982, 6.0, 391.0, 19.2, 396.9, 14.33
+22.4, 0.06263, 0.0, 11.93, 0.0, 0.573, 6.593, 69.1, 2.4786, 1.0, 273.0, 21.0, 391.99, 9.67
+20.6, 0.04527, 0.0, 11.93, 0.0, 0.573, 6.12, 76.7, 2.2875, 1.0, 273.0, 21.0, 396.9, 9.08
+23.9, 0.06076, 0.0, 11.93, 0.0, 0.573, 6.976, 91.0, 2.1675, 1.0, 273.0, 21.0, 396.9, 5.64
+22.0, 0.10959, 0.0, 11.93, 0.0, 0.573, 6.794, 89.3, 2.3889, 1.0, 273.0, 21.0, 393.45, 6.48
+11.9, 0.04741, 0.0, 11.93, 0.0, 0.573, 6.03, 80.8, 2.505, 1.0, 273.0, 21.0, 396.9, 7.88
\ No newline at end of file


[05/15] ignite git commit: Added cache name in GridDhtPartitionTopologyImpl logging.

Posted by sb...@apache.org.
Added cache name in GridDhtPartitionTopologyImpl logging.


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

Branch: refs/heads/ignite-zk
Commit: c7b4201e845a69a829f2e81ba2bcd4c33e9785eb
Parents: 429f954
Author: sboikov <sb...@gridgain.com>
Authored: Thu Dec 28 17:16:10 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Dec 28 17:18:43 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 168 ++++++++++++-------
 1 file changed, 104 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c7b4201e/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 abe276f..e09d611 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
@@ -362,8 +362,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                             assert owned : "Failed to own partition for oldest node [grp=" + grp.cacheOrGroupName() +
                                 ", part=" + locPart + ']';
 
-                            if (log.isDebugEnabled())
-                                log.debug("Owned partition for oldest node: " + locPart);
+                            if (log.isDebugEnabled()) {
+                                log.debug("Owned partition for oldest node [grp=" + grp.cacheOrGroupName() +
+                                    ", part=" + locPart + ']');
+                            }
 
                             updateSeq = updateLocal(p, locPart.state(), updateSeq, affVer);
                         }
@@ -389,9 +391,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                                 updateSeq = updateLocal(p, locPart.state(), updateSeq, affVer);
 
-                                if (log.isDebugEnabled())
-                                    log.debug("Evicting partition with rebalancing disabled " +
-                                        "(it does not belong to affinity): " + locPart);
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Evicting partition with rebalancing disabled (it does not belong to " +
+                                        "affinity) [grp=" + grp.cacheOrGroupName() + ", part=" + locPart + ']');
+                                }
                             }
                         }
                         else
@@ -476,8 +479,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     ClusterNode oldest = discoCache.oldestAliveServerNode();
 
                     if (log.isDebugEnabled()) {
-                        log.debug("Partition map beforeExchange [exchId=" + exchFut.exchangeId() +
-                            ", fullMap=" + fullMapString() + ']');
+                        log.debug("Partition map beforeExchange [grp=" + grp.cacheOrGroupName() +
+                            ", exchId=" + exchFut.exchangeId() + ", fullMap=" + fullMapString() + ']');
                     }
 
                     long updateSeq = this.updateSeq.incrementAndGet();
@@ -491,9 +494,11 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         if (node2part == null) {
                             node2part = new GridDhtPartitionFullMap(oldest.id(), oldest.order(), updateSeq);
 
-                            if (log.isDebugEnabled())
-                                log.debug("Created brand new full topology map on oldest node [exchId=" +
-                                    exchFut.exchangeId() + ", fullMap=" + fullMapString() + ']');
+                            if (log.isDebugEnabled()) {
+                                log.debug("Created brand new full topology map on oldest node [" +
+                                    "grp=" + grp.cacheOrGroupName() + ", exchId=" + exchFut.exchangeId() +
+                                    ", fullMap=" + fullMapString() + ']');
+                            }
                         }
                         else if (!node2part.valid()) {
                             node2part = new GridDhtPartitionFullMap(oldest.id(),
@@ -503,7 +508,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                 false);
 
                             if (log.isDebugEnabled()) {
-                                log.debug("Created new full topology map on oldest node [exchId=" + exchFut.exchangeId() +
+                                log.debug("Created new full topology map on oldest node [" +
+                                    "grp=" +  grp.cacheOrGroupName() + ", exchId=" + exchFut.exchangeId() +
                                     ", fullMap=" + node2part + ']');
                             }
                         }
@@ -516,7 +522,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                             if (log.isDebugEnabled()) {
                                 log.debug("Copied old map into new map on oldest node (previous oldest node left) [" +
-                                    "exchId=" + exchFut.exchangeId() + ", fullMap=" + fullMapString() + ']');
+                                    "grp=" + grp.cacheOrGroupName() + ", exchId=" + exchFut.exchangeId() +
+                                    ", fullMap=" + fullMapString() + ']');
                             }
                         }
                     }
@@ -560,8 +567,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     }
 
                     if (log.isDebugEnabled()) {
-                        log.debug("Partition map after beforeExchange [exchId=" + exchFut.exchangeId() +
-                            ", fullMap=" + fullMapString() + ']');
+                        log.debug("Partition map after beforeExchange [grp=" + grp.cacheOrGroupName() + ", " +
+                            "exchId=" + exchFut.exchangeId() + ", fullMap=" + fullMapString() + ']');
                     }
                 }
                 finally {
@@ -584,7 +591,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException {
+    @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) {
         boolean changed = false;
 
         int num = grp.affinity().partitions();
@@ -592,7 +599,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         AffinityTopologyVersion topVer = exchFut.context().events().topologyVersion();
 
         assert grp.affinity().lastVersion().equals(topVer) : "Affinity is not initialized " +
-            "[topVer=" + topVer +
+            "[grp=" + grp.cacheOrGroupName() +
+            ", topVer=" + topVer +
             ", affVer=" + grp.affinity().lastVersion() +
             ", fut=" + exchFut + ']';
 
@@ -609,9 +617,11 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 assert readyTopVer.initialized() : readyTopVer;
                 assert lastTopChangeVer.equals(readyTopVer);
 
-                if (log.isDebugEnabled())
-                    log.debug("Partition map before afterExchange [exchId=" + exchFut.exchangeId() + ", fullMap=" +
-                        fullMapString() + ']');
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition map before afterExchange [grp=" + grp.cacheOrGroupName() +
+                        ", exchId=" + exchFut.exchangeId() +
+                        ", fullMap=" + fullMapString() + ']');
+                }
 
                 long updateSeq = this.updateSeq.incrementAndGet();
 
@@ -622,8 +632,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         // This partition will be created during next topology event,
                         // which obviously has not happened at this point.
                         if (locPart == null) {
-                            if (log.isDebugEnabled())
-                                log.debug("Skipping local partition afterExchange (will not create): " + p);
+                            if (log.isDebugEnabled()) {
+                                log.debug("Skipping local partition afterExchange (will not create) [" +
+                                    "grp=" + grp.cacheOrGroupName() + ", p=" + p + ']');
+                            }
 
                             continue;
                         }
@@ -638,8 +650,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                 if (F.isEmpty(owners)) {
                                     boolean owned = locPart.own();
 
-                                    assert owned : "Failed to own partition [grp=" + grp.cacheOrGroupName() + ", locPart=" +
-                                        locPart + ']';
+                                    assert owned : "Failed to own partition [grp=" + grp.cacheOrGroupName() +
+                                        ", locPart=" + locPart + ']';
 
                                     updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
 
@@ -655,12 +667,14 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                             discoEvt.timestamp());
                                     }
 
-                                    if (log.isDebugEnabled())
-                                        log.debug("Owned partition: " + locPart);
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("Owned partition [grp=" + grp.cacheOrGroupName() +
+                                            ", part=" + locPart + ']');
+                                    }
                                 }
                                 else if (log.isDebugEnabled())
-                                    log.debug("Will not own partition (there are owners to rebalance from) [locPart=" +
-                                        locPart + ", owners = " + owners + ']');
+                                    log.debug("Will not own partition (there are owners to rebalance from) [grp=" + grp.cacheOrGroupName() +
+                                        ", locPart=" + locPart + ", owners = " + owners + ']');
                             }
                             else
                                 updateSeq = updateLocal(p, locPart.state(), updateSeq, topVer);
@@ -677,8 +691,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                                 changed = true;
 
-                                if (log.isDebugEnabled())
-                                    log.debug("Evicting moving partition (it does not belong to affinity): " + locPart);
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Evicting moving partition (it does not belong to affinity) [" +
+                                        "grp=" + grp.cacheOrGroupName() + ", part=" + locPart + ']');
+                                }
                             }
                         }
                     }
@@ -834,20 +850,22 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     if (!belongs) {
                         throw new GridDhtInvalidPartitionException(p, "Adding entry to evicted partition " +
                             "(often may be caused by inconsistent 'key.hashCode()' implementation) " +
-                            "[part=" + p + ", topVer=" + topVer + ", this.topVer=" + this.readyTopVer + ']');
+                            "[grp=" + grp.cacheOrGroupName() + ", part=" + p + ", topVer=" + topVer +
+                            ", this.topVer=" + this.readyTopVer + ']');
                     }
                 }
                 else if (loc != null && state == RENTING && !showRenting) {
                     throw new GridDhtInvalidPartitionException(p, "Adding entry to partition that is concurrently " +
-                        "evicted [part=" + p + ", shouldBeMoving=" + loc.reload() + ", belongs=" + belongs +
-                        ", topVer=" + topVer + ", curTopVer=" + this.readyTopVer + "]");
+                        "evicted [grp=" + grp.cacheOrGroupName() + ", part=" + p + ", shouldBeMoving=" +
+                        loc.reload() + ", belongs=" + belongs + ", topVer=" + topVer + ", curTopVer=" + this.readyTopVer + "]");
                 }
 
                 if (loc == null) {
                     if (!belongs)
                         throw new GridDhtInvalidPartitionException(p, "Creating partition which does not belong to " +
                             "local node (often may be caused by inconsistent 'key.hashCode()' implementation) " +
-                            "[part=" + p + ", topVer=" + topVer + ", this.topVer=" + this.readyTopVer + ']');
+                            "[grp=" + grp.cacheOrGroupName() + ", part=" + p + ", topVer=" + topVer +
+                            ", this.topVer=" + this.readyTopVer + ']');
 
                     locParts.set(p, loc = new GridDhtLocalPartition(ctx, grp, p));
 
@@ -857,7 +875,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     created = true;
 
                     if (log.isDebugEnabled())
-                        log.debug("Created local partition: " + loc);
+                        log.debug("Created local partition [grp=" + grp.cacheOrGroupName() + ", part=" + loc + ']');
                 }
             }
             finally {
@@ -1026,7 +1044,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             if (!topVer.equals(diffFromAffinityVer)) {
                 LT.warn(log, "Requested topology version does not match calculated diff, will require full iteration to" +
-                    "calculate mapping [topVer=" + topVer + ", diffVer=" + diffFromAffinityVer + "]");
+                    "calculate mapping [grp=" + grp.cacheOrGroupName() + ", topVer=" + topVer +
+                    ", diffVer=" + diffFromAffinityVer + "]");
 
                 nodes = new ArrayList<>();
 
@@ -1054,7 +1073,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 for (UUID nodeId : diffIds) {
                     if (affIds.contains(nodeId)) {
-                        U.warn(log, "Node from diff " + nodeId + " is affinity node. Skipping it.");
+                        U.warn(log, "Node from diff is affinity node, skipping it [grp=" + grp.cacheOrGroupName() +
+                            ", node=" + nodeId + ']');
 
                         continue;
                     }
@@ -1202,8 +1222,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         @Nullable CachePartitionFullCountersMap incomeCntrMap,
         Set<Integer> partsToReload,
         @Nullable AffinityTopologyVersion msgTopVer) {
-        if (log.isDebugEnabled())
-            log.debug("Updating full partition map [exchVer=" + exchangeVer + ", parts=" + fullMapString() + ']');
+        if (log.isDebugEnabled()) {
+            log.debug("Updating full partition map [grp=" + grp.cacheOrGroupName() + ", exchVer=" + exchangeVer +
+                ", fullMap=" + fullMapString() + ']');
+        }
 
         assert partMap != null;
 
@@ -1239,7 +1261,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     // Ignore if exchange already finished or new exchange started.
                     if (readyTopVer.compareTo(exchangeVer) > 0 || lastTopChangeVer.compareTo(exchangeVer) > 0) {
                         U.warn(log, "Stale exchange id for full partition map update (will ignore) [" +
-                            "lastTopChange=" + lastTopChangeVer +
+                            "grp=" + grp.cacheOrGroupName() +
+                            ", lastTopChange=" + lastTopChangeVer +
                             ", readTopVer=" + readyTopVer +
                             ", exchVer=" + exchangeVer + ']');
 
@@ -1249,7 +1272,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 if (msgTopVer != null && lastTopChangeVer.compareTo(msgTopVer) > 0) {
                     U.warn(log, "Stale version for full partition map update message (will ignore) [" +
-                        "lastTopChange=" + lastTopChangeVer +
+                        "grp=" + grp.cacheOrGroupName() +
+                        ", lastTopChange=" + lastTopChangeVer +
                         ", readTopVer=" + readyTopVer +
                         ", msgVer=" + msgTopVer + ']');
 
@@ -1266,7 +1290,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                             fullMapUpdated = true;
 
                             if (log.isDebugEnabled()) {
-                                log.debug("Overriding partition map in full update map [exchVer=" + exchangeVer +
+                                log.debug("Overriding partition map in full update map [" +
+                                    "grp=" + grp.cacheOrGroupName() +
+                                    ", exchVer=" + exchangeVer +
                                     ", curPart=" + mapString(part) +
                                     ", newPart=" + mapString(newPart) + ']');
                             }
@@ -1295,8 +1321,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                         if (!ctx.discovery().alive(nodeId)) {
                             if (log.isDebugEnabled())
-                                log.debug("Removing left node from full map update [nodeId=" + nodeId + ", partMap=" +
-                                    partMap + ']');
+                                log.debug("Removing left node from full map update [grp=" + grp.cacheOrGroupName() +
+                                    ", nodeId=" + nodeId + ", partMap=" + partMap + ']');
 
                             it.remove();
                         }
@@ -1311,7 +1337,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 if (!fullMapUpdated) {
                     if (log.isDebugEnabled()) {
-                        log.debug("No updates for full partition map (will ignore) [lastExch=" + lastTopChangeVer +
+                        log.debug("No updates for full partition map (will ignore) [" +
+                            "grp=" + grp.cacheOrGroupName() +
+                            ", lastExch=" + lastTopChangeVer +
                             ", exchVer=" + exchangeVer +
                             ", curMap=" + node2part +
                             ", newMap=" + partMap + ']');
@@ -1426,8 +1454,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                 consistencyCheck();
 
-                if (log.isDebugEnabled())
-                    log.debug("Partition map after full update: " + fullMapString());
+                if (log.isDebugEnabled()) {
+                    log.debug("Partition map after full update [grp=" + grp.cacheOrGroupName() +
+                        ", map=" + fullMapString() + ']');
+                }
 
                 if (changed)
                     ctx.exchange().scheduleResendPartitions();
@@ -1457,7 +1487,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (acquired - now >= 100) {
                 if (timeLog.isInfoEnabled())
                     timeLog.info("Waited too long to acquire topology write lock " +
-                        "[cache=" + grp.groupId() + ", waitTime=" + (acquired - now) + ']');
+                        "[grp=" + grp.cacheOrGroupName() + ", waitTime=" + (acquired - now) + ']');
             }
 
             if (stopping)
@@ -1492,7 +1522,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (acquired - now >= 100) {
                 if (timeLog.isInfoEnabled())
                     timeLog.info("Waited too long to acquire topology write lock " +
-                        "[cache=" + grp.groupId() + ", waitTime=" + (acquired - now) + ']');
+                        "[grp=" + grp.cacheOrGroupName() + ", waitTime=" + (acquired - now) + ']');
             }
 
             if (stopping)
@@ -1540,13 +1570,16 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         GridDhtPartitionMap parts,
         boolean force
     ) {
-        if (log.isDebugEnabled())
-            log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
+        if (log.isDebugEnabled()) {
+            log.debug("Updating single partition map [grp=" + grp.cacheOrGroupName() + ", exchId=" + exchId +
+                ", parts=" + mapString(parts) + ']');
+        }
 
         if (!ctx.discovery().alive(parts.nodeId())) {
-            if (log.isDebugEnabled())
-                log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId +
-                    ", parts=" + parts + ']');
+            if (log.isDebugEnabled()) {
+                log.debug("Received partition update for non-existing node (will ignore) [grp=" + grp.cacheOrGroupName() +
+                    ", exchId=" + exchId + ", parts=" + parts + ']');
+            }
 
             return false;
         }
@@ -1563,7 +1596,8 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 if (!force) {
                     if (lastTopChangeVer.initialized() && exchId != null && lastTopChangeVer.compareTo(exchId.topologyVersion()) > 0) {
                         U.warn(log, "Stale exchange id for single partition map update (will ignore) [" +
-                            "lastTopChange=" + lastTopChangeVer +
+                            "grp=" + grp.cacheOrGroupName() +
+                            ", lastTopChange=" + lastTopChangeVer +
                             ", readTopVer=" + readyTopVer +
                             ", exch=" + exchId.topologyVersion() + ']');
 
@@ -1582,7 +1616,9 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         parts.updateSequence(cur.updateSequence(), cur.topologyVersion());
                 }
                 else if (isStaleUpdate(cur, parts)) {
-                    U.warn(log, "Stale update for single partition map update (will ignore) [exchId=" + exchId +
+                    U.warn(log, "Stale update for single partition map update (will ignore) [" +
+                        "grp=" + grp.cacheOrGroupName() +
+                        ", exchId=" + exchId +
                         ", curMap=" + cur +
                         ", newMap=" + parts + ']');
 
@@ -1659,7 +1695,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 consistencyCheck();
 
                 if (log.isDebugEnabled())
-                    log.debug("Partition map after single update: " + fullMapString());
+                    log.debug("Partition map after single update [grp=" + grp.cacheOrGroupName() + ", map=" + fullMapString() + ']');
 
                 if (changed && exchId == null)
                     ctx.exchange().scheduleResendPartitions();
@@ -1979,7 +2015,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         }
 
                         U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                            "[nodeId=" + ctx.localNodeId() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
+                            "[nodeId=" + ctx.localNodeId() + ", grp=" + grp.cacheOrGroupName() +
                             ", partId=" + locPart.id() + ", haveHistory=" + haveHistory + "]");
 
                     }
@@ -2006,7 +2042,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                             this.updateSeq.setIfGreater(partMap.updateSequence());
 
                         U.warn(log, "Partition has been scheduled for rebalancing due to outdated update counter " +
-                            "[nodeId=" + e.getKey() + ", cacheOrGroupName=" + grp.cacheOrGroupName() +
+                            "[nodeId=" + e.getKey() + ", grp=" + grp.cacheOrGroupName() +
                             ", partId=" + p + ", haveHistory=" + haveHistory + "]");
                     }
                 }
@@ -2060,8 +2096,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                         changed = true;
 
-                        if (log.isDebugEnabled())
-                            log.debug("Evicted local partition (all affinity nodes are owners): " + part);
+                        if (log.isDebugEnabled()) {
+                            log.debug("Evicted local partition (all affinity nodes are owners) [grp=" + grp.cacheOrGroupName() +
+                                ", part=" + part + ']');
+                        }
                     }
                     else {
                         int ownerCnt = nodeIds.size();
@@ -2088,9 +2126,10 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
                                     changed = true;
 
-                                    if (log.isDebugEnabled())
-                                        log.debug("Evicted local partition (this node is oldest non-affinity node): " +
-                                            part);
+                                    if (log.isDebugEnabled()) {
+                                        log.debug("Evicted local partition (this node is oldest non-affinity node) [" +
+                                            "grp=" + grp.cacheOrGroupName() + ", part=" + part + ']');
+                                    }
 
                                     break;
                                 }
@@ -2134,6 +2173,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         boolean b = this.updateSeq.compareAndSet(seq0, seq + 1);
 
                         assert b : "Invalid update sequence [updateSeq=" + updateSeq +
+                            ", grp=" + grp.cacheOrGroupName() +
                             ", seq=" + seq +
                             ", curUpdateSeq=" + this.updateSeq.get() +
                             ", node2part=" + node2part.toFullString() + ']';
@@ -2468,7 +2508,7 @@ public class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             rebalancedTopVer = readyTopVer;
 
             if (log.isDebugEnabled())
-                log.debug("Updated rebalanced version [cache=" + grp.cacheOrGroupName() + ", ver=" + rebalancedTopVer + ']');
+                log.debug("Updated rebalanced version [grp=" + grp.cacheOrGroupName() + ", ver=" + rebalancedTopVer + ']');
         }
     }
 


[10/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTest.java
deleted file mode 100644
index 2774028..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTest.java
+++ /dev/null
@@ -1,820 +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;
-
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.NullArgumentException;
-import org.apache.ignite.ml.math.exceptions.SingularMatrixException;
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.apache.ignite.ml.math.util.MatrixUtil;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Tests for {@link OLSMultipleLinearRegression}.
- */
-public class OLSMultipleLinearRegressionTest extends AbstractMultipleLinearRegressionTest {
-    /** */
-    private double[] y;
-
-    /** */
-    private double[][] x;
-
-    /** */
-    @Before
-    @Override public void setUp() {
-        y = new double[] {11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        x = new double[6][];
-        x[0] = new double[] {0, 0, 0, 0, 0};
-        x[1] = new double[] {2.0, 0, 0, 0, 0};
-        x[2] = new double[] {0, 3.0, 0, 0, 0};
-        x[3] = new double[] {0, 0, 4.0, 0, 0};
-        x[4] = new double[] {0, 0, 0, 5.0, 0};
-        x[5] = new double[] {0, 0, 0, 0, 6.0};
-        super.setUp();
-    }
-
-    /** */
-    @Override protected OLSMultipleLinearRegression createRegression() {
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new DenseLocalOnHeapVector(y), new DenseLocalOnHeapMatrix(x));
-        return regression;
-    }
-
-    /** */
-    @Override protected int getNumberOfRegressors() {
-        return x[0].length + 1;
-    }
-
-    /** */
-    @Override protected int getSampleSize() {
-        return y.length;
-    }
-
-    /** */
-    @Test(expected = MathIllegalArgumentException.class)
-    public void cannotAddSampleDataWithSizeMismatch() {
-        double[] y = new double[] {1.0, 2.0};
-        double[][] x = new double[1][];
-        x[0] = new double[] {1.0, 0};
-        createRegression().newSampleData(new DenseLocalOnHeapVector(y), new DenseLocalOnHeapMatrix(x));
-    }
-
-    /** */
-    @Test
-    public void testPerfectFit() {
-        double[] betaHat = regression.estimateRegressionParameters();
-        TestUtils.assertEquals(new double[] {11.0, 1.0 / 2.0, 2.0 / 3.0, 3.0 / 4.0, 4.0 / 5.0, 5.0 / 6.0},
-            betaHat,
-            1e-13);
-        double[] residuals = regression.estimateResiduals();
-        TestUtils.assertEquals(new double[] {0d, 0d, 0d, 0d, 0d, 0d}, residuals,
-            1e-13);
-        Matrix errors = regression.estimateRegressionParametersVariance();
-        final double[] s = {1.0, -1.0 / 2.0, -1.0 / 3.0, -1.0 / 4.0, -1.0 / 5.0, -1.0 / 6.0};
-        Matrix refVar = new DenseLocalOnHeapMatrix(s.length, s.length);
-        for (int i = 0; i < refVar.rowSize(); i++)
-            for (int j = 0; j < refVar.columnSize(); j++) {
-                if (i == 0) {
-                    refVar.setX(i, j, s[j]);
-                    continue;
-                }
-                double x = s[i] * s[j];
-                refVar.setX(i, j, (i == j) ? 2 * x : x);
-            }
-        Assert.assertEquals(0.0,
-            TestUtils.maximumAbsoluteRowSum(errors.minus(refVar)),
-            5.0e-16 * TestUtils.maximumAbsoluteRowSum(refVar));
-        Assert.assertEquals(1, ((OLSMultipleLinearRegression)regression).calculateRSquared(), 1E-12);
-    }
-
-    /**
-     * Test Longley dataset against certified values provided by NIST.
-     * Data Source: J. Longley (1967) "An Appraisal of Least Squares
-     * Programs for the Electronic Computer from the Point of View of the User"
-     * Journal of the American Statistical Association, vol. 62. September,
-     * pp. 819-841.
-     *
-     * Certified values (and data) are from NIST:
-     * http://www.itl.nist.gov/div898/strd/lls/data/LINKS/DATA/Longley.dat
-     */
-    @Test
-    public void testLongly() {
-        // Y values are first, then independent vars
-        // Each row is one observation
-        double[] design = 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;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from NIST
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -3482258.63459582, 15.0618722713733,
-                -0.358191792925910E-01, -2.02022980381683,
-                -1.03322686717359, -0.511041056535807E-01,
-                1829.15146461355}, 2E-6); //
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                267.340029759711, -94.0139423988359, 46.28716775752924,
-                -410.114621930906, 309.7145907602313, -249.3112153297231,
-                -164.0489563956039, -13.18035686637081, 14.30477260005235,
-                455.394094551857, -17.26892711483297, -39.0550425226967,
-                -155.5499735953195, -85.6713080421283, 341.9315139607727,
-                -206.7578251937366},
-            1E-7);
-
-        // Check standard errors from NIST
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            890420.383607373,
-            84.9149257747669,
-            0.334910077722432E-01,
-            0.488399681651699,
-            0.214274163161675,
-            0.226073200069370,
-            455.478499142212}, errors, 1E-6);
-
-        // Check regression standard error against R
-        Assert.assertEquals(304.8540735619638, mdl.estimateRegressionStandardError(), 1E-8);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.995479004577296, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.992465007628826, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate model without intercept
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -52.99357013868291, 0.07107319907358,
-                -0.42346585566399, -0.57256866841929,
-                -0.41420358884978, 48.41786562001326}, 1E-8);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            129.54486693117232, 0.03016640003786,
-            0.41773654056612, 0.27899087467676, 0.32128496193363,
-            17.68948737819961}, errors, 1E-11);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                279.90274927293092, -130.32465380836874, 90.73228661967445, -401.31252201634948,
-                -440.46768772620027, -543.54512853774793, 201.32111639536299, 215.90889365977932,
-                73.09368242049943, 913.21694494481869, 424.82484953610174, -8.56475876776709,
-                -361.32974610842876, 27.34560497213464, 151.28955976355002, -492.49937355336846},
-            1E-8);
-
-        // Check regression standard error against R
-        Assert.assertEquals(475.1655079819517, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.9999670130706, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.999947220913, mdl.calculateAdjustedRSquared(), 1E-12);
-
-    }
-
-    /**
-     * Test R Swiss fertility dataset against R.
-     * Data Source: R datasets package
-     */
-    @Test
-    public void testSwissFertility() {
-        double[] design = new double[] {
-            80.2, 17.0, 15, 12, 9.96,
-            83.1, 45.1, 6, 9, 84.84,
-            92.5, 39.7, 5, 5, 93.40,
-            85.8, 36.5, 12, 7, 33.77,
-            76.9, 43.5, 17, 15, 5.16,
-            76.1, 35.3, 9, 7, 90.57,
-            83.8, 70.2, 16, 7, 92.85,
-            92.4, 67.8, 14, 8, 97.16,
-            82.4, 53.3, 12, 7, 97.67,
-            82.9, 45.2, 16, 13, 91.38,
-            87.1, 64.5, 14, 6, 98.61,
-            64.1, 62.0, 21, 12, 8.52,
-            66.9, 67.5, 14, 7, 2.27,
-            68.9, 60.7, 19, 12, 4.43,
-            61.7, 69.3, 22, 5, 2.82,
-            68.3, 72.6, 18, 2, 24.20,
-            71.7, 34.0, 17, 8, 3.30,
-            55.7, 19.4, 26, 28, 12.11,
-            54.3, 15.2, 31, 20, 2.15,
-            65.1, 73.0, 19, 9, 2.84,
-            65.5, 59.8, 22, 10, 5.23,
-            65.0, 55.1, 14, 3, 4.52,
-            56.6, 50.9, 22, 12, 15.14,
-            57.4, 54.1, 20, 6, 4.20,
-            72.5, 71.2, 12, 1, 2.40,
-            74.2, 58.1, 14, 8, 5.23,
-            72.0, 63.5, 6, 3, 2.56,
-            60.5, 60.8, 16, 10, 7.72,
-            58.3, 26.8, 25, 19, 18.46,
-            65.4, 49.5, 15, 8, 6.10,
-            75.5, 85.9, 3, 2, 99.71,
-            69.3, 84.9, 7, 6, 99.68,
-            77.3, 89.7, 5, 2, 100.00,
-            70.5, 78.2, 12, 6, 98.96,
-            79.4, 64.9, 7, 3, 98.22,
-            65.0, 75.9, 9, 9, 99.06,
-            92.2, 84.6, 3, 3, 99.46,
-            79.3, 63.1, 13, 13, 96.83,
-            70.4, 38.4, 26, 12, 5.62,
-            65.7, 7.7, 29, 11, 13.79,
-            72.7, 16.7, 22, 13, 11.22,
-            64.4, 17.6, 35, 32, 16.92,
-            77.6, 37.6, 15, 7, 4.97,
-            67.6, 18.7, 25, 7, 8.65,
-            35.0, 1.2, 37, 53, 42.34,
-            44.7, 46.6, 16, 29, 50.43,
-            42.8, 27.7, 22, 29, 58.33
-        };
-
-        final int nobs = 47;
-        final int nvars = 4;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from R
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                91.05542390271397,
-                -0.22064551045715,
-                -0.26058239824328,
-                -0.96161238456030,
-                0.12441843147162}, 1E-12);
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                7.1044267859730512, 1.6580347433531366,
-                4.6944952770029644, 8.4548022690166160, 13.6547432343186212,
-                -9.3586864458500774, 7.5822446330520386, 15.5568995563859289,
-                0.8113090736598980, 7.1186762732484308, 7.4251378771228724,
-                2.6761316873234109, 0.8351584810309354, 7.1769991119615177,
-                -3.8746753206299553, -3.1337779476387251, -0.1412575244091504,
-                1.1186809170469780, -6.3588097346816594, 3.4039270429434074,
-                2.3374058329820175, -7.9272368576900503, -7.8361010968497959,
-                -11.2597369269357070, 0.9445333697827101, 6.6544245101380328,
-                -0.9146136301118665, -4.3152449403848570, -4.3536932047009183,
-                -3.8907885169304661, -6.3027643926302188, -7.8308982189289091,
-                -3.1792280015332750, -6.7167298771158226, -4.8469946718041754,
-                -10.6335664353633685, 11.1031134362036958, 6.0084032641811733,
-                5.4326230830188482, -7.2375578629692230, 2.1671550814448222,
-                15.0147574652763112, 4.8625103516321015, -7.1597256413907706,
-                -0.4515205619767598, -10.2916870903837587, -15.7812984571900063},
-            1E-12);
-
-        // Check standard errors from R
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            6.94881329475087,
-            0.07360008972340,
-            0.27410957467466,
-            0.19454551679325,
-            0.03726654773803}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(7.73642194433223, mdl.estimateRegressionStandardError(), 1E-12);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.649789742860228, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.6164363850373927, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate the model with no intercept
-        mdl = new OLSMultipleLinearRegression();
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                0.52191832900513,
-                2.36588087917963,
-                -0.94770353802795,
-                0.30851985863609}, 1E-12);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                44.138759883538249, 27.720705122356215, 35.873200836126799,
-                34.574619581211977, 26.600168342080213, 15.074636243026923, -12.704904871199814,
-                1.497443824078134, 2.691972687079431, 5.582798774291231, -4.422986561283165,
-                -9.198581600334345, 4.481765170730647, 2.273520207553216, -22.649827853221336,
-                -17.747900013943308, 20.298314638496436, 6.861405135329779, -8.684712790954924,
-                -10.298639278062371, -9.896618896845819, 4.568568616351242, -15.313570491727944,
-                -13.762961360873966, 7.156100301980509, 16.722282219843990, 26.716200609071898,
-                -1.991466398777079, -2.523342564719335, 9.776486693095093, -5.297535127628603,
-                -16.639070567471094, -10.302057295211819, -23.549487860816846, 1.506624392156384,
-                -17.939174438345930, 13.105792202765040, -1.943329906928462, -1.516005841666695,
-                -0.759066561832886, 20.793137744128977, -2.485236153005426, 27.588238710486976,
-                2.658333257106881, -15.998337823623046, -5.550742066720694, -14.219077806826615},
-            1E-12);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            0.10470063765677, 0.41684100584290,
-            0.43370143099691, 0.07694953606522}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(17.24710630547, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.946350722085, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.9413600915813, mdl.calculateAdjustedRSquared(), 1E-12);
-    }
-
-    /**
-     * Test hat matrix computation
-     */
-    @Test
-    public void testHat() {
-
-        /*
-         * This example is from "The Hat Matrix in Regression and ANOVA",
-         * David C. Hoaglin and Roy E. Welsch,
-         * The American Statistician, Vol. 32, No. 1 (Feb., 1978), pp. 17-22.
-         *
-         */
-        double[] design = new double[] {
-            11.14, .499, 11.1,
-            12.74, .558, 8.9,
-            13.13, .604, 8.8,
-            11.51, .441, 8.9,
-            12.38, .550, 8.8,
-            12.60, .528, 9.9,
-            11.13, .418, 10.7,
-            11.7, .480, 10.5,
-            11.02, .406, 10.5,
-            11.41, .467, 10.7
-        };
-
-        int nobs = 10;
-        int nvars = 2;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new DenseLocalOnHeapMatrix());
-
-        Matrix hat = mdl.calculateHat();
-
-
-        // Reference data is upper half of symmetric hat matrix
-        double[] refData = new double[] {
-            .418, -.002, .079, -.274, -.046, .181, .128, .222, .050, .242,
-            .242, .292, .136, .243, .128, -.041, .033, -.035, .004,
-            .417, -.019, .273, .187, -.126, .044, -.153, .004,
-            .604, .197, -.038, .168, -.022, .275, -.028,
-            .252, .111, -.030, .019, -.010, -.010,
-            .148, .042, .117, .012, .111,
-            .262, .145, .277, .174,
-            .154, .120, .168,
-            .315, .148,
-            .187
-        };
-
-        // Check against reference data and verify symmetry
-        int k = 0;
-        for (int i = 0; i < 10; i++) {
-            for (int j = i; j < 10; j++) {
-                Assert.assertEquals(refData[k], hat.getX(i, j), 10e-3);
-                Assert.assertEquals(hat.getX(i, j), hat.getX(j, i), 10e-12);
-                k++;
-            }
-        }
-
-        /*
-         * Verify that residuals computed using the hat matrix are close to
-         * what we get from direct computation, i.e. r = (I - H) y
-         */
-        double[] residuals = mdl.estimateResiduals();
-        Matrix id = MatrixUtil.identityLike(hat, 10);
-        double[] hatResiduals = id.minus(hat).times(mdl.getY()).getStorage().data();
-        TestUtils.assertEquals(residuals, hatResiduals, 10e-12);
-    }
-
-    /**
-     * test calculateYVariance
-     */
-    @Test
-    public void testYVariance() {
-        // assumes: y = new double[]{11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(new DenseLocalOnHeapVector(y), new DenseLocalOnHeapMatrix(x));
-        TestUtils.assertEquals(mdl.calculateYVariance(), 3.5, 0);
-    }
-
-    /**
-     * Verifies that setting X and Y separately has the same effect as newSample(X,Y).
-     */
-    @Test
-    public void testNewSample2() {
-        double[] y = new double[] {1, 2, 3, 4};
-        double[][] x = new double[][] {
-            {19, 22, 33},
-            {20, 30, 40},
-            {25, 35, 45},
-            {27, 37, 47}
-        };
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new DenseLocalOnHeapVector(y), new DenseLocalOnHeapMatrix(x));
-        Matrix combinedX = regression.getX().copy();
-        Vector combinedY = regression.getY().copy();
-        regression.newXSampleData(new DenseLocalOnHeapMatrix(x));
-        regression.newYSampleData(new DenseLocalOnHeapVector(y));
-        Assert.assertEquals(combinedX, regression.getX());
-        Assert.assertEquals(combinedY, regression.getY());
-
-        // No intercept
-        regression.setNoIntercept(true);
-        regression.newSampleData(new DenseLocalOnHeapVector(y), new DenseLocalOnHeapMatrix(x));
-        combinedX = regression.getX().copy();
-        combinedY = regression.getY().copy();
-        regression.newXSampleData(new DenseLocalOnHeapMatrix(x));
-        regression.newYSampleData(new DenseLocalOnHeapVector(y));
-        Assert.assertEquals(combinedX, regression.getX());
-        Assert.assertEquals(combinedY, regression.getY());
-    }
-
-    /** */
-    @Test(expected = NullArgumentException.class)
-    public void testNewSampleDataYNull() {
-        createRegression().newSampleData(null, new DenseLocalOnHeapMatrix(new double[][] {{1}}));
-    }
-
-    /** */
-    @Test(expected = NullArgumentException.class)
-    public void testNewSampleDataXNull() {
-        createRegression().newSampleData(new DenseLocalOnHeapVector(new double[] {}), null);
-    }
-
-    /**
-     * This is a test based on the Wampler1 data set
-     * http://www.itl.nist.gov/div898/strd/lls/data/Wampler1.shtml
-     */
-    @Test
-    public void testWampler1() {
-        double[] data = new double[] {
-            1, 0,
-            6, 1,
-            63, 2,
-            364, 3,
-            1365, 4,
-            3906, 5,
-            9331, 6,
-            19608, 7,
-            37449, 8,
-            66430, 9,
-            111111, 10,
-            177156, 11,
-            271453, 12,
-            402234, 13,
-            579195, 14,
-            813616, 15,
-            1118481, 16,
-            1508598, 17,
-            2000719, 18,
-            2613660, 19,
-            3368421, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new DenseLocalOnHeapMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0, 1.0,
-                1.0, 1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler2 data set
-     * http://www.itl.nist.gov/div898/strd/lls/data/Wampler2.shtml
-     */
-    @Test
-    public void testWampler2() {
-        double[] data = new double[] {
-            1.00000, 0,
-            1.11111, 1,
-            1.24992, 2,
-            1.42753, 3,
-            1.65984, 4,
-            1.96875, 5,
-            2.38336, 6,
-            2.94117, 7,
-            3.68928, 8,
-            4.68559, 9,
-            6.00000, 10,
-            7.71561, 11,
-            9.92992, 12,
-            12.75603, 13,
-            16.32384, 14,
-            20.78125, 15,
-            26.29536, 16,
-            33.05367, 17,
-            41.26528, 18,
-            51.16209, 19,
-            63.00000, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new DenseLocalOnHeapMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0e-1,
-                1.0e-2,
-                1.0e-3, 1.0e-4,
-                1.0e-5}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler3 data set
-     * http://www.itl.nist.gov/div898/strd/lls/data/Wampler3.shtml
-     */
-    @Test
-    public void testWampler3() {
-        double[] data = new double[] {
-            760, 0,
-            -2042, 1,
-            2111, 2,
-            -1684, 3,
-            3888, 4,
-            1858, 5,
-            11379, 6,
-            17560, 7,
-            39287, 8,
-            64382, 9,
-            113159, 10,
-            175108, 11,
-            273291, 12,
-            400186, 13,
-            581243, 14,
-            811568, 15,
-            1121004, 16,
-            1506550, 17,
-            2002767, 18,
-            2611612, 19,
-            3369180, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new DenseLocalOnHeapMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                2152.32624678170,
-                2363.55173469681, 779.343524331583,
-                101.475507550350, 5.64566512170752,
-                0.112324854679312}, 1E-8); //
-
-        TestUtils.assertEquals(.999995559025820, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(5570284.53333333, mdl.estimateErrorVariance(), 1.0e-6);
-        TestUtils.assertEquals(83554268.0000000, mdl.calculateResidualSumOfSquares(), 1.0e-5);
-    }
-
-    /**
-     * This is a test based on the Wampler4 data set
-     * http://www.itl.nist.gov/div898/strd/lls/data/Wampler4.shtml
-     */
-    @Test
-    public void testWampler4() {
-        double[] data = new double[] {
-            75901, 0,
-            -204794, 1,
-            204863, 2,
-            -204436, 3,
-            253665, 4,
-            -200894, 5,
-            214131, 6,
-            -185192, 7,
-            221249, 8,
-            -138370, 9,
-            315911, 10,
-            -27644, 11,
-            455253, 12,
-            197434, 13,
-            783995, 14,
-            608816, 15,
-            1370781, 16,
-            1303798, 17,
-            2205519, 18,
-            2408860, 19,
-            3444321, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new DenseLocalOnHeapMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-6);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                215232.624678170,
-                236355.173469681, 77934.3524331583,
-                10147.5507550350, 564.566512170752,
-                11.2324854679312}, 1E-8);
-
-        TestUtils.assertEquals(.957478440825662, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(55702845333.3333, mdl.estimateErrorVariance(), 1.0e-4);
-        TestUtils.assertEquals(835542680000.000, mdl.calculateResidualSumOfSquares(), 1.0e-3);
-    }
-
-    /**
-     * Anything requiring beta calculation should advertise SME.
-     */
-    @Test(expected = SingularMatrixException.class)
-    public void testSingularCalculateBeta() {
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression(1e-15);
-        mdl.newSampleData(new double[] {1, 2, 3, 1, 2, 3, 1, 2, 3}, 3, 2, new DenseLocalOnHeapMatrix());
-        mdl.calculateBeta();
-    }
-
-    /** */
-    @Test(expected = NullPointerException.class)
-    public void testNoDataNPECalculateBeta() {
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.calculateBeta();
-    }
-
-    /** */
-    @Test(expected = NullPointerException.class)
-    public void testNoDataNPECalculateHat() {
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.calculateHat();
-    }
-
-    /** */
-    @Test(expected = NullPointerException.class)
-    public void testNoDataNPESSTO() {
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.calculateTotalSumOfSquares();
-    }
-
-    /** */
-    @Test(expected = MathIllegalArgumentException.class)
-    public void testMathIllegalArgumentException() {
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.validateSampleData(new DenseLocalOnHeapMatrix(1, 2), new DenseLocalOnHeapVector(1));
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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 be71934..5c79c8f 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,6 +17,13 @@
 
 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.LinearRegressionModelTest;
+import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionQRTrainerTest;
+import org.apache.ignite.ml.regressions.linear.LocalLinearRegressionSGDTrainerTest;
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
 
@@ -25,11 +32,14 @@ import org.junit.runners.Suite;
  */
 @RunWith(Suite.class)
 @Suite.SuiteClasses({
-    OLSMultipleLinearRegressionTest.class,
-    DistributedOLSMultipleLinearRegressionTest.class,
-    DistributedBlockOLSMultipleLinearRegressionTest.class,
-    OLSMultipleLinearRegressionModelTest.class
+    LinearRegressionModelTest.class,
+    LocalLinearRegressionQRTrainerTest.class,
+    LocalLinearRegressionSGDTrainerTest.class,
+    DistributedLinearRegressionQRTrainerTest.class,
+    DistributedLinearRegressionSGDTrainerTest.class,
+    BlockDistributedLinearRegressionQRTrainerTest.class,
+    BlockDistributedLinearRegressionSGDTrainerTest.class
 })
 public class RegressionsTestSuite {
     // No-op.
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..ed6bf36
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/ArtificialRegressionDatasets.java
@@ -0,0 +1,404 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..0c09d75
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionQRTrainerTest.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..58037e2
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/BlockDistributedLinearRegressionSGDTrainerTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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/b2060855/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
new file mode 100644
index 0000000..2a506d9
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionQRTrainerTest.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..71d3b3b
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/DistributedLinearRegressionSGDTrainerTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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/b2060855/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
new file mode 100644
index 0000000..a55623c
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/linear/GenericLinearRegressionTrainerTest.java
@@ -0,0 +1,206 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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);
+    }
+}


[14/15] ignite git commit: IGNITE-6647 Web Console: Support recreate index in migrations.

Posted by sb...@apache.org.
IGNITE-6647 Web Console: Support recreate index in migrations.


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

Branch: refs/heads/ignite-zk
Commit: a1b1f6c94778dc5277276a46500baa93c3beaad7
Parents: b206085
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Dec 28 17:07:58 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Dec 29 10:23:30 2017 +0700

----------------------------------------------------------------------
 modules/web-console/backend/index.js            |  6 ++--
 .../backend/migrations/recreate-index.js        | 30 ++++++++++++++++++++
 2 files changed, 34 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a1b1f6c9/modules/web-console/backend/index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/index.js b/modules/web-console/backend/index.js
index 266fa54..013de47 100644
--- a/modules/web-console/backend/index.js
+++ b/modules/web-console/backend/index.js
@@ -106,11 +106,13 @@ const init = ([settings, apiSrv, agentsHnd, browsersHnd]) => {
  * @param dbConnectionUri Mongo connection url.
  * @param group Migrations group.
  * @param migrationsPath Migrations path.
+ * @param collectionName Name of collection where migrations write info about applied scripts.
  */
-const migrate = (dbConnectionUri, group, migrationsPath) => {
+const migrate = (dbConnectionUri, group, migrationsPath, collectionName) => {
     const migrator = new MigrateMongoose({
         migrationsPath,
         dbConnectionUri,
+        collectionName,
         autosync: true
     });
 
@@ -144,7 +146,7 @@ injector.log.debug = () => {};
 Promise.all([injector('settings'), injector('mongo')])
     .then(([{mongoUrl}]) => {
         return migrate(mongoUrl, 'Ignite', path.join(__dirname, 'migrations'))
-            .then(() => migrate(mongoUrl, 'Ignite Modules', path.join(igniteModules, 'migrations')));
+            .then(() => migrate(mongoUrl, 'Ignite Modules', path.join(igniteModules, 'migrations'), 'migrationsModules'));
     })
     .then(() => Promise.all([injector('settings'), injector('api-server'), injector('agents-handler'), injector('browsers-handler')]))
     .then(init)

http://git-wip-us.apache.org/repos/asf/ignite/blob/a1b1f6c9/modules/web-console/backend/migrations/recreate-index.js
----------------------------------------------------------------------
diff --git a/modules/web-console/backend/migrations/recreate-index.js b/modules/web-console/backend/migrations/recreate-index.js
new file mode 100644
index 0000000..328ed43
--- /dev/null
+++ b/modules/web-console/backend/migrations/recreate-index.js
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+'use strict';
+
+module.exports = function(done, model, oldIdxName, oldIdx, newIdx) {
+    model.indexExists(oldIdxName)
+        .then((exists) => {
+            if (exists) {
+                return model.dropIndex(oldIdx)
+                    .then(() => model.createIndex(newIdx, {unique: true}));
+            }
+        })
+        .then(() => done())
+        .catch(done);
+};


[13/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
IGNITE-5217: Gradient descent for OLS lin reg

this closes #3308


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

Branch: refs/heads/ignite-zk
Commit: b20608557d96b120f2b944a4aad6e07ecafb407e
Parents: a394687
Author: dmitrievanthony <dm...@gmail.com>
Authored: Thu Dec 28 19:08:49 2017 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Thu Dec 28 19:08:49 2017 +0300

----------------------------------------------------------------------
 .../DistributedRegressionExample.java           | 149 ---
 .../DistributedRegressionModelExample.java      | 134 ---
 ...tedLinearRegressionExampleWithQRTrainer.java | 136 +++
 ...edLinearRegressionExampleWithSGDTrainer.java | 137 +++
 .../ml/regression/linear/package-info.java      |  22 +
 .../matrix/SparseBlockDistributedMatrix.java    |   5 +-
 .../ml/optimization/BarzilaiBorweinUpdater.java |  51 ++
 .../ignite/ml/optimization/GradientDescent.java | 201 +++++
 .../ml/optimization/GradientFunction.java       |  31 +
 .../LeastSquaresGradientFunction.java           |  33 +
 .../ignite/ml/optimization/SimpleUpdater.java   |  45 +
 .../apache/ignite/ml/optimization/Updater.java  |  30 +
 .../ignite/ml/optimization/package-info.java    |  22 +
 .../util/SparseDistributedMatrixMapReducer.java |  84 ++
 .../ml/optimization/util/package-info.java      |  22 +
 .../AbstractMultipleLinearRegression.java       | 378 --------
 .../regressions/MultipleLinearRegression.java   |  71 --
 .../OLSMultipleLinearRegression.java            | 257 ------
 .../OLSMultipleLinearRegressionModel.java       |  77 --
 .../OLSMultipleLinearRegressionModelFormat.java |  46 -
 .../OLSMultipleLinearRegressionTrainer.java     |  62 --
 .../regressions/RegressionsErrorMessages.java   |  28 -
 .../linear/LinearRegressionModel.java           | 107 +++
 .../linear/LinearRegressionQRTrainer.java       |  72 ++
 .../linear/LinearRegressionSGDTrainer.java      |  67 ++
 .../ml/regressions/linear/package-info.java     |  22 +
 .../org/apache/ignite/ml/LocalModelsTest.java   |  40 +-
 .../ml/optimization/GradientDescentTest.java    |  64 ++
 .../SparseDistributedMatrixMapReducerTest.java  | 135 +++
 .../AbstractMultipleLinearRegressionTest.java   | 164 ----
 ...tedBlockOLSMultipleLinearRegressionTest.java | 901 ------------------
 ...tributedOLSMultipleLinearRegressionTest.java | 903 -------------------
 .../OLSMultipleLinearRegressionModelTest.java   |  53 --
 .../OLSMultipleLinearRegressionTest.java        | 820 -----------------
 .../ml/regressions/RegressionsTestSuite.java    |  20 +-
 .../linear/ArtificialRegressionDatasets.java    | 404 +++++++++
 ...istributedLinearRegressionQRTrainerTest.java |  36 +
 ...stributedLinearRegressionSGDTrainerTest.java |  35 +
 ...istributedLinearRegressionQRTrainerTest.java |  36 +
 ...stributedLinearRegressionSGDTrainerTest.java |  35 +
 .../GenericLinearRegressionTrainerTest.java     | 206 +++++
 ...wareAbstractLinearRegressionTrainerTest.java | 124 +++
 .../linear/LinearRegressionModelTest.java       |  66 ++
 .../LocalLinearRegressionQRTrainerTest.java     |  36 +
 .../LocalLinearRegressionSGDTrainerTest.java    |  35 +
 .../resources/datasets/regression/README.md     |  98 ++
 .../resources/datasets/regression/boston.csv    | 506 +++++++++++
 .../resources/datasets/regression/diabetes.csv  | 442 +++++++++
 .../jdbc/JdbcPutIndexedValue8Benchmark.java     |   3 -
 ...iteOLSMultipleLinearRegressionBenchmark.java |  67 +-
 parent/pom.xml                                  |   1 +
 51 files changed, 3388 insertions(+), 4131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionExample.java b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionExample.java
deleted file mode 100644
index 3e65527..0000000
--- a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionExample.java
+++ /dev/null
@@ -1,149 +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;
-
-import java.util.Arrays;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.math.StorageConstants;
-import org.apache.ignite.ml.math.Tracer;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.regressions.OLSMultipleLinearRegression;
-import org.apache.ignite.thread.IgniteThread;
-
-/**
- * Run linear regression over distributed matrix.
- *
- * TODO: IGNITE-6222, Currently works only in local mode.
- *
- * @see OLSMultipleLinearRegression
- */
-public class DistributedRegressionExample {
-    /** Run example. */
-    public static void main(String[] args) throws InterruptedException {
-        System.out.println();
-        System.out.println(">>> Linear regression 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(), () -> {
-
-                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
-                };
-
-                final int nobs = 53;
-                final int nvars = 4;
-
-                System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread.");
-                // Create SparseDistributedMatrix, new cache will be created automagically.
-                SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(0, 0,
-                    StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE);
-
-                System.out.println(">>> Create new linear regression object");
-                OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-                regression.newSampleData(data, nobs, nvars, distributedMatrix);
-                System.out.println();
-
-                System.out.println(">>> Estimates the regression parameters b:");
-                System.out.println(Arrays.toString(regression.estimateRegressionParameters()));
-
-                System.out.println(">>> Estimates the residuals, ie u = y - X*b:");
-                System.out.println(Arrays.toString(regression.estimateResiduals()));
-
-                System.out.println(">>> Standard errors of the regression parameters:");
-                System.out.println(Arrays.toString(regression.estimateRegressionParametersStandardErrors()));
-
-                System.out.println(">>> Estimates the variance of the regression parameters, ie Var(b):");
-                Tracer.showAscii(regression.estimateRegressionParametersVariance());
-
-                System.out.println(">>> Estimates the standard error of the regression:");
-                System.out.println(regression.estimateRegressionStandardError());
-
-                System.out.println(">>> R-Squared statistic:");
-                System.out.println(regression.calculateRSquared());
-
-                System.out.println(">>> Adjusted R-squared statistic:");
-                System.out.println(regression.calculateAdjustedRSquared());
-
-                System.out.println(">>> Returns the variance of the regressand, ie Var(y):");
-                System.out.println(regression.estimateErrorVariance());
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionModelExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionModelExample.java b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionModelExample.java
deleted file mode 100644
index 38de97e..0000000
--- a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/DistributedRegressionModelExample.java
+++ /dev/null
@@ -1,134 +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;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.examples.ml.math.matrix.SparseDistributedMatrixExample;
-import org.apache.ignite.ml.math.StorageConstants;
-import org.apache.ignite.ml.math.Tracer;
-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.OLSMultipleLinearRegressionModel;
-import org.apache.ignite.ml.regressions.OLSMultipleLinearRegressionTrainer;
-import org.apache.ignite.thread.IgniteThread;
-
-/**
- * Run linear regression model over distributed matrix.
- *
- * @see OLSMultipleLinearRegressionModel
- */
-public class DistributedRegressionModelExample {
-    /** 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(), () -> {
-                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
-                };
-
-                final int nobs = 53;
-                final int nvars = 4;
-
-                System.out.println(">>> Create new SparseDistributedMatrix inside IgniteThread.");
-                // Create SparseDistributedMatrix, new cache will be created automagically.
-                SparseDistributedMatrix distributedMatrix = new SparseDistributedMatrix(0, 0,
-                    StorageConstants.ROW_STORAGE_MODE, StorageConstants.RANDOM_ACCESS_MODE);
-
-                System.out.println(">>> Create new linear regression trainer object.");
-                OLSMultipleLinearRegressionTrainer trainer
-                    = new OLSMultipleLinearRegressionTrainer(0, nobs, nvars, distributedMatrix);
-                System.out.println(">>> Perform the training to get the model.");
-                OLSMultipleLinearRegressionModel mdl = trainer.train(data);
-                System.out.println();
-
-                Vector val = new SparseDistributedVector(nobs).assign((i) -> data[i * (nvars + 1)]);
-
-                System.out.println(">>> The input data:");
-                Tracer.showAscii(val);
-
-                System.out.println(">>> Trained model prediction results:");
-                Tracer.showAscii(mdl.apply(val));
-            });
-
-            igniteThread.start();
-
-            igniteThread.join();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java
new file mode 100644
index 0000000..98ff2a2
--- /dev/null
+++ b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithQRTrainer.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.util.Arrays;
+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;
+
+/**
+ * Run linear regression model over distributed matrix.
+ *
+ * @see LinearRegressionQRTrainer
+ */
+public class DistributedLinearRegressionExampleWithQRTrainer {
+    /** */
+    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 model = trainer.train(distributedMatrix);
+                System.out.println(">>> Linear regression model: " + model);
+
+                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);
+                }
+                System.out.println(">>> ---------------------------------");
+            });
+
+            igniteThread.start();
+
+            igniteThread.join();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java
new file mode 100644
index 0000000..3f61762
--- /dev/null
+++ b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/DistributedLinearRegressionExampleWithSGDTrainer.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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 java.util.Arrays;
+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.ml.regressions.linear.LinearRegressionSGDTrainer;
+import org.apache.ignite.thread.IgniteThread;
+
+/**
+ * Run linear regression model over distributed matrix.
+ *
+ * @see LinearRegressionQRTrainer
+ */
+public class DistributedLinearRegressionExampleWithSGDTrainer {
+    /** */
+    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 LinearRegressionSGDTrainer(100_000, 1e-12);
+
+                System.out.println(">>> Perform the training to get the model.");
+                LinearRegressionModel model = trainer.train(distributedMatrix);
+                System.out.println(">>> Linear regression model: " + model);
+
+                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);
+                }
+                System.out.println(">>> ---------------------------------");
+            });
+
+            igniteThread.start();
+
+            igniteThread.join();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/package-info.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/package-info.java b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/package-info.java
new file mode 100644
index 0000000..d0441a4
--- /dev/null
+++ b/examples/src/main/ml/org/apache/ignite/examples/ml/regression/linear/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. -->
+ * ML linear regression examples.
+ */
+package org.apache.ignite.examples.ml.regression.linear;

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/math/impls/matrix/SparseBlockDistributedMatrix.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/math/impls/matrix/SparseBlockDistributedMatrix.java b/modules/ml/src/main/java/org/apache/ignite/ml/math/impls/matrix/SparseBlockDistributedMatrix.java
index 2d822d2..d387d21 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/math/impls/matrix/SparseBlockDistributedMatrix.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/math/impls/matrix/SparseBlockDistributedMatrix.java
@@ -38,7 +38,6 @@ import org.apache.ignite.ml.math.functions.IgniteDoubleFunction;
 import org.apache.ignite.ml.math.impls.storage.matrix.BlockMatrixStorage;
 import org.apache.ignite.ml.math.impls.storage.matrix.BlockVectorStorage;
 import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
 import org.apache.ignite.ml.math.impls.vector.VectorBlockEntry;
 
 /**
@@ -229,7 +228,7 @@ public class SparseBlockDistributedMatrix extends AbstractMatrix implements Stor
     @Override public Vector getCol(int col) {
         checkColumnIndex(col);
 
-        Vector res = new SparseDistributedVector(rowSize());
+        Vector res = new SparseBlockDistributedVector(rowSize());
 
         for (int i = 0; i < rowSize(); i++)
             res.setX(i, getX(i, col));
@@ -240,7 +239,7 @@ public class SparseBlockDistributedMatrix extends AbstractMatrix implements Stor
     @Override public Vector getRow(int row) {
         checkRowIndex(row);
 
-        Vector res = new SparseDistributedVector(columnSize());
+        Vector res = new SparseBlockDistributedVector(columnSize());
 
         for (int i = 0; i < columnSize(); i++)
             res.setX(i, getX(row, i));

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/optimization/BarzilaiBorweinUpdater.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/BarzilaiBorweinUpdater.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/BarzilaiBorweinUpdater.java
new file mode 100644
index 0000000..2190d86
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/BarzilaiBorweinUpdater.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.Vector;
+
+/**
+ * Updater based in Barzilai-Borwein method which guarantees convergence.
+ */
+public class BarzilaiBorweinUpdater implements Updater {
+    /** */
+    private static final long serialVersionUID = 5046575099408708472L;
+
+    /**
+     * Learning rate used on the first iteration.
+     */
+    private static final double INITIAL_LEARNING_RATE = 1.0;
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public Vector compute(Vector oldWeights, Vector oldGradient, Vector weights, Vector gradient, int iteration) {
+        double learningRate = computeLearningRate(oldWeights != null ? oldWeights.copy() : null, oldGradient != null ? oldGradient.copy() : null, weights.copy(), gradient.copy());
+        return weights.copy().minus(gradient.copy().times(learningRate));
+    }
+
+    /** */
+    private double computeLearningRate(Vector oldWeights, Vector oldGradient, Vector weights, Vector gradient) {
+        if (oldWeights == null || oldGradient == null)
+            return INITIAL_LEARNING_RATE;
+        else {
+            Vector gradientDiff = gradient.minus(oldGradient);
+            return weights.minus(oldWeights).dot(gradientDiff) / Math.pow(gradientDiff.kNorm(2.0), 2.0);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..f02bcb3
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientDescent.java
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..7dc6674
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/GradientFunction.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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 point);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..4d90e3b
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/LeastSquaresGradientFunction.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/optimization/SimpleUpdater.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/SimpleUpdater.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/SimpleUpdater.java
new file mode 100644
index 0000000..0f6d520
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/SimpleUpdater.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.Vector;
+
+/**
+ * Simple updater with fixed learning rate which doesn't guarantee convergence.
+ */
+public class SimpleUpdater implements Updater {
+    /** */
+    private static final long serialVersionUID = 6417716224818162225L;
+
+    /** */
+    private final double learningRate;
+
+    /** */
+    public SimpleUpdater(double learningRate) {
+        assert learningRate > 0;
+
+        this.learningRate = learningRate;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public Vector compute(Vector oldWeights, Vector oldGradient, Vector weights, Vector gradient, int iteration) {
+        return weights.minus(gradient.times(learningRate));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/optimization/Updater.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/Updater.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/Updater.java
new file mode 100644
index 0000000..83405d7
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/Updater.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.Vector;
+
+/**
+ * Weights updater applied on every gradient descent step to decide how weights should be changed.
+ */
+@FunctionalInterface
+public interface Updater extends Serializable {
+    /** */
+    Vector compute(Vector oldWeights, Vector oldGradient, Vector weights, Vector gradient, int iteration);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/optimization/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/optimization/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/package-info.java
new file mode 100644
index 0000000..96b0acf
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/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 implementations of optimization algorithms and related classes.
+ */
+package org.apache.ignite.ml.optimization;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..7a5f90b
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducer.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..cb01ab6
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/optimization/util/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 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/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegression.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegression.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegression.java
deleted file mode 100644
index 5bc92c9..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegression.java
+++ /dev/null
@@ -1,378 +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;
-
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.exceptions.CardinalityException;
-import org.apache.ignite.ml.math.exceptions.InsufficientDataException;
-import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.NoDataException;
-import org.apache.ignite.ml.math.exceptions.NonSquareMatrixException;
-import org.apache.ignite.ml.math.exceptions.NullArgumentException;
-import org.apache.ignite.ml.math.functions.Functions;
-import org.apache.ignite.ml.math.util.MatrixUtil;
-
-/**
- * This class is based on the corresponding class from Apache Common Math lib.
- * Abstract base class for implementations of MultipleLinearRegression.
- */
-public abstract class AbstractMultipleLinearRegression implements MultipleLinearRegression {
-    /** X sample data. */
-    private Matrix xMatrix;
-
-    /** Y sample data. */
-    private Vector yVector;
-
-    /** Whether or not the regression model includes an intercept.  True means no intercept. */
-    private boolean noIntercept = false;
-
-    /**
-     * @return the X sample data.
-     */
-    protected Matrix getX() {
-        return xMatrix;
-    }
-
-    /**
-     * @return the Y sample data.
-     */
-    protected Vector getY() {
-        return yVector;
-    }
-
-    /**
-     * @return true if the model has no intercept term; false otherwise
-     */
-    public boolean isNoIntercept() {
-        return noIntercept;
-    }
-
-    /**
-     * @param noIntercept true means the model is to be estimated without an intercept term
-     */
-    public void setNoIntercept(boolean noIntercept) {
-        this.noIntercept = noIntercept;
-    }
-
-    /**
-     * <p>Loads model x and y sample data from a flat input array, overriding any previous sample.
-     * </p>
-     * <p>Assumes that rows are concatenated with y values first in each row.  For example, an input
-     * <code>data</code> array containing the sequence of values (1, 2, 3, 4, 5, 6, 7, 8, 9) with
-     * <code>nobs = 3</code> and <code>nvars = 2</code> creates a regression dataset with two
-     * independent variables, as below:
-     * <pre>
-     *   y   x[0]  x[1]
-     *   --------------
-     *   1     2     3
-     *   4     5     6
-     *   7     8     9
-     * </pre>
-     * </p>
-     * <p>Note that there is no need to add an initial unitary column (column of 1's) when
-     * specifying a model including an intercept term.  If {@link #isNoIntercept()} is <code>true</code>,
-     * the X matrix will be created without an initial column of "1"s; otherwise this column will
-     * be added.
-     * </p>
-     * <p>Throws IllegalArgumentException if any of the following preconditions fail:
-     * <ul><li><code>data</code> cannot be null</li>
-     * <li><code>data.length = nobs * (nvars + 1)</li>
-     * <li><code>nobs > nvars</code></li></ul>
-     * </p>
-     *
-     * @param data input data array
-     * @param nobs number of observations (rows)
-     * @param nvars number of independent variables (columns, not counting y)
-     * @param like matrix(maybe empty) indicating how data should be stored
-     * @throws NullArgumentException if the data array is null
-     * @throws CardinalityException if the length of the data array is not equal to <code>nobs * (nvars + 1)</code>
-     * @throws InsufficientDataException if <code>nobs</code> is less than <code>nvars + 1</code>
-     */
-    public void newSampleData(double[] data, int nobs, int nvars, Matrix like) {
-        if (data == null)
-            throw new NullArgumentException();
-        if (data.length != nobs * (nvars + 1))
-            throw new CardinalityException(nobs * (nvars + 1), data.length);
-        if (nobs <= nvars)
-            throw new InsufficientDataException(RegressionsErrorMessages.INSUFFICIENT_OBSERVED_POINTS_IN_SAMPLE);
-        double[] y = new double[nobs];
-        final int cols = noIntercept ? nvars : nvars + 1;
-        double[][] x = new double[nobs][cols];
-        int pointer = 0;
-        for (int i = 0; i < nobs; i++) {
-            y[i] = data[pointer++];
-            if (!noIntercept)
-                x[i][0] = 1.0d;
-            for (int j = noIntercept ? 0 : 1; j < cols; j++)
-                x[i][j] = data[pointer++];
-        }
-        xMatrix = MatrixUtil.like(like, nobs, cols).assign(x);
-        yVector = MatrixUtil.likeVector(like, y.length).assign(y);
-    }
-
-    /**
-     * Loads new y sample data, overriding any previous data.
-     *
-     * @param y the array representing the y sample
-     * @throws NullArgumentException if y is null
-     * @throws NoDataException if y is empty
-     */
-    protected void newYSampleData(Vector y) {
-        if (y == null)
-            throw new NullArgumentException();
-        if (y.size() == 0)
-            throw new NoDataException();
-        // TODO: IGNITE-5826, Should we copy here?
-        yVector = y;
-    }
-
-    /**
-     * <p>Loads new x sample data, overriding any previous data.
-     * </p>
-     * The input <code>x</code> array should have one row for each sample
-     * observation, with columns corresponding to independent variables.
-     * For example, if <pre>
-     * <code> x = new double[][] {{1, 2}, {3, 4}, {5, 6}} </code></pre>
-     * then <code>setXSampleData(x) </code> results in a model with two independent
-     * variables and 3 observations:
-     * <pre>
-     *   x[0]  x[1]
-     *   ----------
-     *     1    2
-     *     3    4
-     *     5    6
-     * </pre>
-     * </p>
-     * <p>Note that there is no need to add an initial unitary column (column of 1's) when
-     * specifying a model including an intercept term.
-     * </p>
-     *
-     * @param x the rectangular array representing the x sample
-     * @throws NullArgumentException if x is null
-     * @throws NoDataException if x is empty
-     * @throws CardinalityException if x is not rectangular
-     */
-    protected void newXSampleData(Matrix x) {
-        if (x == null)
-            throw new NullArgumentException();
-        if (x.rowSize() == 0)
-            throw new NoDataException();
-        if (noIntercept)
-            // TODO: IGNITE-5826, Should we copy here?
-            xMatrix = x;
-        else { // Augment design matrix with initial unitary column
-            xMatrix = MatrixUtil.like(x, x.rowSize(), x.columnSize() + 1);
-            xMatrix.viewColumn(0).map(Functions.constant(1.0));
-            xMatrix.viewPart(0, x.rowSize(), 1, x.columnSize()).assign(x);
-        }
-    }
-
-    /**
-     * Validates sample data.  Checks that
-     * <ul><li>Neither x nor y is null or empty;</li>
-     * <li>The length (i.e. number of rows) of x equals the length of y</li>
-     * <li>x has at least one more row than it has columns (i.e. there is
-     * sufficient data to estimate regression coefficients for each of the
-     * columns in x plus an intercept.</li>
-     * </ul>
-     *
-     * @param x the n x k matrix representing the x data
-     * @param y the n-sized vector representing the y data
-     * @throws NullArgumentException if {@code x} or {@code y} is null
-     * @throws CardinalityException if {@code x} and {@code y} do not have the same length
-     * @throws NoDataException if {@code x} or {@code y} are zero-length
-     * @throws MathIllegalArgumentException if the number of rows of {@code x} is not larger than the number of columns
-     * + 1
-     */
-    protected void validateSampleData(Matrix x, Vector y) throws MathIllegalArgumentException {
-        if ((x == null) || (y == null))
-            throw new NullArgumentException();
-        if (x.rowSize() != y.size())
-            throw new CardinalityException(y.size(), x.rowSize());
-        if (x.rowSize() == 0) {  // Must be no y data either
-            throw new NoDataException();
-        }
-        if (x.columnSize() + 1 > x.rowSize()) {
-            throw new MathIllegalArgumentException(
-                RegressionsErrorMessages.NOT_ENOUGH_DATA_FOR_NUMBER_OF_PREDICTORS,
-                x.rowSize(), x.columnSize());
-        }
-    }
-
-    /**
-     * Validates that the x data and covariance matrix have the same
-     * number of rows and that the covariance matrix is square.
-     *
-     * @param x the [n,k] array representing the x sample
-     * @param covariance the [n,n] array representing the covariance matrix
-     * @throws CardinalityException if the number of rows in x is not equal to the number of rows in covariance
-     * @throws NonSquareMatrixException if the covariance matrix is not square
-     */
-    protected void validateCovarianceData(double[][] x, double[][] covariance) {
-        if (x.length != covariance.length)
-            throw new CardinalityException(x.length, covariance.length);
-        if (covariance.length > 0 && covariance.length != covariance[0].length)
-            throw new NonSquareMatrixException(covariance.length, covariance[0].length);
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override public double[] estimateRegressionParameters() {
-        Vector b = calculateBeta();
-        return b.getStorage().data();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override public double[] estimateResiduals() {
-        Vector b = calculateBeta();
-        Vector e = yVector.minus(xMatrix.times(b));
-        return e.getStorage().data();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override public Matrix estimateRegressionParametersVariance() {
-        return calculateBetaVariance();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override public double[] estimateRegressionParametersStandardErrors() {
-        Matrix betaVariance = estimateRegressionParametersVariance();
-        double sigma = calculateErrorVariance();
-        int len = betaVariance.rowSize();
-        double[] res = new double[len];
-        for (int i = 0; i < len; i++)
-            res[i] = Math.sqrt(sigma * betaVariance.getX(i, i));
-        return res;
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override public double estimateRegressandVariance() {
-        return calculateYVariance();
-    }
-
-    /**
-     * Estimates the variance of the error.
-     *
-     * @return estimate of the error variance
-     */
-    public double estimateErrorVariance() {
-        return calculateErrorVariance();
-
-    }
-
-    /**
-     * Estimates the standard error of the regression.
-     *
-     * @return regression standard error
-     */
-    public double estimateRegressionStandardError() {
-        return Math.sqrt(estimateErrorVariance());
-    }
-
-    /**
-     * Calculates the beta of multiple linear regression in matrix notation.
-     *
-     * @return beta
-     */
-    protected abstract Vector calculateBeta();
-
-    /**
-     * Calculates the beta variance of multiple linear regression in matrix
-     * notation.
-     *
-     * @return beta variance
-     */
-    protected abstract Matrix calculateBetaVariance();
-
-    /**
-     * Calculates the variance of the y values.
-     *
-     * @return Y variance
-     */
-    protected double calculateYVariance() {
-        // Compute initial estimate using definitional formula
-        int vSize = yVector.size();
-        double xbar = yVector.sum() / vSize;
-        // Compute correction factor in second pass
-        final double corr = yVector.foldMap((val, acc) -> acc + val - xbar, Functions.IDENTITY, 0.0);
-        final double mean = xbar - corr;
-        return yVector.foldMap(Functions.PLUS, val -> (val - mean) * (val - mean), 0.0) / (vSize - 1);
-    }
-
-    /**
-     * <p>Calculates the variance of the error term.</p>
-     * Uses the formula <pre>
-     * var(u) = u &middot; u / (n - k)
-     * </pre>
-     * where n and k are the row and column dimensions of the design
-     * matrix X.
-     *
-     * @return error variance estimate
-     */
-    protected double calculateErrorVariance() {
-        Vector residuals = calculateResiduals();
-        return residuals.dot(residuals) /
-            (xMatrix.rowSize() - xMatrix.columnSize());
-    }
-
-    /**
-     * Calculates the residuals of multiple linear regression in matrix
-     * notation.
-     *
-     * <pre>
-     * u = y - X * b
-     * </pre>
-     *
-     * @return The residuals [n,1] matrix
-     */
-    protected Vector calculateResiduals() {
-        Vector b = calculateBeta();
-        return yVector.minus(xMatrix.times(b));
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        AbstractMultipleLinearRegression that = (AbstractMultipleLinearRegression)o;
-
-        return noIntercept == that.noIntercept && xMatrix.equals(that.xMatrix);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = xMatrix.hashCode();
-
-        res = 31 * res + (noIntercept ? 1 : 0);
-
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/MultipleLinearRegression.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/MultipleLinearRegression.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/MultipleLinearRegression.java
deleted file mode 100644
index 2fc4dde..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/MultipleLinearRegression.java
+++ /dev/null
@@ -1,71 +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;
-
-import org.apache.ignite.ml.math.Matrix;
-
-/**
- * This class is based on the corresponding class from Apache Common Math lib. * The multiple linear regression can be
- * represented in matrix-notation.
- * <pre>
- *  y=X*b+u
- * </pre>
- * where y is an <code>n-vector</code> <b>regressand</b>, X is a <code>[n,k]</code> matrix whose <code>k</code> columns
- * are called <b>regressors</b>, b is <code>k-vector</code> of <b>regression parameters</b> and <code>u</code> is an
- * <code>n-vector</code> of <b>error terms</b> or <b>residuals</b>.
- * <p>
- * The notation is quite standard in literature, cf eg <a href="http://www.econ.queensu.ca/ETM">Davidson and MacKinnon,
- * Econometrics Theory and Methods, 2004</a>. </p>
- */
-public interface MultipleLinearRegression {
-    /**
-     * Estimates the regression parameters b.
-     *
-     * @return The [k,1] array representing b
-     */
-    public double[] estimateRegressionParameters();
-
-    /**
-     * Estimates the variance of the regression parameters, ie Var(b).
-     *
-     * @return The k x k matrix representing the variance of b
-     */
-    public Matrix estimateRegressionParametersVariance();
-
-    /**
-     * Estimates the residuals, ie u = y - X*b.
-     *
-     * @return The [n,1] array representing the residuals
-     */
-    public double[] estimateResiduals();
-
-    /**
-     * Returns the variance of the regressand, ie Var(y).
-     *
-     * @return The double representing the variance of y
-     */
-    public double estimateRegressandVariance();
-
-    /**
-     * Returns the standard errors of the regression parameters.
-     *
-     * @return standard errors of estimated regression parameters
-     */
-    public double[] estimateRegressionParametersStandardErrors();
-
-}


[03/15] ignite git commit: ignite-6745 URLClassLoader.getURLs() usages are removed

Posted by sb...@apache.org.
ignite-6745 URLClassLoader.getURLs() usages are removed


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

Branch: refs/heads/ignite-zk
Commit: ee2a6f7c3f2e3c9bd8dc61c8dbdf171e933d9481
Parents: 33ad0ad
Author: Andrey Gura <ag...@apache.org>
Authored: Wed Dec 27 18:50:12 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Dec 28 16:12:23 2017 +0300

----------------------------------------------------------------------
 .../ignite/codegen/MessageCodeGenerator.java    |  8 +--
 .../junits/IgniteCompatibilityAbstractTest.java |  7 +-
 .../junits/IgniteCompatibilityNodeRunner.java   | 15 ++---
 .../ignite/internal/binary/BinaryContext.java   | 66 +++++++++----------
 .../processors/hadoop/HadoopClassLoader.java    |  5 +-
 .../ignite/internal/util/IgniteUtils.java       | 61 ++++++++++++++++-
 .../hadoop/HadoopTestClassLoader.java           |  5 +-
 .../ignite/tools/classgen/ClassesGenerator.java | 69 ++++++++++++++++++--
 .../uri/GridUriDeploymentFileProcessor.java     | 28 ++++----
 9 files changed, 186 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
----------------------------------------------------------------------
diff --git a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
index 3ea0c81..99cf849 100644
--- a/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
+++ b/modules/codegen/src/main/java/org/apache/ignite/codegen/MessageCodeGenerator.java
@@ -26,7 +26,6 @@ import java.io.FileWriter;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.BitSet;
 import java.util.Collection;
@@ -39,13 +38,12 @@ import java.util.Map;
 import java.util.Set;
 import java.util.TreeSet;
 import java.util.UUID;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.GridCodegenConverter;
 import org.apache.ignite.internal.GridDirectCollection;
 import org.apache.ignite.internal.GridDirectMap;
 import org.apache.ignite.internal.GridDirectTransient;
 import org.apache.ignite.internal.IgniteCodeGeneratingFail;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.processors.cache.version.GridCacheVersionEx;
 import org.apache.ignite.internal.util.typedef.internal.SB;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
@@ -879,9 +877,9 @@ public class MessageCodeGenerator {
                 }
             });
 
-        URLClassLoader ldr = (URLClassLoader)getClass().getClassLoader();
+        ClassLoader ldr = getClass().getClassLoader();
 
-        for (URL url : ldr.getURLs()) {
+        for (URL url :  IgniteUtils.classLoaderUrls(ldr)) {
             File file = new File(url.toURI());
 
             int prefixLen = file.getPath().length() + 1;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
index 321da12..8202c1b 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.compatibility.testframework.junits;
 
 import java.io.File;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.UUID;
@@ -32,6 +31,7 @@ import org.apache.ignite.compatibility.testframework.util.MavenUtils;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
@@ -164,12 +164,13 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
                         filteredJvmArgs.add(arg);
                 }
 
-                URLClassLoader ldr = (URLClassLoader)CLASS_LOADER;
+                ClassLoader ldr = CLASS_LOADER;
 
                 final Collection<Dependency> dependencies = getDependencies(ver);
 
                 StringBuilder pathBuilder = new StringBuilder();
-                for (URL url : ldr.getURLs()) {
+
+                for (URL url : IgniteUtils.classLoaderUrls(ldr)) {
                     String path = url.getPath();
 
                     boolean excluded = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
index 7c9a511..7a72ea6 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
@@ -23,7 +23,6 @@ import java.io.BufferedWriter;
 import java.io.File;
 import java.io.IOException;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -33,6 +32,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.Ignition;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.util.GridJavaProcess;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
@@ -137,9 +137,8 @@ public class IgniteCompatibilityNodeRunner extends IgniteNodeRunner {
 
                 X.println("Ignite startup/Init closure/post configuration closure is probably hanging at");
 
-                for (StackTraceElement ste : mainThread.getStackTrace()) {
+                for (StackTraceElement ste : mainThread.getStackTrace())
                     X.println("\t" + ste.toString());
-                }
 
                 X.println("\nDumping classpath");
                 dumpClasspath();
@@ -157,14 +156,10 @@ public class IgniteCompatibilityNodeRunner extends IgniteNodeRunner {
      * Dumps classpath to output stream.
      */
     private static void dumpClasspath() {
-        final ClassLoader clsLdr = IgniteCompatibilityNodeRunner.class.getClassLoader();
-        if (clsLdr instanceof URLClassLoader) {
-            URLClassLoader ldr = (URLClassLoader)clsLdr;
+        ClassLoader clsLdr = IgniteCompatibilityNodeRunner.class.getClassLoader();
 
-            for (URL url : ldr.getURLs()) {
-                X.println("Classpath url: [" + url.getPath() + "]");
-            }
-        }
+        for (URL url : IgniteUtils.classLoaderUrls(clsLdr))
+            X.println("Classpath url: [" + url.getPath() + ']');
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
index 5be1d39..b161129 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/binary/BinaryContext.java
@@ -23,7 +23,6 @@ import java.lang.reflect.Field;
 import java.math.BigDecimal;
 import java.net.URISyntaxException;
 import java.net.URL;
-import java.net.URLClassLoader;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.ArrayList;
@@ -48,6 +47,7 @@ import java.util.jar.JarFile;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.binary.BinaryBasicIdMapper;
 import org.apache.ignite.binary.BinaryBasicNameMapper;
 import org.apache.ignite.binary.BinaryIdMapper;
@@ -553,55 +553,53 @@ public class BinaryContext {
 
         ClassLoader ldr = U.gridClassLoader();
 
-        if (ldr instanceof URLClassLoader) {
-            String pkgPath = pkgName.replaceAll("\\.", "/");
+        String pkgPath = pkgName.replaceAll("\\.", "/");
 
-            URL[] urls = ((URLClassLoader)ldr).getURLs();
+        URL[] urls = IgniteUtils.classLoaderUrls(ldr);
 
-            for (URL url : urls) {
-                String proto = url.getProtocol().toLowerCase();
+        for (URL url : urls) {
+            String proto = url.getProtocol().toLowerCase();
 
-                if ("file".equals(proto)) {
-                    try {
-                        File cpElement = new File(url.toURI());
+            if ("file".equals(proto)) {
+                try {
+                    File cpElement = new File(url.toURI());
 
-                        if (cpElement.isDirectory()) {
-                            File pkgDir = new File(cpElement, pkgPath);
+                    if (cpElement.isDirectory()) {
+                        File pkgDir = new File(cpElement, pkgPath);
 
-                            if (pkgDir.isDirectory()) {
-                                for (File file : pkgDir.listFiles()) {
-                                    String fileName = file.getName();
+                        if (pkgDir.isDirectory()) {
+                            for (File file : pkgDir.listFiles()) {
+                                String fileName = file.getName();
 
-                                    if (file.isFile() && fileName.toLowerCase().endsWith(".class"))
-                                        clsNames.add(pkgName + '.' + fileName.substring(0, fileName.length() - 6));
-                                }
+                                if (file.isFile() && fileName.toLowerCase().endsWith(".class"))
+                                    clsNames.add(pkgName + '.' + fileName.substring(0, fileName.length() - 6));
                             }
                         }
-                        else if (cpElement.isFile()) {
-                            try {
-                                JarFile jar = new JarFile(cpElement);
+                    }
+                    else if (cpElement.isFile()) {
+                        try {
+                            JarFile jar = new JarFile(cpElement);
 
-                                Enumeration<JarEntry> entries = jar.entries();
+                            Enumeration<JarEntry> entries = jar.entries();
 
-                                while (entries.hasMoreElements()) {
-                                    String entry = entries.nextElement().getName();
+                            while (entries.hasMoreElements()) {
+                                String entry = entries.nextElement().getName();
 
-                                    if (entry.startsWith(pkgPath) && entry.endsWith(".class")) {
-                                        String clsName = entry.substring(pkgPath.length() + 1, entry.length() - 6);
+                                if (entry.startsWith(pkgPath) && entry.endsWith(".class")) {
+                                    String clsName = entry.substring(pkgPath.length() + 1, entry.length() - 6);
 
-                                        if (!clsName.contains("/") && !clsName.contains("\\"))
-                                            clsNames.add(pkgName + '.' + clsName);
-                                    }
+                                    if (!clsName.contains("/") && !clsName.contains("\\"))
+                                        clsNames.add(pkgName + '.' + clsName);
                                 }
                             }
-                            catch (IOException ignored) {
-                                // No-op.
-                            }
+                        }
+                        catch (IOException ignored) {
+                            // No-op.
                         }
                     }
-                    catch (URISyntaxException ignored) {
-                        // No-op.
-                    }
+                }
+                catch (URISyntaxException ignored) {
+                    // No-op.
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index b1b4d18..24367f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.hadoop;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.util.ClassCache;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -60,10 +61,10 @@ public class HadoopClassLoader extends URLClassLoader implements ClassCache {
         "org.apache.ignite.internal.processors.hadoop.impl.v2.HadoopShutdownHookManager";
 
     /** */
-    private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopClassLoader.class.getClassLoader();
+    private static final ClassLoader APP_CLS_LDR = HadoopClassLoader.class.getClassLoader();
 
     /** */
-    private static final Collection<URL> appJars = F.asList(APP_CLS_LDR.getURLs());
+    private static final Collection<URL> appJars = F.asList(IgniteUtils.classLoaderUrls(APP_CLS_LDR));
 
     /** Mutex for native libraries initialization. */
     private static final Object LIBS_MUX = new Object();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/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 b3ca6ff..6fb19a1 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
@@ -240,8 +240,6 @@ import org.apache.ignite.transactions.TransactionTimeoutException;
 import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
-import sun.misc.SharedSecrets;
-import sun.misc.URLClassPath;
 import sun.misc.Unsafe;
 
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DISABLE_HOSTNAME_VERIFIER;
@@ -527,6 +525,21 @@ public abstract class IgniteUtils {
     /** */
     private static final boolean assertionsEnabled;
 
+    /** Empty URL array. */
+    private static final URL[] EMPTY_URL_ARR = new URL[0];
+
+    /** Builtin class loader class.
+     *
+     * Note: needs for compatibility with Java 9.
+     */
+    private static final Class bltClsLdrCls = defaultClassLoaderClass();
+
+    /** Url class loader field.
+     *
+     * Note: needs for compatibility with Java 9.
+     */
+    private static final Field urlClsLdrField = urlClassLoaderField();
+
     /*
      * Initializes enterprise check.
      */
@@ -7536,6 +7549,50 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Returns URLs of class loader
+     *
+     * @param clsLdr Class loader.
+     */
+    public static URL[] classLoaderUrls(ClassLoader clsLdr) {
+        if (clsLdr == null)
+            return EMPTY_URL_ARR;
+        else if (clsLdr instanceof URLClassLoader)
+            return ((URLClassLoader)clsLdr).getURLs();
+        else if (bltClsLdrCls != null && urlClsLdrField != null && bltClsLdrCls.isAssignableFrom(clsLdr.getClass())) {
+            try {
+                return ((URLClassLoader)urlClsLdrField.get(clsLdr)).getURLs();
+            }
+            catch (IllegalAccessException e) {
+                return EMPTY_URL_ARR;
+            }
+        }
+        else
+            return EMPTY_URL_ARR;
+    }
+
+    /** */
+    @Nullable private static Class defaultClassLoaderClass() {
+        try {
+            return Class.forName("jdk.internal.loader.BuiltinClassLoader");
+        }
+        catch (ClassNotFoundException e) {
+            return null;
+        }
+    }
+
+    /** */
+    @Nullable private static Field urlClassLoaderField() {
+        try {
+            Class cls = defaultClassLoaderClass();
+
+            return cls == null ? null : cls.getDeclaredField("ucp");
+        }
+        catch (NoSuchFieldException e) {
+            return null;
+        }
+    }
+
+    /**
      * Sleeps for given number of milliseconds.
      *
      * @param ms Time to sleep.

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java
index 91ee0c9..2ba67c8 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTestClassLoader.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.X;
 
@@ -32,10 +33,10 @@ import java.util.List;
  */
 public class HadoopTestClassLoader extends URLClassLoader {
     /** Parent class loader. */
-    private static final URLClassLoader APP_CLS_LDR = (URLClassLoader)HadoopTestClassLoader.class.getClassLoader();
+   private static final ClassLoader APP_CLS_LDR = HadoopTestClassLoader.class.getClassLoader();
 
     /** */
-    private static final Collection<URL> APP_JARS = F.asList(APP_CLS_LDR.getURLs());
+    private static final Collection<URL> APP_JARS = F.asList(IgniteUtils.classLoaderUrls(APP_CLS_LDR));
 
     /** All participating URLs. */
     private static final URL[] URLS;

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
index ecc0654..2be847b 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
@@ -35,7 +35,6 @@ import java.util.Comparator;
 import java.util.TreeSet;
 import java.util.jar.JarEntry;
 import java.util.jar.JarInputStream;
-
 /**
  * Serialized classes generator.
  */
@@ -51,6 +50,21 @@ public class ClassesGenerator {
         "org.apache.ignite.tools"
     };
 
+    /** Empty URL array. */
+    private static final URL[] EMPTY_URL_ARR = new URL[0];
+
+    /** Builtin class loader class.
+     *
+     * Note: needs for compatibility with Java 9.
+     */
+    private static final Class bltClsLdrCls = defaultClassLoaderClass();
+
+    /** Url class loader field.
+     *
+     * Note: needs for compatibility with Java 9.
+     */
+    private static final Field urlClsLdrField = urlClassLoaderField();
+
     /**
      * @param args Arguments.
      * @throws Exception In case of error.
@@ -69,9 +83,6 @@ public class ClassesGenerator {
     }
 
     /** */
-    private final URLClassLoader ldr = (URLClassLoader)getClass().getClassLoader();
-
-    /** */
     private final Collection<Class> classes = new TreeSet<>(new Comparator<Class>() {
         @Override public int compare(Class c1, Class c2) {
             return c1.getName().compareTo(c2.getName());
@@ -112,7 +123,8 @@ public class ClassesGenerator {
     private void generate() throws Exception {
         System.out.println("Generating classnames.properties...");
 
-        for (URL url : ldr.getURLs())
+
+        for (URL url : classLoaderUrls(getClass().getClassLoader()))
             processUrl(url);
 
         if (!errs.isEmpty()) {
@@ -185,6 +197,51 @@ public class ClassesGenerator {
     }
 
     /**
+     * Returns URLs of class loader
+     *
+     * @param clsLdr Class loader.
+     */
+    public static URL[] classLoaderUrls(ClassLoader clsLdr) {
+        if (clsLdr == null)
+            return EMPTY_URL_ARR;
+        else if (clsLdr instanceof URLClassLoader)
+            return ((URLClassLoader)clsLdr).getURLs();
+        else if (bltClsLdrCls != null && urlClsLdrField != null && bltClsLdrCls.isAssignableFrom(clsLdr.getClass())) {
+            try {
+                return ((URLClassLoader)urlClsLdrField.get(clsLdr)).getURLs();
+            }
+            catch (IllegalAccessException e) {
+                return EMPTY_URL_ARR;
+            }
+        }
+        else
+            return EMPTY_URL_ARR;
+    }
+
+    /** */
+    private static Class defaultClassLoaderClass() {
+        try {
+            return Class.forName("jdk.internal.loader.BuiltinClassLoader");
+        }
+        catch (ClassNotFoundException e) {
+            return null;
+        }
+    }
+
+    /** */
+    private static Field urlClassLoaderField() {
+        try {
+            Class cls = defaultClassLoaderClass();
+
+            return cls == null ? null : cls.getDeclaredField("ucp");
+        }
+        catch (NoSuchFieldException e) {
+            return null;
+        }
+    }
+
+
+    /**
      * @param path File path.
      * @param prefixLen Prefix length.
      * @throws Exception In case of error.
@@ -209,7 +266,7 @@ public class ClassesGenerator {
         }
 
         if (included) {
-            Class<?> cls = Class.forName(clsName, false, ldr);
+            Class<?> cls = Class.forName(clsName, false, getClass().getClassLoader());
 
             if (Serializable.class.isAssignableFrom(cls) &&
                 !(cls.getName().endsWith("Future") || cls.getName().endsWith("FutureAdapter"))) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/ee2a6f7c/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/GridUriDeploymentFileProcessor.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/GridUriDeploymentFileProcessor.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/GridUriDeploymentFileProcessor.java
index d82a57e..3564f2f 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/GridUriDeploymentFileProcessor.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/GridUriDeploymentFileProcessor.java
@@ -35,6 +35,7 @@ import org.apache.commons.codec.digest.DigestUtils;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.compute.ComputeTask;
+import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.spi.IgniteSpiException;
@@ -292,29 +293,28 @@ final class GridUriDeploymentFileProcessor {
     static void cleanupUnit(ClassLoader clsLdr, IgniteLogger log) {
         assert clsLdr != null;
         assert log != null;
+        assert clsLdr instanceof URLClassLoader;
 
-        if (clsLdr instanceof URLClassLoader) {
-            URLClassLoader clsLdr0 = (URLClassLoader)clsLdr;
+        URLClassLoader clsLdr0 = (URLClassLoader)clsLdr;
 
-            U.close(clsLdr0, log);
+        U.close(clsLdr0, log);
 
-            try {
-                URL url = clsLdr0.getURLs()[0];
+        try {
+            URL url = IgniteUtils.classLoaderUrls(clsLdr)[0];
 
-                File dir = new File(url.toURI());
+            File dir = new File(url.toURI());
 
-                U.delete(dir);
+            U.delete(dir);
 
-                if (dir.getName().startsWith("dirzip_")) {
-                    File jarFile = new File(dir.getParentFile(), dir.getName().substring(7));
+            if (dir.getName().startsWith("dirzip_")) {
+                File jarFile = new File(dir.getParentFile(), dir.getName().substring(7));
 
-                    U.delete(jarFile);
-                }
-            }
-            catch (Exception e) {
-                U.error(log, "Failed to cleanup unit [clsLdr=" + clsLdr + ']', e);
+                U.delete(jarFile);
             }
         }
+        catch (Exception e) {
+            U.error(log, "Failed to cleanup unit [clsLdr=" + clsLdr + ']', e);
+        }
     }
 
     /**


[07/15] ignite git commit: IGNITE-5949: SQL: added ALTER TABLE DROP COLUMN support. This closes #3258.

Posted by sb...@apache.org.
IGNITE-5949: SQL: added ALTER TABLE DROP COLUMN support. This closes #3258.


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

Branch: refs/heads/ignite-zk
Commit: a39468733d4d472ae5054bcd55634dde73c1c5a5
Parents: df3c4df
Author: Sergey Kalashnikov <sk...@gridgain.com>
Authored: Thu Dec 28 18:42:08 2017 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu Dec 28 18:42:08 2017 +0300

----------------------------------------------------------------------
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |   2 +-
 .../processors/query/GridQueryIndexing.java     |  15 +
 .../processors/query/GridQueryProcessor.java    | 133 +++++++-
 .../internal/processors/query/QuerySchema.java  |  30 +-
 .../query/QueryTypeDescriptorImpl.java          |  19 ++
 .../internal/processors/query/QueryUtils.java   |  69 ++++
 .../SchemaAlterTableDropColumnOperation.java    |  95 ++++++
 ...IgniteClientCacheInitializationFailTest.java |   6 +
 .../processors/query/h2/IgniteH2Indexing.java   |  33 ++
 .../query/h2/ddl/DdlStatementsProcessor.java    |  55 ++++
 .../processors/query/h2/opt/GridH2Table.java    |  58 ++++
 .../h2/sql/GridSqlAlterTableDropColumn.java     | 113 +++++++
 .../query/h2/sql/GridSqlQueryParser.java        |  45 ++-
 ...ynamicColumnsAbstractConcurrentSelfTest.java | 238 +++++++++++---
 .../cache/index/DynamicColumnsAbstractTest.java |  59 +++-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  | 324 ++++++++++++++++++-
 .../IgnitePersistentStoreSchemaLoadTest.java    |   8 +-
 17 files changed, 1247 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
index fb96f31..6f6d6c5 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/JdbcErrorsAbstractSelfTest.java
@@ -115,7 +115,7 @@ public abstract class JdbcErrorsAbstractSelfTest extends GridCommonAbstractTest
      * @throws SQLException if failed.
      */
     public void testUnsupportedSql() throws SQLException {
-        checkErrorState("ALTER TABLE \"test\".Integer DROP COLUMN _key", "0A000");
+        checkErrorState("ALTER TABLE \"test\".Integer MODIFY COLUMN _key CHAR", "0A000");
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index 4610025..2a34bfc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -181,6 +181,21 @@ public interface GridQueryIndexing {
         boolean ifColNotExists) throws IgniteCheckedException;
 
     /**
+     * Drop columns from dynamic table.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table does not exist (instead of throwing an error).
+     * @param ifColExists Ignore operation if column does not exist (instead of throwing an error) - is honored only
+     *     for single column case.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("SynchronizationOnLocalVariableOrMethodParameter")
+    public void dynamicDropColumn(String schemaName, String tblName, List<String> cols, boolean ifTblExists,
+        boolean ifColExists) throws IgniteCheckedException;
+
+    /**
      * Registers cache.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 2d1e392..dbe2e9b 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
@@ -87,6 +87,7 @@ import org.apache.ignite.internal.processors.query.schema.message.SchemaOperatio
 import org.apache.ignite.internal.processors.query.schema.message.SchemaProposeDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation;
+import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
@@ -774,6 +775,16 @@ public class GridQueryProcessor extends GridProcessorAdapter {
 
                                         processDynamicAddColumn(typeDesc, opAddCol.columns());
                                     }
+                                    else if (op0 instanceof SchemaAlterTableDropColumnOperation) {
+                                        SchemaAlterTableDropColumnOperation opDropCol =
+                                            (SchemaAlterTableDropColumnOperation)op0;
+
+                                        QueryTypeDescriptorImpl typeDesc = tblTypMap.get(opDropCol.tableName());
+
+                                        assert typeDesc != null;
+
+                                        processDynamicDropColumn(typeDesc, opDropCol.columns());
+                                    }
                                     else
                                         assert false;
                                 }
@@ -1012,6 +1023,39 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
+        else if (op instanceof SchemaAlterTableDropColumnOperation) {
+            SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+            type = type(cacheName, op0.tableName());
+
+            if (type == null) {
+                if (op0.ifTableExists())
+                    nop = true;
+                else
+                    err = new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                        op0.tableName());
+            }
+            else {
+                for (String name : op0.columns()) {
+                    if (err != null)
+                        break;
+
+                    if (!type.hasField(name)) {
+                        if (op0.ifExists()) {
+                            assert op0.columns().size() == 1;
+
+                            nop = true;
+                        }
+                        else
+                            err = new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND, name);
+
+                        break;
+                    }
+
+                    err = QueryUtils.validateDropColumn(type, name);
+                }
+            }
+        }
         else
             err = new SchemaOperationException("Unsupported operation: " + op);
 
@@ -1142,6 +1186,53 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 }
             }
         }
+        else if (op instanceof SchemaAlterTableDropColumnOperation) {
+            SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+            QueryEntity e = tblMap.get(op0.tableName());
+
+            if (e == null) {
+                if (op0.ifTableExists())
+                    nop = true;
+                else
+                    err = new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                        op0.tableName());
+            }
+            else {
+                Map<String, String> aliases = e.getAliases();
+
+                for (String colName : op0.columns()) {
+                    if (err != null)
+                        break;
+
+                    String fldName = colName;
+
+                    if (!F.isEmpty(aliases)) {
+                        for (Map.Entry<String, String> a : aliases.entrySet()) {
+                            if (colName.equals(a.getValue())) {
+                                fldName = a.getKey();
+
+                                break;
+                            }
+                        }
+                    }
+
+                    if (!e.getFields().containsKey(fldName)) {
+                        if (op0.ifExists()) {
+                            assert op0.columns().size() == 1;
+
+                            nop = true;
+                        }
+                        else
+                            err = new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND, fldName);
+
+                        break;
+                    }
+
+                    err = QueryUtils.validateDropColumn(e, fldName, colName);
+                }
+            }
+        }
         else
             err = new SchemaOperationException("Unsupported operation: " + op);
 
@@ -1262,7 +1353,8 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                     idxs.remove(idxKey);
                 }
                 else {
-                    assert op instanceof SchemaAlterTableAddColumnOperation;
+                    assert (op instanceof SchemaAlterTableAddColumnOperation ||
+                        op instanceof SchemaAlterTableDropColumnOperation);
 
                     // No-op - all processing is done at "local" stage
                     // as we must update both table and type descriptor atomically.
@@ -1353,6 +1445,14 @@ public class GridQueryProcessor extends GridProcessorAdapter {
                 idx.dynamicAddColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
                     op0.ifNotExists());
             }
+            else if (op instanceof SchemaAlterTableDropColumnOperation) {
+                SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+                processDynamicDropColumn(type, op0.columns());
+
+                idx.dynamicDropColumn(op0.schemaName(), op0.tableName(), op0.columns(), op0.ifTableExists(),
+                    op0.ifExists());
+            }
             else
                 throw new SchemaOperationException("Unsupported operation: " + op);
         }
@@ -2226,6 +2326,24 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Entry point for drop column procedure.
+     *
+     * @param schemaName Schema name.
+     * @param tblName Target table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table doesn't exist.
+     * @param ifExists Ignore operation if column does not exist.
+     */
+    public IgniteInternalFuture<?> dynamicColumnRemove(String cacheName, String schemaName, String tblName,
+        List<String> cols, boolean ifTblExists, boolean ifExists) {
+
+        SchemaAlterTableDropColumnOperation op = new SchemaAlterTableDropColumnOperation(UUID.randomUUID(), cacheName,
+            schemaName, tblName, cols, ifTblExists, ifExists);
+
+        return startIndexOperationDistributed(op);
+    }
+
+    /**
      * Start distributed index change operation.
      *
      * @param op Operation.
@@ -2317,6 +2435,19 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Remove fields from type descriptor.
+     *
+     * @param d Type descriptor to update.
+     * @param cols Columns to remove.
+     * @throws IgniteCheckedException
+     */
+    private void processDynamicDropColumn(QueryTypeDescriptorImpl d, List<String> cols)
+        throws IgniteCheckedException {
+        for (String field : cols)
+            d.removeProperty(field);
+    }
+
+    /**
      *
      * @param cacheName Cache name.
      * @param sql Query.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
index 62a9ecd..5cbae29 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QuerySchema.java
@@ -30,6 +30,7 @@ import org.apache.ignite.cache.QueryIndex;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAbstractOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableAddColumnOperation;
+import org.apache.ignite.internal.processors.query.schema.operation.SchemaAlterTableDropColumnOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexCreateOperation;
 import org.apache.ignite.internal.processors.query.schema.operation.SchemaIndexDropOperation;
 import org.apache.ignite.internal.util.typedef.F;
@@ -148,9 +149,7 @@ public class QuerySchema implements Serializable {
                     }
                 }
             }
-            else {
-                assert op instanceof SchemaAlterTableAddColumnOperation;
-
+            else if (op instanceof SchemaAlterTableAddColumnOperation) {
                 SchemaAlterTableAddColumnOperation op0 = (SchemaAlterTableAddColumnOperation)op;
 
                 int targetIdx = -1;
@@ -199,6 +198,31 @@ public class QuerySchema implements Serializable {
                 if (replaceTarget)
                     ((List<QueryEntity>)entities).set(targetIdx, target);
             }
+            else {
+                assert op instanceof SchemaAlterTableDropColumnOperation;
+
+                SchemaAlterTableDropColumnOperation op0 = (SchemaAlterTableDropColumnOperation)op;
+
+                int targetIdx = -1;
+
+                for (int i = 0; i < entities.size(); i++) {
+                    QueryEntity entity = ((List<QueryEntity>)entities).get(i);
+
+                    if (F.eq(entity.getTableName(), op0.tableName())) {
+                        targetIdx = i;
+
+                        break;
+                    }
+                }
+
+                if (targetIdx == -1)
+                    return;
+
+                QueryEntity entity = ((List<QueryEntity>)entities).get(targetIdx);
+
+                for (String field : op0.columns())
+                    entity.getFields().remove(field);
+            }
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
index 72adefd..de58a4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/QueryTypeDescriptorImpl.java
@@ -384,6 +384,25 @@ public class QueryTypeDescriptorImpl implements GridQueryTypeDescriptor {
     }
 
     /**
+     * Removes a property with specified name.
+     *
+     * @param name Name of a property to remove.
+     */
+    public void removeProperty(String name) throws IgniteCheckedException {
+        GridQueryProperty prop = props.remove(name);
+
+        if (prop == null)
+            throw new IgniteCheckedException("Property with name '" + name + "' does not exist.");
+
+        if (validateProps != null)
+            validateProps.remove(prop);
+
+        uppercaseProps.remove(name.toUpperCase());
+
+        fields.remove(name);
+    }
+
+    /**
      * @param schemaName Schema name.
      */
     public void schemaName(String schemaName) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 9584e05..91509f4 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
@@ -1238,6 +1238,75 @@ public class QueryUtils {
     }
 
     /**
+     * Checks if given column can be removed from table using its {@link QueryEntity}.
+     *
+     * @param entity Query entity.
+     * @param fieldName Name of the field of the key or value object.
+     * @param colName Name of the column.
+     * @return {@code null} if it's OK to remove the column and exception otherwise.
+     */
+    public static SchemaOperationException validateDropColumn(QueryEntity entity, String fieldName, String colName) {
+        if (F.eq(fieldName, entity.getKeyFieldName()) || KEY_FIELD_NAME.equalsIgnoreCase(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache key");
+
+        if (F.eq(fieldName, entity.getValueFieldName()) || VAL_FIELD_NAME.equalsIgnoreCase(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache value");
+
+        Set<String> keyFields = entity.getKeyFields();
+
+        if (keyFields != null && keyFields.contains(fieldName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it is a part of a cache key");
+
+        Collection<QueryIndex> indexes = entity.getIndexes();
+
+        if (indexes != null) {
+            for (QueryIndex idxDesc : indexes) {
+                if (idxDesc.getFields().containsKey(fieldName))
+                    return new SchemaOperationException("Cannot drop column \"" + colName +
+                        "\" because an index exists (\"" + idxDesc.getName() + "\") that uses the column.");
+            }
+        }
+
+        return null;
+    }
+
+    /**
+     * Checks if given column can be removed from the table using its {@link GridQueryTypeDescriptor}.
+     *
+     * @param type Type descriptor.
+     * @param colName Name of the column.
+     * @return {@code null} if it's OK to remove the column and exception otherwise.
+     */
+    public static SchemaOperationException validateDropColumn(GridQueryTypeDescriptor type, String colName) {
+        if (F.eq(colName, type.keyFieldName()) || KEY_FIELD_NAME.equalsIgnoreCase(colName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache key");
+
+        if (F.eq(colName, type.valueFieldName()) || VAL_FIELD_NAME.equalsIgnoreCase(colName))
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it represents an entire cache value");
+
+        GridQueryProperty prop = type.property(colName);
+
+        if (prop != null && prop.key())
+            return new SchemaOperationException("Cannot drop column \"" + colName +
+                "\" because it is a part of a cache key");
+
+        Collection<GridQueryIndexDescriptor> indexes = type.indexes().values();
+
+        for (GridQueryIndexDescriptor idxDesc : indexes) {
+            if (idxDesc.fields().contains(colName))
+                return new SchemaOperationException("Cannot drop column \"" + colName +
+                    "\" because an index exists (\"" + idxDesc.name() + "\") that uses the column.");
+        }
+
+        return null;
+    }
+
+    /**
      * Private constructor.
      */
     private QueryUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
new file mode 100644
index 0000000..0437548
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/schema/operation/SchemaAlterTableDropColumnOperation.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query.schema.operation;
+
+import java.util.List;
+import java.util.UUID;
+import org.apache.ignite.internal.util.typedef.internal.S;
+
+/**
+ * Schema alter table drop column operation.
+ */
+public class SchemaAlterTableDropColumnOperation extends SchemaAbstractAlterTableOperation {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Target table name. */
+    private final String tblName;
+
+    /** Columns to drop. */
+    private final List<String> cols;
+
+    /** Ignore operation if target table doesn't exist. */
+    private final boolean ifTblExists;
+
+    /** Ignore operation if column does not exist. */
+    private final boolean ifExists;
+
+    /**
+     * Constructor.
+     *
+     * @param opId Operation id.
+     * @param schemaName Schema name.
+     * @param tblName Target table name.
+     * @param cols Columns to drop.
+     * @param ifTblExists Ignore operation if target table doesn't exist.
+     * @param ifExists Ignore operation if column does not exist.
+     */
+    public SchemaAlterTableDropColumnOperation(UUID opId, String cacheName, String schemaName, String tblName,
+        List<String> cols, boolean ifTblExists, boolean ifExists) {
+        super(opId, cacheName, schemaName);
+
+        this.tblName = tblName;
+        this.cols = cols;
+        this.ifTblExists = ifTblExists;
+        this.ifExists = ifExists;
+    }
+
+    /**
+     * @return Ignore operation if table doesn't exist.
+     */
+    public boolean ifTableExists() {
+        return ifTblExists;
+    }
+
+    /**
+     * @return Columns to drop.
+     */
+    public List<String> columns() {
+        return cols;
+    }
+
+    /**
+     * @return Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @return Target table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(SchemaAlterTableDropColumnOperation.class, this, "parent", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
index 366230d..4b93a46 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientCacheInitializationFailTest.java
@@ -292,6 +292,12 @@ public class IgniteClientCacheInitializationFailTest extends GridCommonAbstractT
         }
 
         /** {@inheritDoc} */
+        @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols,
+            boolean ifTblExists, boolean ifColExists) throws IgniteCheckedException {
+            // No-op
+        }
+
+        /** {@inheritDoc} */
         @Override public void registerCache(String cacheName, String schemaName,
             GridCacheContext<?, ?> cctx) throws IgniteCheckedException {
             if (FAILED_CACHES.contains(cctx.name()) && cctx.kernalContext().clientNode())

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 6fdcd27..7c451a5 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -144,6 +144,7 @@ import org.h2.api.ErrorCode;
 import org.h2.api.JavaObjectSerializer;
 import org.h2.command.Prepared;
 import org.h2.command.dml.Insert;
+import org.h2.command.dml.NoOperation;
 import org.h2.engine.Session;
 import org.h2.engine.SysProperties;
 import org.h2.index.Index;
@@ -764,6 +765,27 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         clearCachedQueries();
     }
 
+    /** {@inheritDoc} */
+    @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols, boolean ifTblExists,
+        boolean ifColExists) throws IgniteCheckedException {
+        // Locate table.
+        H2Schema schema = schemas.get(schemaName);
+
+        H2TableDescriptor desc = (schema != null ? schema.tableByName(tblName) : null);
+
+        if (desc == null) {
+            if (!ifTblExists)
+                throw new IgniteCheckedException("Table not found in internal H2 database [schemaName=" + schemaName +
+                    ",tblName=" + tblName + ']');
+            else
+                return;
+        }
+
+        desc.table().dropColumns(cols, ifColExists);
+
+        clearCachedQueries();
+    }
+
     /**
      * Execute DDL command.
      *
@@ -1576,6 +1598,17 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             throw new IgniteSQLException("Failed to execute DDL statement [stmt=" + sqlQry + ']', e);
                         }
                     }
+
+                    if (prepared instanceof NoOperation) {
+                        QueryCursorImpl<List<?>> resCur = (QueryCursorImpl<List<?>>)new QueryCursorImpl(
+                            Collections.singletonList(Collections.singletonList(0L)), null, false);
+
+                        resCur.fieldsMeta(UPDATE_RESULT_META);
+
+                        res.add(resCur);
+
+                        continue;
+                    }
                 }
 
                 assert twoStepQry != null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/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 68aab49..b198922 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
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.h2.opt.GridH2Table;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlterTableAddColumn;
+import org.apache.ignite.internal.processors.query.h2.sql.GridSqlAlterTableDropColumn;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlColumn;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateIndex;
 import org.apache.ignite.internal.processors.query.h2.sql.GridSqlCreateTable;
@@ -364,6 +365,60 @@ public class DdlStatementsProcessor {
                     }
                 }
             }
+            else if (stmt0 instanceof GridSqlAlterTableDropColumn) {
+                GridSqlAlterTableDropColumn cmd = (GridSqlAlterTableDropColumn)stmt0;
+
+                GridH2Table tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
+
+                if (tbl == null && cmd.ifTableExists()) {
+                    ctx.cache().createMissingQueryCaches();
+
+                    tbl = idx.dataTable(cmd.schemaName(), cmd.tableName());
+                }
+
+                if (tbl == null) {
+                    if (!cmd.ifTableExists())
+                        throw new SchemaOperationException(SchemaOperationException.CODE_TABLE_NOT_FOUND,
+                            cmd.tableName());
+                }
+                else {
+                    assert tbl.rowDescriptor() != null;
+
+                    if (QueryUtils.isSqlType(tbl.rowDescriptor().type().valueClass()))
+                        throw new SchemaOperationException("Cannot drop column(s) because table was created " +
+                            "with " + PARAM_WRAP_VALUE + "=false option.");
+
+                    List<String> cols = new ArrayList<>(cmd.columns().length);
+
+                    GridQueryTypeDescriptor type = tbl.rowDescriptor().type();
+
+                    for (String colName : cmd.columns()) {
+                        if (!tbl.doesColumnExist(colName)) {
+                            if ((!cmd.ifExists() || cmd.columns().length != 1)) {
+                                throw new SchemaOperationException(SchemaOperationException.CODE_COLUMN_NOT_FOUND,
+                                    colName);
+                            }
+                            else {
+                                cols = null;
+
+                                break;
+                            }
+                        }
+
+                        SchemaOperationException err = QueryUtils.validateDropColumn(type, colName);
+
+                        if (err != null)
+                            throw err;
+
+                        cols.add(colName);
+                    }
+
+                    if (cols != null) {
+                        fut = ctx.query().dynamicColumnRemove(tbl.cacheName(), cmd.schemaName(),
+                            type.tableName(), cols, cmd.ifTableExists(), cmd.ifExists());
+                    }
+                }
+            }
             else
                 throw new IgniteSQLException("Unsupported DDL operation: " + sql,
                     IgniteQueryErrorCode.UNSUPPORTED_OPERATION);

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
index cdffa16..c803a36 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Table.java
@@ -57,6 +57,7 @@ import org.jsr166.ConcurrentHashMap8;
 import org.jsr166.LongAdder8;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.KEY_COL;
 
 /**
@@ -939,6 +940,63 @@ public class GridH2Table extends TableBase {
         }
     }
 
+    /**
+     *
+     * @param cols
+     * @param ifExists
+     */
+    public void dropColumns(List<String> cols, boolean ifExists) {
+        assert !ifExists || cols.size() == 1;
+
+        lock(true);
+
+        try {
+            int size = columns.length;
+
+            for (String name : cols) {
+                if (!doesColumnExist(name)) {
+                    if (ifExists && cols.size() == 1)
+                        return;
+                    else
+                        throw new IgniteSQLException("Column does not exist [tblName=" + getName() +
+                            ", colName=" + name + ']');
+                }
+
+                size --;
+            }
+
+            assert size > DEFAULT_COLUMNS_COUNT;
+
+            Column[] newCols = new Column[size];
+
+            int dst = 0;
+
+            for (int i = 0; i < columns.length; i++) {
+                Column column = columns[i];
+
+                for (String name : cols) {
+                    if (F.eq(name, column.getName())) {
+                        column = null;
+
+                        break;
+                    }
+                }
+
+                if (column != null)
+                    newCols[dst++] = column;
+            }
+
+            setColumns(newCols);
+
+            desc.refreshMetadataFromTypeDescriptor();
+
+            setModified();
+        }
+        finally {
+            unlock(true);
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public Column[] getColumns() {
         Boolean insertHack = INSERT_HACK.get();

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
new file mode 100644
index 0000000..6f8c923
--- /dev/null
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlAlterTableDropColumn.java
@@ -0,0 +1,113 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.query.h2.sql;
+
+/**
+ * ALTER TABLE DROP COLUMN command data holder.
+ */
+public class GridSqlAlterTableDropColumn extends GridSqlStatement {
+    /** Schema name. */
+    private String schemaName;
+
+    /** Target table name. */
+    private String tblName;
+
+    /** Columns to drop. */
+    private String[] cols;
+
+    /** Quietly abort this command if column does not exist (honored only in single column case). */
+    private boolean ifExists;
+
+    /** Quietly abort this command if target table does not exist. */
+    private boolean ifTblExists;
+
+    /**
+     * @return Columns to drop.
+     */
+    public String[] columns() {
+        return cols;
+    }
+
+    /**
+     * @param cols Columns to drop.
+     */
+    public void columns(String[] cols) {
+        this.cols = cols;
+    }
+
+    /**
+     * @return Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public boolean ifExists() {
+        return ifExists;
+    }
+
+    /**
+     * @param ifExists Quietly abort this command if column does not exist (honored only in single column case).
+     */
+    public void ifExists(boolean ifExists) {
+        this.ifExists = ifExists;
+    }
+
+    /**
+     * @return Quietly abort this command if target table does not exist.
+     */
+    public boolean ifTableExists() {
+        return ifTblExists;
+    }
+
+    /**
+     * @param ifTblExists Quietly abort this command if target table does not exist.
+     */
+    public void ifTableExists(boolean ifTblExists) {
+        this.ifTblExists = ifTblExists;
+    }
+
+    /**
+     * @return Target table name.
+     */
+    public String tableName() {
+        return tblName;
+    }
+
+    /**
+     * @param tblName Target table name.
+     */
+    public void tableName(String tblName) {
+        this.tblName = tblName;
+    }
+
+    /**
+     * @return Schema name.
+     */
+    public String schemaName() {
+        return schemaName;
+    }
+
+    /**
+     * @param schemaName Schema name.
+     */
+    public void schemaName(String schemaName) {
+        this.schemaName = schemaName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSQL() {
+        throw new UnsupportedOperationException();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
index 3b19e39..388231f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQueryParser.java
@@ -430,6 +430,10 @@ public class GridSqlQueryParser {
         getter(AlterTableAlterColumn.class, "columnsToAdd");
 
     /** */
+    private static final Getter<AlterTableAlterColumn, ArrayList<Column>> ALTER_COLUMN_REMOVE_COLS =
+        getter(AlterTableAlterColumn.class, "columnsToRemove");
+
+    /** */
     private static final Getter<AlterTableAlterColumn, Boolean> ALTER_COLUMN_IF_NOT_EXISTS =
         getter(AlterTableAlterColumn.class, "ifNotExists");
 
@@ -1150,6 +1154,9 @@ public class GridSqlQueryParser {
             case CommandInterface.ALTER_TABLE_ADD_COLUMN:
                 return parseAddColumn(stmt);
 
+            case CommandInterface.ALTER_TABLE_DROP_COLUMN:
+                return parseDropColumn(stmt);
+
             default: {
                 String stmtName = null;
 
@@ -1164,11 +1171,6 @@ public class GridSqlQueryParser {
                         stmtName = "ALTER COLUMN";
 
                         break;
-
-                    case CommandInterface.ALTER_TABLE_DROP_COLUMN:
-                        stmtName = "DROP COLUMN";
-
-                        break;
                 }
 
                 if (stmtName == null) {
@@ -1258,6 +1260,39 @@ public class GridSqlQueryParser {
     }
 
     /**
+     * Parse {@code ALTER TABLE ... DROP COLUMN} statement.
+     * @param dropCol H2 statement.
+     * @see <a href="http://www.h2database.com/html/grammar.html#alter_table_add"></a>
+     */
+    private GridSqlStatement parseDropColumn(AlterTableAlterColumn dropCol) {
+        assert dropCol.getType() == CommandInterface.ALTER_TABLE_DROP_COLUMN;
+
+        GridSqlAlterTableDropColumn res = new GridSqlAlterTableDropColumn();
+
+        ArrayList<Column> h2DropCols = ALTER_COLUMN_REMOVE_COLS.get(dropCol);
+
+        String[] gridDropCols = new String[h2DropCols.size()];
+
+        for (int i = 0; i < h2DropCols.size(); i++)
+            gridDropCols[i] = h2DropCols.get(i).getName();
+
+        res.columns(gridDropCols);
+
+        if (gridDropCols.length == 1)
+            res.ifExists(!ALTER_COLUMN_IF_NOT_EXISTS.get(dropCol));
+
+        res.ifTableExists(ALTER_COLUMN_IF_TBL_EXISTS.get(dropCol));
+
+        Schema schema = SCHEMA_COMMAND_SCHEMA.get(dropCol);
+
+        res.schemaName(schema.getName());
+
+        res.tableName(ALTER_COLUMN_TBL_NAME.get(dropCol));
+
+        return res;
+    }
+
+    /**
      * @param name Param name.
      * @param val Param value.
      * @param res Table params to update.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
index 145947a..31ccd67 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractConcurrentSelfTest.java
@@ -54,6 +54,7 @@ import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
 import org.apache.ignite.internal.processors.query.schema.SchemaOperationException;
 import org.apache.ignite.internal.processors.query.schema.message.SchemaFinishDiscoveryMessage;
+import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T3;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -84,6 +85,9 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     /** SQL statement to create test table accompanied by template specification. */
     private final String createSql;
 
+    /** SQL statement to create test table with additional columns. */
+    private final String createSql4Cols;
+
     /** Latches to block certain index operations. */
     private static final ConcurrentHashMap<UUID, T3<CountDownLatch, AtomicBoolean, CountDownLatch>> BLOCKS =
         new ConcurrentHashMap<>();
@@ -103,7 +107,11 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     DynamicColumnsAbstractConcurrentSelfTest(CacheMode cacheMode, CacheAtomicityMode atomicityMode) {
         this.cacheMode = cacheMode;
         this.atomicityMode = atomicityMode;
-        createSql =  CREATE_SQL + " WITH \"template=TPL\"";
+
+        final String template = " WITH \"template=TPL\"";
+
+        createSql =  CREATE_SQL + template;
+        createSql4Cols = CREATE_SQL_4_COLS + template;
     }
 
     /** {@inheritDoc} */
@@ -143,7 +151,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testCoordinatorChange() throws Exception {
+    public void testAddColumnCoordinatorChange() throws Exception {
+        checkCoordinatorChange(true);
+    }
+
+    /**
+     * Make sure that coordinator migrates correctly between nodes.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropColumnCoordinatorChange() throws Exception {
+        checkCoordinatorChange(false);
+    }
+
+    /**
+     * Make sure that coordinator migrates correctly between nodes.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    public void checkCoordinatorChange(boolean addOrRemove) throws Exception {
         CountDownLatch finishLatch = new CountDownLatch(2);
 
         // Start servers.
@@ -159,12 +186,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(cli);
 
-        run(cli, createSql);
+        run(cli, addOrRemove ? createSql : createSql4Cols);
 
         // Test migration between normal servers.
         CountDownLatch idxLatch = blockIndexing(srv1Id);
 
-        IgniteInternalFuture<?> colFut1 = addCols(cli, QueryUtils.DFLT_SCHEMA, c("age", Integer.class.getName()));
+        IgniteInternalFuture<?> colFut1 = addOrRemove ?
+            addCols(cli, QueryUtils.DFLT_SCHEMA, c("age", Integer.class.getName())) :
+            dropCols(cli, QueryUtils.DFLT_SCHEMA, "AGE");
 
         U.await(idxLatch);
 
@@ -176,12 +205,15 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         colFut1.get();
 
         // Port number is for srv2.
-        checkTableState(srv2, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("age", Integer.class.getName()));
+        checkTableState(srv2, QueryUtils.DFLT_SCHEMA, TBL_NAME,
+            addOrRemove ? c("age", Integer.class.getName()) : c("CITY", String.class.getName()));
 
         // Test migration from normal server to non-affinity server.
         idxLatch = blockIndexing(srv2Id);
 
-        IgniteInternalFuture<?> colFut2 = addCols(cli, QueryUtils.DFLT_SCHEMA, c("city", String.class.getName()));
+        IgniteInternalFuture<?> colFut2 = addOrRemove ?
+            addCols(cli, QueryUtils.DFLT_SCHEMA, c("city", String.class.getName())) :
+            dropCols(cli, QueryUtils.DFLT_SCHEMA, "CITY");
 
         idxLatch.countDown();
 
@@ -196,7 +228,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         srv3.cache(QueryUtils.createTableCacheName(QueryUtils.DFLT_SCHEMA, "PERSON"));
 
         // Port number is for srv3.
-        checkTableState(srv3, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("city", String.class.getName()));
+        checkTableState(srv3, QueryUtils.DFLT_SCHEMA, TBL_NAME,
+            addOrRemove ? c("city", String.class.getName()) : c("NAME", String.class.getName()));
     }
 
     /**
@@ -220,12 +253,16 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
-        QueryField c1 = c("age", Integer.class.getName());
-        QueryField c2 = c("city", String.class.getName());
+        QueryField c0 = c("ID", Integer.class.getName());
+        QueryField c1 = c("NAME", String.class.getName());
+        QueryField c2 = c("age", Integer.class.getName());
+        QueryField c3 = c("city", String.class.getName());
+
+        IgniteInternalFuture<?> colFut1 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c2);
 
-        IgniteInternalFuture<?> colFut1 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c1);
+        IgniteInternalFuture<?> colFut2 = dropCols(srv1, QueryUtils.DFLT_SCHEMA, c1.name());
 
-        IgniteInternalFuture<?> colFut2 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c2);
+        IgniteInternalFuture<?> colFut3 = addCols(srv1, QueryUtils.DFLT_SCHEMA, c3);
 
         U.await(idxLatch);
 
@@ -236,36 +273,58 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         assert !colFut1.isDone();
         assert !colFut2.isDone();
+        assert !colFut3.isDone();
 
         unblockIndexing(srv1);
 
         colFut1.get();
         colFut2.get();
+        colFut3.get();
 
         U.await(finishLatch);
 
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c1, c2);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c0, c2, c3);
+    }
+
+    /**
+     * Test node join on pending add column operation.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNodeJoinOnPendingAddOperation() throws Exception {
+        checkNodeJoinOnPendingOperation(true);
     }
 
     /**
-     * Test node join on pending operation.
+     * Test node join on pending drop column operation.
      *
      * @throws Exception If failed.
      */
-    public void testNodeJoinOnPendingOperation() throws Exception {
+    public void testNodeJoinOnPendingDropOperation() throws Exception {
+        checkNodeJoinOnPendingOperation(false);
+    }
+
+    /**
+     * Check node join on pending operation.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    private void checkNodeJoinOnPendingOperation(boolean addOrRemove) throws Exception {
         CountDownLatch finishLatch = new CountDownLatch(4);
 
         IgniteEx srv1 = ignitionStart(serverConfiguration(1), finishLatch);
 
         createSqlCache(srv1);
 
-        run(srv1, createSql);
+        run(srv1, addOrRemove ? createSql : createSql4Cols);
 
         CountDownLatch idxLatch = blockIndexing(srv1);
 
-        QueryField c = c("age", Integer.class.getName());
+        QueryField c = c("AGE", Integer.class.getName());
 
-        IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        IgniteInternalFuture<?> idxFut = addOrRemove ? addCols(srv1, QueryUtils.DFLT_SCHEMA, c) :
+            dropCols(srv1, QueryUtils.DFLT_SCHEMA, "CITY");
 
         U.await(idxLatch);
 
@@ -285,7 +344,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * PUT/REMOVE data from cache and add column concurrently.
+     * PUT/REMOVE data from cache and add/drop column concurrently.
      *
      * @throws Exception If failed,
      */
@@ -302,7 +361,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         createSqlCache(srv1);
 
-        run(srv1, createSql);
+        run(srv1, createSql4Cols);
 
         // Start data change operations from several threads.
         final AtomicBoolean stopped = new AtomicBoolean();
@@ -332,6 +391,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         addCols(srv1, QueryUtils.DFLT_SCHEMA, c("v", Integer.class.getName())).get();
 
+        dropCols(srv1, QueryUtils.DFLT_SCHEMA, "CITY").get();
+
         // Stop updates once index is ready.
         stopped.set(true);
 
@@ -340,7 +401,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         finishLatch.await();
 
         // Make sure new column is there.
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("v", Integer.class.getName()));
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c("AGE", Integer.class.getName()),
+            c("v", Integer.class.getName()));
 
         run(srv1, "update person set \"v\" = case when mod(id, 2) <> 0 then substring(name, 7, length(name) - 6) " +
             "else null end");
@@ -417,7 +479,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testConcurrentRebalance() throws Exception {
+    public void testAddConcurrentRebalance() throws Exception {
+        checkConcurrentRebalance(true);
+    }
+
+    /**
+     * Test index consistency on re-balance.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropConcurrentRebalance() throws Exception {
+        checkConcurrentRebalance(false);
+    }
+
+    /**
+     * Check index consistency on re-balance.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    public void checkConcurrentRebalance(boolean addOrRemove) throws Exception {
         // Start cache and populate it with data.
         IgniteEx srv1 = ignitionStart(serverConfiguration(1));
         Ignite srv2 = ignitionStart(serverConfiguration(2));
@@ -436,7 +517,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         QueryField c = c("salary", Double.class.getName());
 
-        final IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        final IgniteInternalFuture<?> idxFut = addOrRemove ?
+            addCols(srv1, QueryUtils.DFLT_SCHEMA, c) : dropCols(srv1, QueryUtils.DFLT_SCHEMA, "NAME");
 
         U.await(idxLatch1);
         U.await(idxLatch2);
@@ -454,7 +536,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Validate index state.
         idxFut.get();
 
-        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, c);
+        checkTableState(srv1, QueryUtils.DFLT_SCHEMA, TBL_NAME, addOrRemove ? c : c("ID", Integer.class.getName()));
     }
 
     /**
@@ -473,7 +555,26 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
      *
      * @throws Exception If failed.
      */
-    public void testConcurrentCacheDestroy() throws Exception {
+    public void testAddConcurrentCacheDestroy() throws Exception {
+        checkConcurrentCacheDestroy(true);
+    }
+
+    /**
+     * Check what happens in case cache is destroyed before operation is started.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDropConcurrentCacheDestroy() throws Exception {
+        checkConcurrentCacheDestroy(false);
+    }
+
+    /**
+     * Check what happens in case cache is destroyed before operation is started.
+     *
+     * @param addOrRemove Pass {@code true} to check add column. Otherwise, drop column is checked.
+     * @throws Exception If failed.
+     */
+    private void checkConcurrentCacheDestroy(boolean addOrRemove) throws Exception {
         // Start complex topology.
         Ignite srv1 = ignitionStart(serverConfiguration(1));
 
@@ -482,6 +583,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         Ignite cli = ignitionStart(clientConfiguration(4));
 
+        waitForDiscovery(srv1, grid(2), grid(3), cli);
+
         // Start cache and populate it with data.
         createSqlCache(cli);
 
@@ -494,7 +597,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         QueryField c = c("city", String.class.getName());
 
-        final IgniteInternalFuture<?> idxFut = addCols(srv1, QueryUtils.DFLT_SCHEMA, c);
+        final IgniteInternalFuture<?> idxFut = addOrRemove ?
+            addCols(srv1, QueryUtils.DFLT_SCHEMA, c) : dropCols(srv1, QueryUtils.DFLT_SCHEMA, "NAME");
 
         idxLatch.await();
 
@@ -515,8 +619,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * Make sure that contended operations on the same index from different nodes do not hang when we issue both
-     * CREATE/DROP and SELECT statements.
+     * Make sure that contended operations on the same table from different nodes do not hang when we issue both
+     * ADD/DROP COLUMN and SELECT statements.
      *
      * @throws Exception If failed.
      */
@@ -538,16 +642,32 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         final AtomicInteger dynColCnt = new AtomicInteger();
 
+        final GridConcurrentHashSet<Integer> fields = new GridConcurrentHashSet<>();
+
         IgniteInternalFuture fut = multithreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
-                        dynColCnt.getAndIncrement(), Integer.class.getName()));
+                    IgniteInternalFuture fut;
+
+                    int fieldNum = ThreadLocalRandom.current().nextInt(0, dynColCnt.get() + 1);
+
+                    boolean removed = fields.remove(fieldNum);
+
+                    if (removed)
+                        fut = dropCols(node, QueryUtils.DFLT_SCHEMA, "newCol" + fieldNum);
+                    else {
+                        fieldNum = dynColCnt.getAndIncrement();
 
+                        fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + fieldNum,
+                            Integer.class.getName()));
+                    }
                     try {
                         fut.get();
+
+                        if (!removed)
+                            fields.add(fieldNum);
                     }
                     catch (SchemaOperationException e) {
                         // No-op.
@@ -657,6 +777,8 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         reconnectClientNode(srv, cli, restartCache, dynamicCache, new RunnableX() {
             @Override public void run() throws Exception {
                 addCols(srv, schemaName, cols).get();
+
+                dropCols(srv, schemaName, "NAME").get();
             }
         });
 
@@ -713,7 +835,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
-     * Test concurrent node start/stop along with index operations. Nothing should hang.
+     * Test concurrent node start/stop along with add/drop column operations. Nothing should hang.
      *
      * @throws Exception If failed.
      */
@@ -782,16 +904,32 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
             }
         }, 1);
 
+        final GridConcurrentHashSet<Integer> fields = new GridConcurrentHashSet<>();
+
         IgniteInternalFuture idxFut = multithreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 while (!stopped.get()) {
                     Ignite node = grid(ThreadLocalRandom.current().nextInt(1, 5));
 
-                    IgniteInternalFuture fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" +
-                        dynColCnt.getAndIncrement(), Integer.class.getName()));
+                    IgniteInternalFuture fut;
+
+                    int fieldNum = ThreadLocalRandom.current().nextInt(0, dynColCnt.get() + 1);
+
+                    boolean removed = fields.remove(fieldNum);
+
+                    if (removed)
+                        fut = dropCols(node, QueryUtils.DFLT_SCHEMA, "newCol" + fieldNum);
+                    else {
+                        fieldNum = dynColCnt.getAndIncrement();
+
+                        fut = addCols(node, QueryUtils.DFLT_SCHEMA, c("newCol" + fieldNum, Integer.class.getName()));
+                    }
 
                     try {
                         fut.get();
+
+                        if (!removed)
+                            fields.add(fieldNum);
                     }
                     catch (SchemaOperationException e) {
                         // No-op.
@@ -816,7 +954,7 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
         // Make sure cache is operational at this point.
         createSqlCache(cli);
 
-        QueryField[] expCols = new QueryField[dynColCnt.get()];
+        QueryField[] expCols = new QueryField[fields.size()];
 
         // Too many index columns kills indexing internals, have to limit number of the columns
         // to build the index on.
@@ -828,8 +966,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
         String idxQry = "CREATE INDEX idx ON " + TBL_NAME + '(';
 
+        Integer[] sorted = fields.toArray(new Integer[fields.size()]);
+
+        Arrays.sort(sorted);
+
         for (int i = 0; i < expCols.length; i++) {
-            expCols[i] = c("newCol" + i, Integer.class.getName());
+            int fieldNum = sorted[i];
+
+            expCols[i] = c("newCol" + fieldNum, Integer.class.getName());
 
             if (i >= idxColsCnt)
                 continue;
@@ -840,9 +984,9 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
                 idxQry += ", ";
             }
 
-            updQry += "\"newCol" + i + "\" = id + ?";
+            updQry += "\"newCol" + fieldNum + "\" = id + ?";
 
-            idxQry += "\"newCol" + i + '"';
+            idxQry += "\"newCol" + fieldNum + '"';
 
             args[i] = i;
         }
@@ -952,6 +1096,14 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
 
             super.dynamicAddColumn(schemaName, tblName, cols, ifTblExists, ifColNotExists);
         }
+
+        /** {@inheritDoc} */
+        @Override public void dynamicDropColumn(String schemaName, String tblName, List<String> cols,
+            boolean ifTblExists, boolean ifColExists) throws IgniteCheckedException {
+            awaitIndexing(ctx.localNodeId());
+
+            super.dynamicDropColumn(schemaName, tblName, cols, ifTblExists, ifColExists);
+        }
     }
 
     /**
@@ -969,6 +1121,20 @@ public abstract class DynamicColumnsAbstractConcurrentSelfTest extends DynamicCo
     }
 
     /**
+     *
+     * @param node Target node.
+     * @param schemaName Schema name.
+     * @param flds Columns to remove.
+     * @return DDL operation future.
+     */
+    private static IgniteInternalFuture<?> dropCols(Ignite node, String schemaName, String... flds) {
+        final String cacheName = F.eq(schemaName, QueryUtils.DFLT_SCHEMA) ? CACHE_NAME : "idx";
+
+        return ((IgniteEx)node).context().query().dynamicColumnRemove(cacheName, schemaName, TBL_NAME,
+            Arrays.asList(flds), false, false);
+    }
+
+    /**
      * Start SQL cache on given node.
      * @param node Node to create cache on.
      * @return Created cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
index 611f857..e507217 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/DynamicColumnsAbstractTest.java
@@ -54,6 +54,10 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
     /** SQL to create test table. */
     final static String CREATE_SQL = "CREATE TABLE IF NOT EXISTS Person (id int primary key, name varchar)";
 
+    /** SQL to create test table with additional columns to drop. */
+    final static String CREATE_SQL_4_COLS = "CREATE TABLE IF NOT EXISTS Person (id int primary key, " +
+        "name varchar, age int, city varchar)";
+
     /** SQL to drop test table. */
     final static String DROP_SQL = "DROP TABLE Person";
 
@@ -66,8 +70,9 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
      * @param schemaName Schema name to look for the table in.
      * @param tblName Table name to check.
      * @param cols Columns whose presence must be checked.
+     * @return Number of other columns.
      */
-    static void checkTableState(IgniteEx node, String schemaName, String tblName, QueryField... cols)
+    static int checkTableState(IgniteEx node, String schemaName, String tblName, QueryField... cols)
         throws SQLException {
         List<QueryField> flds = new ArrayList<>();
 
@@ -103,6 +108,39 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
 
             assertEquals(exp.isNullable(), act.isNullable());
         }
+
+        return flds.size() - cols.length;
+    }
+
+    /**
+     * Checks presence of specific table column and returns it.
+     *
+     * @param node Node to check.
+     * @param schemaName Schema name to look for the table in.
+     * @param tblName Table name to check.
+     * @param colName Column name whose presence must be checked.
+     * @return field or {@code null} if not found.
+     * @throws SQLException if failed.
+     */
+    static QueryField getColumnMeta(IgniteEx node, String schemaName, String tblName, String colName)
+        throws SQLException {
+        try (Connection c = connect(node)) {
+            try (ResultSet rs = c.getMetaData().getColumns(null, schemaName, tblName, colName)) {
+                while (rs.next()) {
+                    String name = rs.getString("COLUMN_NAME");
+
+                    short type = rs.getShort("DATA_TYPE");
+
+                    String typeClsName = DataType.getTypeClassName(DataType.convertSQLTypeToValueType(type));
+
+                    short nullable = rs.getShort("NULLABLE");
+
+                    return new QueryField(name, typeClsName, nullable == 1);
+                }
+            }
+        }
+
+        return null;
     }
 
     /**
@@ -203,4 +241,23 @@ public abstract class DynamicColumnsAbstractTest extends GridCommonAbstractTest
             }
         }, IgniteSQLException.class, msg);
     }
+
+    /**
+     * Run specified statement expected to throw an exception with specified class and message.
+     *
+     * @param sql Statement.
+     * @param cls Expected exception class.
+     * @param msg Expected message.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    protected void assertThrowsAnyCause(final Ignite node, final String sql, Class<? extends Throwable> cls,
+        String msg) {
+        GridTestUtils.assertThrowsAnyCause(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                run(node, sql);
+
+                return null;
+            }
+        }, cls, msg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
index eb6d7e6..4e28ae9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/index/H2DynamicColumnsAbstractBasicSelfTest.java
@@ -30,6 +30,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.processors.query.QueryField;
 import org.apache.ignite.internal.processors.query.QueryUtils;
 import org.apache.ignite.testframework.config.GridTestProperties;
+import org.h2.jdbc.JdbcSQLException;
 
 import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
 
@@ -99,10 +100,11 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
      * @param schemaName Schema name.
      * @param tblName Table name.
      * @param cols Columns to look for.
+     * @return Number of other columns.
      * @throws SQLException if failed.
      */
-    private void checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
-        checkTableState(grid(nodeIndex()), schemaName, tblName, cols);
+    private int checkTableState(String schemaName, String tblName, QueryField... cols) throws SQLException {
+        return checkTableState(grid(nodeIndex()), schemaName, tblName, cols);
     }
 
     /**
@@ -185,9 +187,11 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
 
         run(cache, "CREATE INDEX cidx2 ON City(name)");
 
-        run(cache, "INSERT INTO City(id, name, population, state) values (5, 'New York', 15000000, 'New York')," +
+        run(cache, "INSERT INTO City(id, name, population, state_name) values (5, 'New York', 15000000, 'New York')," +
             "(7, 'Denver', 3000000, 'Colorado')");
 
+        run(cache, "ALTER TABLE City DROP COLUMN state_name");
+
         List<List<?>> res = run(cache, "SELECT p.name from Person p join City c on p.city = c.name where " +
             "c.population > 5000000 order by p.name");
 
@@ -253,10 +257,10 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
 
         checkTableState("City", "CITY", c);
 
-        run(cache, "INSERT INTO \"City\".City (_key, id, name, state, population) values " +
+        run(cache, "INSERT INTO \"City\".City (_key, id, name, state_name, population) values " +
             "(1, 1, 'Washington', 'DC', 2500000)");
 
-        List<List<?>> res = run(cache, "select _key, id, name, state, population from \"City\".City");
+        List<List<?>> res = run(cache, "select _key, id, name, state_name, population from \"City\".City");
 
         assertEquals(Collections.singletonList(Arrays.asList(1, 1, "Washington", "DC", 2500000)), res);
 
@@ -340,6 +344,302 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumn() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN a");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN IF EXISTS a");
+
+            assertThrowsAnyCause("ALTER TABLE test DROP COLUMN a", JdbcSQLException.class, "Column \"A\" not found");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDroppedColumnMeta() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            QueryField fld = getColumnMeta(grid(nodeIndex()), QueryUtils.DFLT_SCHEMA, "TEST", "A");
+
+            assertEquals("A", fld.name());
+            assertEquals(Integer.class.getName(), fld.typeName());
+
+            run("ALTER TABLE test DROP COLUMN a");
+
+            assertNull(getColumnMeta(grid(nodeIndex()), QueryUtils.DFLT_SCHEMA, "TEST", "A"));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropMultipleColumns() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR, c INT)");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true),
+                new QueryField("C", Integer.class.getName(), true)));
+
+            run("ALTER TABLE test DROP COLUMN a, c");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropNonExistingColumn() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT)");
+
+            assertThrowsAnyCause("ALTER TABLE test DROP COLUMN b", JdbcSQLException.class, "Column \"B\" not found");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnNonExistingTable() throws Exception {
+        assertThrowsAnyCause("ALTER TABLE nosuchtable DROP COLUMN a", JdbcSQLException.class,
+            "Table \"NOSUCHTABLE\" not found");
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIfTableExists() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b CHAR)");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN a");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("B", String.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIfExists() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT)");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN IF EXISTS a");
+
+            run("ALTER TABLE IF EXISTS test DROP COLUMN IF EXISTS b");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnIndexPresent() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, a INT, b INT)");
+
+            run("CREATE INDEX b_index ON test(b)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN b",
+                "Cannot drop column \"B\" because an index exists (\"B_INDEX\") that uses the column.");
+
+            run("DROP INDEX b_index");
+
+            run("ALTER TABLE test DROP COLUMN b");
+
+            assertEquals(0, checkTableState(QueryUtils.DFLT_SCHEMA, "TEST",
+                new QueryField("ID", Integer.class.getName(), true),
+                new QueryField("A", Integer.class.getName(), true)));
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnOnRealClassValuedTable() throws Exception {
+        try {
+            run("CREATE TABLE test (id INT PRIMARY KEY, x VARCHAR) with \"wrap_value=false\"");
+
+            assertThrows("ALTER TABLE test DROP COLUMN x",
+                "Cannot drop column(s) because table was created with WRAP_VALUE=false option.");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsPartOfKey() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT, a INT, b CHAR, PRIMARY KEY(id, a))");
+
+            assertThrows("ALTER TABLE test DROP COLUMN a",
+                "Cannot drop column \"A\" because it is a part of a cache key");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsKey() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN id",
+                "Cannot drop column \"ID\" because it represents an entire cache key");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     *
+     * @throws Exception if failed.
+     */
+    public void testDropColumnThatIsValue() throws Exception {
+        try {
+            run("CREATE TABLE test(id INT PRIMARY KEY, a INT, b CHAR)");
+
+            assertThrows("ALTER TABLE test DROP COLUMN _val",
+                "Cannot drop column \"_VAL\" because it represents an entire cache value");
+        }
+        finally {
+            run("DROP TABLE IF EXISTS test");
+        }
+    }
+
+    /**
+     * Test that we can drop columns dynamically from tables associated
+     * with non dynamic caches storing user types as well.
+     *
+     * @throws SQLException if failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void testDropColumnFromNonDynamicCacheWithRealValueType() throws SQLException {
+        CacheConfiguration<Integer, City> ccfg = defaultCacheConfiguration().setName("City")
+            .setIndexedTypes(Integer.class, City.class);
+
+        IgniteCache<Integer, ?> cache = ignite(nodeIndex()).getOrCreateCache(ccfg);
+
+        run(cache, "INSERT INTO \"City\".City (_key, id, name, state_name) VALUES " +
+            "(1, 1, 'Washington', 'DC')");
+
+        run(cache, "ALTER TABLE \"City\".City DROP COLUMN state_name");
+
+        doSleep(500);
+
+        QueryField c = c("NAME", String.class.getName());
+
+        checkTableState("City", "CITY", c);
+
+        run(cache, "INSERT INTO \"City\".City (_key, id, name) VALUES " +
+            "(2, 2, 'New York')");
+
+        assertThrowsAnyCause("SELECT state_name FROM \"City\".City",
+            JdbcSQLException.class, "Column \"STATE_NAME\" not found");
+
+        List<List<?>> res = run(cache, "SELECT _key, id, name FROM \"City\".City WHERE id = 1");
+
+        assertEquals(Collections.singletonList(Arrays.asList(1, 1, "Washington")), res);
+
+        res = run(cache, "SELECT * FROM \"City\".City WHERE id = 2");
+
+        assertEquals(Collections.singletonList(Arrays.asList(2, "New York")), res);
+
+        if (!Boolean.valueOf(GridTestProperties.getProperty(BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER))) {
+            City city = (City)cache.get(1);
+
+            assertEquals(1, city.id());
+            assertEquals("Washington", city.name());
+            assertEquals("DC", city.state());
+
+            city = (City)cache.get(2);
+
+            assertEquals(2, city.id());
+            assertEquals("New York", city.name());
+            assertEquals(null, city.state());
+        }
+        else {
+            BinaryObject city = (BinaryObject)cache.withKeepBinary().get(1);
+
+            assertEquals(1, (int)city.field("id"));
+            assertEquals("Washington", (String)city.field("name"));
+            assertEquals("DC", (String)city.field("state"));
+
+            city = (BinaryObject)cache.withKeepBinary().get(2);
+
+            assertEquals(2, (int)city.field("id"));
+            assertEquals("New York", (String)city.field("name"));
+            assertEquals(null, (String)city.field("state"));
+        }
+
+        cache.destroy();
+    }
+
+    /**
      * Test that {@code ADD COLUMN} fails for tables that have flat value.
      * @param tblName table name.
      */
@@ -364,6 +664,18 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
     }
 
     /**
+     * Run specified statement expected to throw an exception of specified class and message.
+     *
+     * @param sql Statement.
+     * @param cls Expected exception class.
+     * @param msg Expected message.
+     */
+    @SuppressWarnings("ThrowableResultOfMethodCallIgnored")
+    protected void assertThrowsAnyCause(final String sql, Class<? extends Throwable> cls, String msg) {
+        assertThrowsAnyCause(grid(nodeIndex()), sql, cls, msg);
+    }
+
+    /**
      * Execute SQL command and return resulting dataset.
      * @param sql Statement.
      * @return result.
@@ -383,7 +695,7 @@ public abstract class H2DynamicColumnsAbstractBasicSelfTest extends DynamicColum
         private String name;
 
         /** City state. */
-        @QuerySqlField
+        @QuerySqlField(name = "state_name")
         private String state;
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/a3946873/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
index 1474954..2f6977a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/database/IgnitePersistentStoreSchemaLoadTest.java
@@ -285,8 +285,12 @@ public class IgnitePersistentStoreSchemaLoadTest extends GridCommonAbstractTest
                 .getAll();
 
         node.context().query().querySqlFieldsNoCache(
-            new SqlFieldsQuery("alter table \"Person\" add column \"age\" int").setSchema(schema), false)
-                .getAll();
+            new SqlFieldsQuery("alter table \"Person\" add column (\"age\" int, \"city\" char)")
+            .setSchema(schema), false).getAll();
+
+        node.context().query().querySqlFieldsNoCache(
+            new SqlFieldsQuery("alter table \"Person\" drop column \"city\"").setSchema(schema), false)
+            .getAll();
     }
 
     /**


[11/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedBlockOLSMultipleLinearRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedBlockOLSMultipleLinearRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedBlockOLSMultipleLinearRegressionTest.java
deleted file mode 100644
index 8c9d429..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedBlockOLSMultipleLinearRegressionTest.java
+++ /dev/null
@@ -1,901 +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;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.NullArgumentException;
-import org.apache.ignite.ml.math.exceptions.SingularMatrixException;
-import org.apache.ignite.ml.math.impls.matrix.SparseBlockDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
-import org.apache.ignite.ml.math.util.MatrixUtil;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-import org.junit.Assert;
-
-/**
- * Tests for {@link OLSMultipleLinearRegression}.
- */
-@GridCommonTest(group = "Distributed Models")
-public class DistributedBlockOLSMultipleLinearRegressionTest extends GridCommonAbstractTest {
-    /** */
-    private double[] y;
-
-    /** */
-    private double[][] x;
-
-    /** */
-    private AbstractMultipleLinearRegression regression;
-
-    /** Number of nodes in grid */
-    private static final int NODE_COUNT = 3;
-
-    /** */
-    private static final double PRECISION = 1E-12;
-
-    /** Grid instance. */
-    private Ignite ignite;
-
-    /** */
-    public DistributedBlockOLSMultipleLinearRegressionTest() {
-        super(false);
-    }
-
-    /** {@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();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-
-        ignite.configuration().setPeerClassLoadingEnabled(true);
-
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        y = new double[] {11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        x = new double[6][];
-        x[0] = new double[] {0, 0, 0, 0, 0};
-        x[1] = new double[] {2.0, 0, 0, 0, 0};
-        x[2] = new double[] {0, 3.0, 0, 0, 0};
-        x[3] = new double[] {0, 0, 4.0, 0, 0};
-        x[4] = new double[] {0, 0, 0, 5.0, 0};
-        x[5] = new double[] {0, 0, 0, 0, 6.0};
-
-        regression = createRegression();
-    }
-
-    /** */
-    private OLSMultipleLinearRegression createRegression() {
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new SparseBlockDistributedVector(y), new SparseBlockDistributedMatrix(x));
-        return regression;
-    }
-
-    /** */
-    public void testPerfectFit() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        double[] betaHat = regression.estimateRegressionParameters();
-        TestUtils.assertEquals(new double[] {11.0, 1.0 / 2.0, 2.0 / 3.0, 3.0 / 4.0, 4.0 / 5.0, 5.0 / 6.0},
-            betaHat,
-            1e-13);
-        double[] residuals = regression.estimateResiduals();
-        TestUtils.assertEquals(new double[] {0d, 0d, 0d, 0d, 0d, 0d}, residuals,
-            1e-13);
-        Matrix errors = regression.estimateRegressionParametersVariance();
-        final double[] s = {1.0, -1.0 / 2.0, -1.0 / 3.0, -1.0 / 4.0, -1.0 / 5.0, -1.0 / 6.0};
-        Matrix refVar = new SparseBlockDistributedMatrix(s.length, s.length);
-        for (int i = 0; i < refVar.rowSize(); i++)
-            for (int j = 0; j < refVar.columnSize(); j++) {
-                if (i == 0) {
-                    refVar.setX(i, j, s[j]);
-                    continue;
-                }
-                double x = s[i] * s[j];
-                refVar.setX(i, j, (i == j) ? 2 * x : x);
-            }
-        Assert.assertEquals(0.0,
-            TestUtils.maximumAbsoluteRowSum(errors.minus(refVar)),
-            5.0e-16 * TestUtils.maximumAbsoluteRowSum(refVar));
-        Assert.assertEquals(1, ((OLSMultipleLinearRegression)regression).calculateRSquared(), 1E-12);
-    }
-
-    /**
-     * Test Longley dataset against certified values provided by NIST. Data Source: J. Longley (1967) "An Appraisal of
-     * Least Squares Programs for the Electronic Computer from the Point of View of the User" Journal of the American
-     * Statistical Association, vol. 62. September, pp. 819-841.
-     *
-     * Certified values (and data) are from NIST: http://www.itl.nist.gov/div898/strd/lls/data/LINKS/DATA/Longley.dat
-     */
-    public void testLongly() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        // Y values are first, then independent vars
-        // Each row is one observation
-        double[] design = 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;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseBlockDistributedMatrix());
-
-        // Check expected beta values from NIST
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -3482258.63459582, 15.0618722713733,
-                -0.358191792925910E-01, -2.02022980381683,
-                -1.03322686717359, -0.511041056535807E-01,
-                1829.15146461355}, 2E-6); //
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                267.340029759711, -94.0139423988359, 46.28716775752924,
-                -410.114621930906, 309.7145907602313, -249.3112153297231,
-                -164.0489563956039, -13.18035686637081, 14.30477260005235,
-                455.394094551857, -17.26892711483297, -39.0550425226967,
-                -155.5499735953195, -85.6713080421283, 341.9315139607727,
-                -206.7578251937366},
-            1E-7);
-
-        // Check standard errors from NIST
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            890420.383607373,
-            84.9149257747669,
-            0.334910077722432E-01,
-            0.488399681651699,
-            0.214274163161675,
-            0.226073200069370,
-            455.478499142212}, errors, 1E-6);
-
-        // Check regression standard error against R
-        Assert.assertEquals(304.8540735619638, mdl.estimateRegressionStandardError(), 1E-8);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.995479004577296, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.992465007628826, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate model without intercept
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new SparseBlockDistributedMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -52.99357013868291, 0.07107319907358,
-                -0.42346585566399, -0.57256866841929,
-                -0.41420358884978, 48.41786562001326}, 1E-8);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            129.54486693117232, 0.03016640003786,
-            0.41773654056612, 0.27899087467676, 0.32128496193363,
-            17.68948737819961}, errors, 1E-11);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                279.90274927293092, -130.32465380836874, 90.73228661967445, -401.31252201634948,
-                -440.46768772620027, -543.54512853774793, 201.32111639536299, 215.90889365977932,
-                73.09368242049943, 913.21694494481869, 424.82484953610174, -8.56475876776709,
-                -361.32974610842876, 27.34560497213464, 151.28955976355002, -492.49937355336846},
-            1E-8);
-
-        // Check regression standard error against R
-        Assert.assertEquals(475.1655079819517, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.9999670130706, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.999947220913, mdl.calculateAdjustedRSquared(), 1E-12);
-    }
-
-    /**
-     * Test R Swiss fertility dataset against R. Data Source: R datasets package
-     */
-    public void testSwissFertility() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        double[] design = new double[] {
-            80.2, 17.0, 15, 12, 9.96,
-            83.1, 45.1, 6, 9, 84.84,
-            92.5, 39.7, 5, 5, 93.40,
-            85.8, 36.5, 12, 7, 33.77,
-            76.9, 43.5, 17, 15, 5.16,
-            76.1, 35.3, 9, 7, 90.57,
-            83.8, 70.2, 16, 7, 92.85,
-            92.4, 67.8, 14, 8, 97.16,
-            82.4, 53.3, 12, 7, 97.67,
-            82.9, 45.2, 16, 13, 91.38,
-            87.1, 64.5, 14, 6, 98.61,
-            64.1, 62.0, 21, 12, 8.52,
-            66.9, 67.5, 14, 7, 2.27,
-            68.9, 60.7, 19, 12, 4.43,
-            61.7, 69.3, 22, 5, 2.82,
-            68.3, 72.6, 18, 2, 24.20,
-            71.7, 34.0, 17, 8, 3.30,
-            55.7, 19.4, 26, 28, 12.11,
-            54.3, 15.2, 31, 20, 2.15,
-            65.1, 73.0, 19, 9, 2.84,
-            65.5, 59.8, 22, 10, 5.23,
-            65.0, 55.1, 14, 3, 4.52,
-            56.6, 50.9, 22, 12, 15.14,
-            57.4, 54.1, 20, 6, 4.20,
-            72.5, 71.2, 12, 1, 2.40,
-            74.2, 58.1, 14, 8, 5.23,
-            72.0, 63.5, 6, 3, 2.56,
-            60.5, 60.8, 16, 10, 7.72,
-            58.3, 26.8, 25, 19, 18.46,
-            65.4, 49.5, 15, 8, 6.10,
-            75.5, 85.9, 3, 2, 99.71,
-            69.3, 84.9, 7, 6, 99.68,
-            77.3, 89.7, 5, 2, 100.00,
-            70.5, 78.2, 12, 6, 98.96,
-            79.4, 64.9, 7, 3, 98.22,
-            65.0, 75.9, 9, 9, 99.06,
-            92.2, 84.6, 3, 3, 99.46,
-            79.3, 63.1, 13, 13, 96.83,
-            70.4, 38.4, 26, 12, 5.62,
-            65.7, 7.7, 29, 11, 13.79,
-            72.7, 16.7, 22, 13, 11.22,
-            64.4, 17.6, 35, 32, 16.92,
-            77.6, 37.6, 15, 7, 4.97,
-            67.6, 18.7, 25, 7, 8.65,
-            35.0, 1.2, 37, 53, 42.34,
-            44.7, 46.6, 16, 29, 50.43,
-            42.8, 27.7, 22, 29, 58.33
-        };
-
-        final int nobs = 47;
-        final int nvars = 4;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseBlockDistributedMatrix());
-
-        // Check expected beta values from R
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                91.05542390271397,
-                -0.22064551045715,
-                -0.26058239824328,
-                -0.96161238456030,
-                0.12441843147162}, 1E-12);
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                7.1044267859730512, 1.6580347433531366,
-                4.6944952770029644, 8.4548022690166160, 13.6547432343186212,
-                -9.3586864458500774, 7.5822446330520386, 15.5568995563859289,
-                0.8113090736598980, 7.1186762732484308, 7.4251378771228724,
-                2.6761316873234109, 0.8351584810309354, 7.1769991119615177,
-                -3.8746753206299553, -3.1337779476387251, -0.1412575244091504,
-                1.1186809170469780, -6.3588097346816594, 3.4039270429434074,
-                2.3374058329820175, -7.9272368576900503, -7.8361010968497959,
-                -11.2597369269357070, 0.9445333697827101, 6.6544245101380328,
-                -0.9146136301118665, -4.3152449403848570, -4.3536932047009183,
-                -3.8907885169304661, -6.3027643926302188, -7.8308982189289091,
-                -3.1792280015332750, -6.7167298771158226, -4.8469946718041754,
-                -10.6335664353633685, 11.1031134362036958, 6.0084032641811733,
-                5.4326230830188482, -7.2375578629692230, 2.1671550814448222,
-                15.0147574652763112, 4.8625103516321015, -7.1597256413907706,
-                -0.4515205619767598, -10.2916870903837587, -15.7812984571900063},
-            1E-12);
-
-        // Check standard errors from R
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            6.94881329475087,
-            0.07360008972340,
-            0.27410957467466,
-            0.19454551679325,
-            0.03726654773803}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(7.73642194433223, mdl.estimateRegressionStandardError(), 1E-12);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.649789742860228, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.6164363850373927, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate the model with no intercept
-        mdl = new OLSMultipleLinearRegression();
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new SparseBlockDistributedMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                0.52191832900513,
-                2.36588087917963,
-                -0.94770353802795,
-                0.30851985863609}, 1E-12);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                44.138759883538249, 27.720705122356215, 35.873200836126799,
-                34.574619581211977, 26.600168342080213, 15.074636243026923, -12.704904871199814,
-                1.497443824078134, 2.691972687079431, 5.582798774291231, -4.422986561283165,
-                -9.198581600334345, 4.481765170730647, 2.273520207553216, -22.649827853221336,
-                -17.747900013943308, 20.298314638496436, 6.861405135329779, -8.684712790954924,
-                -10.298639278062371, -9.896618896845819, 4.568568616351242, -15.313570491727944,
-                -13.762961360873966, 7.156100301980509, 16.722282219843990, 26.716200609071898,
-                -1.991466398777079, -2.523342564719335, 9.776486693095093, -5.297535127628603,
-                -16.639070567471094, -10.302057295211819, -23.549487860816846, 1.506624392156384,
-                -17.939174438345930, 13.105792202765040, -1.943329906928462, -1.516005841666695,
-                -0.759066561832886, 20.793137744128977, -2.485236153005426, 27.588238710486976,
-                2.658333257106881, -15.998337823623046, -5.550742066720694, -14.219077806826615},
-            1E-12);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            0.10470063765677, 0.41684100584290,
-            0.43370143099691, 0.07694953606522}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(17.24710630547, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.946350722085, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.9413600915813, mdl.calculateAdjustedRSquared(), 1E-12);
-    }
-
-    /**
-     * Test hat matrix computation
-     */
-    public void testHat() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        /*
-         * This example is from "The Hat Matrix in Regression and ANOVA",
-         * David C. Hoaglin and Roy E. Welsch,
-         * The American Statistician, Vol. 32, No. 1 (Feb., 1978), pp. 17-22.
-         *
-         */
-        double[] design = new double[] {
-            11.14, .499, 11.1,
-            12.74, .558, 8.9,
-            13.13, .604, 8.8,
-            11.51, .441, 8.9,
-            12.38, .550, 8.8,
-            12.60, .528, 9.9,
-            11.13, .418, 10.7,
-            11.7, .480, 10.5,
-            11.02, .406, 10.5,
-            11.41, .467, 10.7
-        };
-
-        int nobs = 10;
-        int nvars = 2;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseBlockDistributedMatrix());
-
-        Matrix hat = mdl.calculateHat();
-
-        // Reference data is upper half of symmetric hat matrix
-        double[] refData = new double[] {
-            .418, -.002, .079, -.274, -.046, .181, .128, .222, .050, .242,
-            .242, .292, .136, .243, .128, -.041, .033, -.035, .004,
-            .417, -.019, .273, .187, -.126, .044, -.153, .004,
-            .604, .197, -.038, .168, -.022, .275, -.028,
-            .252, .111, -.030, .019, -.010, -.010,
-            .148, .042, .117, .012, .111,
-            .262, .145, .277, .174,
-            .154, .120, .168,
-            .315, .148,
-            .187
-        };
-
-        // Check against reference data and verify symmetry
-        int k = 0;
-        for (int i = 0; i < 10; i++) {
-            for (int j = i; j < 10; j++) {
-                Assert.assertEquals(refData[k], hat.getX(i, j), 10e-3);
-                Assert.assertEquals(hat.getX(i, j), hat.getX(j, i), 10e-12);
-                k++;
-            }
-        }
-
-        /*
-         * Verify that residuals computed using the hat matrix are close to
-         * what we get from direct computation, i.e. r = (I - H) y
-         */
-        double[] residuals = mdl.estimateResiduals();
-        Matrix id = MatrixUtil.identityLike(hat, 10);
-        double[] hatResiduals = id.minus(hat).times(mdl.getY()).getStorage().data();
-        TestUtils.assertEquals(residuals, hatResiduals, 10e-12);
-    }
-
-    /**
-     * test calculateYVariance
-     */
-    public void testYVariance() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        // assumes: y = new double[]{11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(new SparseBlockDistributedVector(y), new SparseBlockDistributedMatrix(x));
-        TestUtils.assertEquals(mdl.calculateYVariance(), 3.5, 0);
-    }
-
-    /**
-     * Verifies that setting X and Y separately has the same effect as newSample(X,Y).
-     */
-    public void testNewSample2() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] y = new double[] {1, 2, 3, 4};
-        double[][] x = new double[][] {
-            {19, 22, 33},
-            {20, 30, 40},
-            {25, 35, 45},
-            {27, 37, 47}
-        };
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new SparseBlockDistributedVector(y), new SparseBlockDistributedMatrix(x));
-        Matrix combinedX = regression.getX().copy();
-        Vector combinedY = regression.getY().copy();
-        regression.newXSampleData(new SparseBlockDistributedMatrix(x));
-        regression.newYSampleData(new SparseBlockDistributedVector(y));
-        for (int i = 0; i < combinedX.rowSize(); i++) {
-            for (int j = 0; j < combinedX.columnSize(); j++)
-                Assert.assertEquals(combinedX.get(i, j), regression.getX().get(i, j), PRECISION);
-
-        }
-        for (int i = 0; i < combinedY.size(); i++)
-            Assert.assertEquals(combinedY.get(i), regression.getY().get(i), PRECISION);
-
-        // No intercept
-        regression.setNoIntercept(true);
-        regression.newSampleData(new SparseBlockDistributedVector(y), new SparseBlockDistributedMatrix(x));
-        combinedX = regression.getX().copy();
-        combinedY = regression.getY().copy();
-        regression.newXSampleData(new SparseBlockDistributedMatrix(x));
-        regression.newYSampleData(new SparseBlockDistributedVector(y));
-
-        for (int i = 0; i < combinedX.rowSize(); i++) {
-            for (int j = 0; j < combinedX.columnSize(); j++)
-                Assert.assertEquals(combinedX.get(i, j), regression.getX().get(i, j), PRECISION);
-
-        }
-        for (int i = 0; i < combinedY.size(); i++)
-            Assert.assertEquals(combinedY.get(i), regression.getY().get(i), PRECISION);
-
-    }
-
-    /** */
-    public void testNewSampleDataYNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        try {
-            createRegression().newSampleData(null, new SparseBlockDistributedMatrix(new double[][] {{1}}));
-            fail("Expected NullArgumentException was not caught.");
-        }
-        catch (NullArgumentException e) {
-            return;
-        }
-        fail("Expected NullArgumentException was not caught.");
-    }
-
-    /** */
-    public void testNewSampleDataXNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        try {
-            createRegression().newSampleData(new SparseBlockDistributedVector(new double[] {1}), null);
-            fail("Expected NullArgumentException was not caught.");
-        }
-        catch (NullArgumentException e) {
-            return;
-        }
-        fail("Expected NullArgumentException was not caught.");
-    }
-
-    /**
-     * This is a test based on the Wampler1 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler1.shtml
-     */
-    public void testWampler1() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            1, 0,
-            6, 1,
-            63, 2,
-            364, 3,
-            1365, 4,
-            3906, 5,
-            9331, 6,
-            19608, 7,
-            37449, 8,
-            66430, 9,
-            111111, 10,
-            177156, 11,
-            271453, 12,
-            402234, 13,
-            579195, 14,
-            813616, 15,
-            1118481, 16,
-            1508598, 17,
-            2000719, 18,
-            2613660, 19,
-            3368421, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseBlockDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0, 1.0,
-                1.0, 1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler2 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler2.shtml
-     */
-    public void testWampler2() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            1.00000, 0,
-            1.11111, 1,
-            1.24992, 2,
-            1.42753, 3,
-            1.65984, 4,
-            1.96875, 5,
-            2.38336, 6,
-            2.94117, 7,
-            3.68928, 8,
-            4.68559, 9,
-            6.00000, 10,
-            7.71561, 11,
-            9.92992, 12,
-            12.75603, 13,
-            16.32384, 14,
-            20.78125, 15,
-            26.29536, 16,
-            33.05367, 17,
-            41.26528, 18,
-            51.16209, 19,
-            63.00000, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseBlockDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0e-1,
-                1.0e-2,
-                1.0e-3, 1.0e-4,
-                1.0e-5}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler3 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler3.shtml
-     */
-    public void testWampler3() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            760, 0,
-            -2042, 1,
-            2111, 2,
-            -1684, 3,
-            3888, 4,
-            1858, 5,
-            11379, 6,
-            17560, 7,
-            39287, 8,
-            64382, 9,
-            113159, 10,
-            175108, 11,
-            273291, 12,
-            400186, 13,
-            581243, 14,
-            811568, 15,
-            1121004, 16,
-            1506550, 17,
-            2002767, 18,
-            2611612, 19,
-            3369180, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseBlockDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                2152.32624678170,
-                2363.55173469681, 779.343524331583,
-                101.475507550350, 5.64566512170752,
-                0.112324854679312}, 1E-8); //
-
-        TestUtils.assertEquals(.999995559025820, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(5570284.53333333, mdl.estimateErrorVariance(), 1.0e-6);
-        TestUtils.assertEquals(83554268.0000000, mdl.calculateResidualSumOfSquares(), 1.0e-5);
-    }
-
-    /**
-     * This is a test based on the Wampler4 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler4.shtml
-     */
-    public void testWampler4() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            75901, 0,
-            -204794, 1,
-            204863, 2,
-            -204436, 3,
-            253665, 4,
-            -200894, 5,
-            214131, 6,
-            -185192, 7,
-            221249, 8,
-            -138370, 9,
-            315911, 10,
-            -27644, 11,
-            455253, 12,
-            197434, 13,
-            783995, 14,
-            608816, 15,
-            1370781, 16,
-            1303798, 17,
-            2205519, 18,
-            2408860, 19,
-            3444321, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseBlockDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-6);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                215232.624678170,
-                236355.173469681, 77934.3524331583,
-                10147.5507550350, 564.566512170752,
-                11.2324854679312}, 1E-8);
-
-        TestUtils.assertEquals(.957478440825662, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(55702845333.3333, mdl.estimateErrorVariance(), 1.0e-4);
-        TestUtils.assertEquals(835542680000.000, mdl.calculateResidualSumOfSquares(), 1.0e-3);
-    }
-
-    /**
-     * Anything requiring beta calculation should advertise SME.
-     */
-    public void testSingularCalculateBeta() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression(1e-15);
-
-        try {
-            mdl.newSampleData(new double[] {1, 2, 3, 1, 2, 3, 1, 2, 3}, 3, 2, new SparseBlockDistributedMatrix());
-            mdl.calculateBeta();
-            fail("Expected SingularMatrixException was not caught.");
-        }
-        catch (SingularMatrixException e) {
-            return;
-        }
-        fail("Expected SingularMatrixException was not caught.");
-    }
-
-    /** */
-    public void testNoDataNPECalculateBeta() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateBeta();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-    }
-
-    /** */
-    public void testNoDataNPECalculateHat() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateHat();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-    }
-
-    /** */
-    public void testNoDataNPESSTO() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateTotalSumOfSquares();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-    }
-
-    /** */
-    public void testMathIllegalArgumentException() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.validateSampleData(new SparseBlockDistributedMatrix(1, 2), new SparseBlockDistributedVector(1));
-            fail("Expected MathIllegalArgumentException was not caught.");
-        }
-        catch (MathIllegalArgumentException e) {
-            return;
-        }
-        fail("Expected MathIllegalArgumentException was not caught.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedOLSMultipleLinearRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedOLSMultipleLinearRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedOLSMultipleLinearRegressionTest.java
deleted file mode 100644
index f720406..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/DistributedOLSMultipleLinearRegressionTest.java
+++ /dev/null
@@ -1,903 +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;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.util.IgniteUtils;
-import org.apache.ignite.ml.TestUtils;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.NullArgumentException;
-import org.apache.ignite.ml.math.exceptions.SingularMatrixException;
-import org.apache.ignite.ml.math.impls.matrix.SparseDistributedMatrix;
-import org.apache.ignite.ml.math.impls.vector.SparseDistributedVector;
-import org.apache.ignite.ml.math.util.MatrixUtil;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-import org.junit.Assert;
-
-/**
- * Tests for {@link OLSMultipleLinearRegression}.
- */
-@GridCommonTest(group = "Distributed Models")
-public class DistributedOLSMultipleLinearRegressionTest extends GridCommonAbstractTest {
-    /** */
-    private double[] y;
-
-    /** */
-    private double[][] x;
-
-    /** */
-    private AbstractMultipleLinearRegression regression;
-
-    /** Number of nodes in grid */
-    private static final int NODE_COUNT = 3;
-
-    /** */
-    private static final double PRECISION = 1E-12;
-
-    /** Grid instance. */
-    private Ignite ignite;
-
-    /** */
-    public DistributedOLSMultipleLinearRegressionTest() {
-        super(false);
-    }
-
-    /** {@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();
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override protected void beforeTest() throws Exception {
-        ignite = grid(NODE_COUNT);
-
-        ignite.configuration().setPeerClassLoadingEnabled(true);
-
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        y = new double[] {11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        x = new double[6][];
-        x[0] = new double[] {0, 0, 0, 0, 0};
-        x[1] = new double[] {2.0, 0, 0, 0, 0};
-        x[2] = new double[] {0, 3.0, 0, 0, 0};
-        x[3] = new double[] {0, 0, 4.0, 0, 0};
-        x[4] = new double[] {0, 0, 0, 5.0, 0};
-        x[5] = new double[] {0, 0, 0, 0, 6.0};
-
-        regression = createRegression();
-    }
-
-    /** */
-    private OLSMultipleLinearRegression createRegression() {
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new SparseDistributedVector(y), new SparseDistributedMatrix(x));
-        return regression;
-    }
-
-    /** */
-    public void testPerfectFit() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        double[] betaHat = regression.estimateRegressionParameters();
-        TestUtils.assertEquals(new double[] {11.0, 1.0 / 2.0, 2.0 / 3.0, 3.0 / 4.0, 4.0 / 5.0, 5.0 / 6.0},
-            betaHat,
-            1e-13);
-        double[] residuals = regression.estimateResiduals();
-        TestUtils.assertEquals(new double[] {0d, 0d, 0d, 0d, 0d, 0d}, residuals,
-            1e-13);
-        Matrix errors = regression.estimateRegressionParametersVariance();
-        final double[] s = {1.0, -1.0 / 2.0, -1.0 / 3.0, -1.0 / 4.0, -1.0 / 5.0, -1.0 / 6.0};
-        Matrix refVar = new SparseDistributedMatrix(s.length, s.length);
-        for (int i = 0; i < refVar.rowSize(); i++)
-            for (int j = 0; j < refVar.columnSize(); j++) {
-                if (i == 0) {
-                    refVar.setX(i, j, s[j]);
-                    continue;
-                }
-                double x = s[i] * s[j];
-                refVar.setX(i, j, (i == j) ? 2 * x : x);
-            }
-        Assert.assertEquals(0.0,
-            TestUtils.maximumAbsoluteRowSum(errors.minus(refVar)),
-            5.0e-16 * TestUtils.maximumAbsoluteRowSum(refVar));
-        Assert.assertEquals(1, ((OLSMultipleLinearRegression)regression).calculateRSquared(), 1E-12);
-    }
-
-    /**
-     * Test Longley dataset against certified values provided by NIST. Data Source: J. Longley (1967) "An Appraisal of
-     * Least Squares Programs for the Electronic Computer from the Point of View of the User" Journal of the American
-     * Statistical Association, vol. 62. September, pp. 819-841.
-     *
-     * Certified values (and data) are from NIST: http://www.itl.nist.gov/div898/strd/lls/data/LINKS/DATA/Longley.dat
-     */
-    public void testLongly() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        // Y values are first, then independent vars
-        // Each row is one observation
-        double[] design = 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;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseDistributedMatrix());
-
-        // Check expected beta values from NIST
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -3482258.63459582, 15.0618722713733,
-                -0.358191792925910E-01, -2.02022980381683,
-                -1.03322686717359, -0.511041056535807E-01,
-                1829.15146461355}, 2E-6); //
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                267.340029759711, -94.0139423988359, 46.28716775752924,
-                -410.114621930906, 309.7145907602313, -249.3112153297231,
-                -164.0489563956039, -13.18035686637081, 14.30477260005235,
-                455.394094551857, -17.26892711483297, -39.0550425226967,
-                -155.5499735953195, -85.6713080421283, 341.9315139607727,
-                -206.7578251937366},
-            1E-7);
-
-        // Check standard errors from NIST
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            890420.383607373,
-            84.9149257747669,
-            0.334910077722432E-01,
-            0.488399681651699,
-            0.214274163161675,
-            0.226073200069370,
-            455.478499142212}, errors, 1E-6);
-
-        // Check regression standard error against R
-        Assert.assertEquals(304.8540735619638, mdl.estimateRegressionStandardError(), 1E-8);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.995479004577296, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.992465007628826, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate model without intercept
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new SparseDistributedMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                -52.99357013868291, 0.07107319907358,
-                -0.42346585566399, -0.57256866841929,
-                -0.41420358884978, 48.41786562001326}, 1E-8);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            129.54486693117232, 0.03016640003786,
-            0.41773654056612, 0.27899087467676, 0.32128496193363,
-            17.68948737819961}, errors, 1E-11);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                279.90274927293092, -130.32465380836874, 90.73228661967445, -401.31252201634948,
-                -440.46768772620027, -543.54512853774793, 201.32111639536299, 215.90889365977932,
-                73.09368242049943, 913.21694494481869, 424.82484953610174, -8.56475876776709,
-                -361.32974610842876, 27.34560497213464, 151.28955976355002, -492.49937355336846},
-            1E-8);
-
-        // Check regression standard error against R
-        Assert.assertEquals(475.1655079819517, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.9999670130706, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.999947220913, mdl.calculateAdjustedRSquared(), 1E-12);
-    }
-
-    /**
-     * Test R Swiss fertility dataset against R. Data Source: R datasets package
-     */
-    public void testSwissFertility() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        double[] design = new double[] {
-            80.2, 17.0, 15, 12, 9.96,
-            83.1, 45.1, 6, 9, 84.84,
-            92.5, 39.7, 5, 5, 93.40,
-            85.8, 36.5, 12, 7, 33.77,
-            76.9, 43.5, 17, 15, 5.16,
-            76.1, 35.3, 9, 7, 90.57,
-            83.8, 70.2, 16, 7, 92.85,
-            92.4, 67.8, 14, 8, 97.16,
-            82.4, 53.3, 12, 7, 97.67,
-            82.9, 45.2, 16, 13, 91.38,
-            87.1, 64.5, 14, 6, 98.61,
-            64.1, 62.0, 21, 12, 8.52,
-            66.9, 67.5, 14, 7, 2.27,
-            68.9, 60.7, 19, 12, 4.43,
-            61.7, 69.3, 22, 5, 2.82,
-            68.3, 72.6, 18, 2, 24.20,
-            71.7, 34.0, 17, 8, 3.30,
-            55.7, 19.4, 26, 28, 12.11,
-            54.3, 15.2, 31, 20, 2.15,
-            65.1, 73.0, 19, 9, 2.84,
-            65.5, 59.8, 22, 10, 5.23,
-            65.0, 55.1, 14, 3, 4.52,
-            56.6, 50.9, 22, 12, 15.14,
-            57.4, 54.1, 20, 6, 4.20,
-            72.5, 71.2, 12, 1, 2.40,
-            74.2, 58.1, 14, 8, 5.23,
-            72.0, 63.5, 6, 3, 2.56,
-            60.5, 60.8, 16, 10, 7.72,
-            58.3, 26.8, 25, 19, 18.46,
-            65.4, 49.5, 15, 8, 6.10,
-            75.5, 85.9, 3, 2, 99.71,
-            69.3, 84.9, 7, 6, 99.68,
-            77.3, 89.7, 5, 2, 100.00,
-            70.5, 78.2, 12, 6, 98.96,
-            79.4, 64.9, 7, 3, 98.22,
-            65.0, 75.9, 9, 9, 99.06,
-            92.2, 84.6, 3, 3, 99.46,
-            79.3, 63.1, 13, 13, 96.83,
-            70.4, 38.4, 26, 12, 5.62,
-            65.7, 7.7, 29, 11, 13.79,
-            72.7, 16.7, 22, 13, 11.22,
-            64.4, 17.6, 35, 32, 16.92,
-            77.6, 37.6, 15, 7, 4.97,
-            67.6, 18.7, 25, 7, 8.65,
-            35.0, 1.2, 37, 53, 42.34,
-            44.7, 46.6, 16, 29, 50.43,
-            42.8, 27.7, 22, 29, 58.33
-        };
-
-        final int nobs = 47;
-        final int nvars = 4;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseDistributedMatrix());
-
-        // Check expected beta values from R
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                91.05542390271397,
-                -0.22064551045715,
-                -0.26058239824328,
-                -0.96161238456030,
-                0.12441843147162}, 1E-12);
-
-        // Check expected residuals from R
-        double[] residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                7.1044267859730512, 1.6580347433531366,
-                4.6944952770029644, 8.4548022690166160, 13.6547432343186212,
-                -9.3586864458500774, 7.5822446330520386, 15.5568995563859289,
-                0.8113090736598980, 7.1186762732484308, 7.4251378771228724,
-                2.6761316873234109, 0.8351584810309354, 7.1769991119615177,
-                -3.8746753206299553, -3.1337779476387251, -0.1412575244091504,
-                1.1186809170469780, -6.3588097346816594, 3.4039270429434074,
-                2.3374058329820175, -7.9272368576900503, -7.8361010968497959,
-                -11.2597369269357070, 0.9445333697827101, 6.6544245101380328,
-                -0.9146136301118665, -4.3152449403848570, -4.3536932047009183,
-                -3.8907885169304661, -6.3027643926302188, -7.8308982189289091,
-                -3.1792280015332750, -6.7167298771158226, -4.8469946718041754,
-                -10.6335664353633685, 11.1031134362036958, 6.0084032641811733,
-                5.4326230830188482, -7.2375578629692230, 2.1671550814448222,
-                15.0147574652763112, 4.8625103516321015, -7.1597256413907706,
-                -0.4515205619767598, -10.2916870903837587, -15.7812984571900063},
-            1E-12);
-
-        // Check standard errors from R
-        double[] errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            6.94881329475087,
-            0.07360008972340,
-            0.27410957467466,
-            0.19454551679325,
-            0.03726654773803}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(7.73642194433223, mdl.estimateRegressionStandardError(), 1E-12);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.649789742860228, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.6164363850373927, mdl.calculateAdjustedRSquared(), 1E-12);
-
-        // TODO: IGNITE-5826, uncomment.
-        // checkVarianceConsistency(model);
-
-        // Estimate the model with no intercept
-        mdl = new OLSMultipleLinearRegression();
-        mdl.setNoIntercept(true);
-        mdl.newSampleData(design, nobs, nvars, new SparseDistributedMatrix());
-
-        // Check expected beta values from R
-        betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                0.52191832900513,
-                2.36588087917963,
-                -0.94770353802795,
-                0.30851985863609}, 1E-12);
-
-        // Check expected residuals from R
-        residuals = mdl.estimateResiduals();
-        TestUtils.assertEquals(residuals, new double[] {
-                44.138759883538249, 27.720705122356215, 35.873200836126799,
-                34.574619581211977, 26.600168342080213, 15.074636243026923, -12.704904871199814,
-                1.497443824078134, 2.691972687079431, 5.582798774291231, -4.422986561283165,
-                -9.198581600334345, 4.481765170730647, 2.273520207553216, -22.649827853221336,
-                -17.747900013943308, 20.298314638496436, 6.861405135329779, -8.684712790954924,
-                -10.298639278062371, -9.896618896845819, 4.568568616351242, -15.313570491727944,
-                -13.762961360873966, 7.156100301980509, 16.722282219843990, 26.716200609071898,
-                -1.991466398777079, -2.523342564719335, 9.776486693095093, -5.297535127628603,
-                -16.639070567471094, -10.302057295211819, -23.549487860816846, 1.506624392156384,
-                -17.939174438345930, 13.105792202765040, -1.943329906928462, -1.516005841666695,
-                -0.759066561832886, 20.793137744128977, -2.485236153005426, 27.588238710486976,
-                2.658333257106881, -15.998337823623046, -5.550742066720694, -14.219077806826615},
-            1E-12);
-
-        // Check standard errors from R
-        errors = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(new double[] {
-            0.10470063765677, 0.41684100584290,
-            0.43370143099691, 0.07694953606522}, errors, 1E-10);
-
-        // Check regression standard error against R
-        Assert.assertEquals(17.24710630547, mdl.estimateRegressionStandardError(), 1E-10);
-
-        // Check R-Square statistics against R
-        Assert.assertEquals(0.946350722085, mdl.calculateRSquared(), 1E-12);
-        Assert.assertEquals(0.9413600915813, mdl.calculateAdjustedRSquared(), 1E-12);
-    }
-
-    /**
-     * Test hat matrix computation
-     */
-    public void testHat() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        /*
-         * This example is from "The Hat Matrix in Regression and ANOVA",
-         * David C. Hoaglin and Roy E. Welsch,
-         * The American Statistician, Vol. 32, No. 1 (Feb., 1978), pp. 17-22.
-         *
-         */
-        double[] design = new double[] {
-            11.14, .499, 11.1,
-            12.74, .558, 8.9,
-            13.13, .604, 8.8,
-            11.51, .441, 8.9,
-            12.38, .550, 8.8,
-            12.60, .528, 9.9,
-            11.13, .418, 10.7,
-            11.7, .480, 10.5,
-            11.02, .406, 10.5,
-            11.41, .467, 10.7
-        };
-
-        int nobs = 10;
-        int nvars = 2;
-
-        // Estimate the model
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(design, nobs, nvars, new SparseDistributedMatrix());
-
-        Matrix hat = mdl.calculateHat();
-
-        // Reference data is upper half of symmetric hat matrix
-        double[] refData = new double[] {
-            .418, -.002, .079, -.274, -.046, .181, .128, .222, .050, .242,
-            .242, .292, .136, .243, .128, -.041, .033, -.035, .004,
-            .417, -.019, .273, .187, -.126, .044, -.153, .004,
-            .604, .197, -.038, .168, -.022, .275, -.028,
-            .252, .111, -.030, .019, -.010, -.010,
-            .148, .042, .117, .012, .111,
-            .262, .145, .277, .174,
-            .154, .120, .168,
-            .315, .148,
-            .187
-        };
-
-        // Check against reference data and verify symmetry
-        int k = 0;
-        for (int i = 0; i < 10; i++) {
-            for (int j = i; j < 10; j++) {
-                Assert.assertEquals(refData[k], hat.getX(i, j), 10e-3);
-                Assert.assertEquals(hat.getX(i, j), hat.getX(j, i), 10e-12);
-                k++;
-            }
-        }
-
-        /*
-         * Verify that residuals computed using the hat matrix are close to
-         * what we get from direct computation, i.e. r = (I - H) y
-         */
-        double[] residuals = mdl.estimateResiduals();
-        Matrix id = MatrixUtil.identityLike(hat, 10);
-        double[] hatResiduals = id.minus(hat).times(mdl.getY()).getStorage().data();
-        TestUtils.assertEquals(residuals, hatResiduals, 10e-12);
-    }
-
-    /**
-     * test calculateYVariance
-     */
-    public void testYVariance() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        // assumes: y = new double[]{11.0, 12.0, 13.0, 14.0, 15.0, 16.0};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        mdl.newSampleData(new SparseDistributedVector(y), new SparseDistributedMatrix(x));
-        TestUtils.assertEquals(mdl.calculateYVariance(), 3.5, 0);
-    }
-
-    /**
-     * Verifies that setting X and Y separately has the same effect as newSample(X,Y).
-     */
-    public void testNewSample2() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] y = new double[] {1, 2, 3, 4};
-        double[][] x = new double[][] {
-            {19, 22, 33},
-            {20, 30, 40},
-            {25, 35, 45},
-            {27, 37, 47}
-        };
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression();
-        regression.newSampleData(new SparseDistributedVector(y), new SparseDistributedMatrix(x));
-        Matrix combinedX = regression.getX().copy();
-        Vector combinedY = regression.getY().copy();
-        regression.newXSampleData(new SparseDistributedMatrix(x));
-        regression.newYSampleData(new SparseDistributedVector(y));
-        for (int i = 0; i < combinedX.rowSize(); i++) {
-            for (int j = 0; j < combinedX.columnSize(); j++)
-                Assert.assertEquals(combinedX.get(i, j), regression.getX().get(i, j), PRECISION);
-
-        }
-        for (int i = 0; i < combinedY.size(); i++)
-            Assert.assertEquals(combinedY.get(i), regression.getY().get(i), PRECISION);
-
-        // No intercept
-        regression.setNoIntercept(true);
-        regression.newSampleData(new SparseDistributedVector(y), new SparseDistributedMatrix(x));
-        combinedX = regression.getX().copy();
-        combinedY = regression.getY().copy();
-        regression.newXSampleData(new SparseDistributedMatrix(x));
-        regression.newYSampleData(new SparseDistributedVector(y));
-
-        for (int i = 0; i < combinedX.rowSize(); i++) {
-            for (int j = 0; j < combinedX.columnSize(); j++)
-                Assert.assertEquals(combinedX.get(i, j), regression.getX().get(i, j), PRECISION);
-
-        }
-        for (int i = 0; i < combinedY.size(); i++)
-            Assert.assertEquals(combinedY.get(i), regression.getY().get(i), PRECISION);
-    }
-
-    /** */
-    public void testNewSampleDataYNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        try {
-            createRegression().newSampleData(null, new SparseDistributedMatrix(new double[][] {{1}}));
-            fail("Expected NullArgumentException was not caught.");
-        }
-        catch (NullArgumentException e) {
-            return;
-        }
-        fail("Expected NullArgumentException was not caught.");
-    }
-
-    /** */
-    public void testNewSampleDataXNull() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-
-        try {
-            createRegression().newSampleData(new SparseDistributedVector(new double[] {1}), null);
-            fail("Expected NullArgumentException was not caught.");
-        }
-        catch (NullArgumentException e) {
-            return;
-        }
-        fail("Expected NullArgumentException was not caught.");
-
-    }
-
-    /**
-     * This is a test based on the Wampler1 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler1.shtml
-     */
-    public void testWampler1() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            1, 0,
-            6, 1,
-            63, 2,
-            364, 3,
-            1365, 4,
-            3906, 5,
-            9331, 6,
-            19608, 7,
-            37449, 8,
-            66430, 9,
-            111111, 10,
-            177156, 11,
-            271453, 12,
-            402234, 13,
-            579195, 14,
-            813616, 15,
-            1118481, 16,
-            1508598, 17,
-            2000719, 18,
-            2613660, 19,
-            3368421, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0, 1.0,
-                1.0, 1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler2 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler2.shtml
-     */
-    public void testWampler2() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            1.00000, 0,
-            1.11111, 1,
-            1.24992, 2,
-            1.42753, 3,
-            1.65984, 4,
-            1.96875, 5,
-            2.38336, 6,
-            2.94117, 7,
-            3.68928, 8,
-            4.68559, 9,
-            6.00000, 10,
-            7.71561, 11,
-            9.92992, 12,
-            12.75603, 13,
-            16.32384, 14,
-            20.78125, 15,
-            26.29536, 16,
-            33.05367, 17,
-            41.26528, 18,
-            51.16209, 19,
-            63.00000, 20};
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0e-1,
-                1.0e-2,
-                1.0e-3, 1.0e-4,
-                1.0e-5}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                0.0,
-                0.0, 0.0,
-                0.0, 0.0,
-                0.0}, 1E-8);
-        TestUtils.assertEquals(1.0, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(0, mdl.estimateErrorVariance(), 1.0e-7);
-        TestUtils.assertEquals(0.00, mdl.calculateResidualSumOfSquares(), 1.0e-6);
-    }
-
-    /**
-     * This is a test based on the Wampler3 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler3.shtml
-     */
-    public void testWampler3() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            760, 0,
-            -2042, 1,
-            2111, 2,
-            -1684, 3,
-            3888, 4,
-            1858, 5,
-            11379, 6,
-            17560, 7,
-            39287, 8,
-            64382, 9,
-            113159, 10,
-            175108, 11,
-            273291, 12,
-            400186, 13,
-            581243, 14,
-            811568, 15,
-            1121004, 16,
-            1506550, 17,
-            2002767, 18,
-            2611612, 19,
-            3369180, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-8);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                2152.32624678170,
-                2363.55173469681, 779.343524331583,
-                101.475507550350, 5.64566512170752,
-                0.112324854679312}, 1E-8); //
-
-        TestUtils.assertEquals(.999995559025820, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(5570284.53333333, mdl.estimateErrorVariance(), 1.0e-6);
-        TestUtils.assertEquals(83554268.0000000, mdl.calculateResidualSumOfSquares(), 1.0e-5);
-    }
-
-    /**
-     * This is a test based on the Wampler4 data set http://www.itl.nist.gov/div898/strd/lls/data/Wampler4.shtml
-     */
-    public void testWampler4() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        double[] data = new double[] {
-            75901, 0,
-            -204794, 1,
-            204863, 2,
-            -204436, 3,
-            253665, 4,
-            -200894, 5,
-            214131, 6,
-            -185192, 7,
-            221249, 8,
-            -138370, 9,
-            315911, 10,
-            -27644, 11,
-            455253, 12,
-            197434, 13,
-            783995, 14,
-            608816, 15,
-            1370781, 16,
-            1303798, 17,
-            2205519, 18,
-            2408860, 19,
-            3444321, 20};
-
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-        final int nvars = 5;
-        final int nobs = 21;
-        double[] tmp = new double[(nvars + 1) * nobs];
-        int off = 0;
-        int off2 = 0;
-        for (int i = 0; i < nobs; i++) {
-            tmp[off2] = data[off];
-            tmp[off2 + 1] = data[off + 1];
-            tmp[off2 + 2] = tmp[off2 + 1] * tmp[off2 + 1];
-            tmp[off2 + 3] = tmp[off2 + 1] * tmp[off2 + 2];
-            tmp[off2 + 4] = tmp[off2 + 1] * tmp[off2 + 3];
-            tmp[off2 + 5] = tmp[off2 + 1] * tmp[off2 + 4];
-            off2 += (nvars + 1);
-            off += 2;
-        }
-        mdl.newSampleData(tmp, nobs, nvars, new SparseDistributedMatrix());
-        double[] betaHat = mdl.estimateRegressionParameters();
-        TestUtils.assertEquals(betaHat,
-            new double[] {
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0,
-                1.0}, 1E-6);
-
-        double[] se = mdl.estimateRegressionParametersStandardErrors();
-        TestUtils.assertEquals(se,
-            new double[] {
-                215232.624678170,
-                236355.173469681, 77934.3524331583,
-                10147.5507550350, 564.566512170752,
-                11.2324854679312}, 1E-8);
-
-        TestUtils.assertEquals(.957478440825662, mdl.calculateRSquared(), 1.0e-10);
-        TestUtils.assertEquals(55702845333.3333, mdl.estimateErrorVariance(), 1.0e-4);
-        TestUtils.assertEquals(835542680000.000, mdl.calculateResidualSumOfSquares(), 1.0e-3);
-    }
-
-    /**
-     * Anything requiring beta calculation should advertise SME.
-     */
-    public void testSingularCalculateBeta() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression(1e-15);
-
-        try {
-            mdl.newSampleData(new double[] {1, 2, 3, 1, 2, 3, 1, 2, 3}, 3, 2, new SparseDistributedMatrix());
-            mdl.calculateBeta();
-            fail("Expected SingularMatrixException was not caught.");
-        }
-        catch (SingularMatrixException e) {
-            return;
-        }
-        fail("Expected SingularMatrixException was not caught.");
-
-    }
-
-    /** */
-    public void testNoDataNPECalculateBeta() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateBeta();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-
-    }
-
-    /** */
-    public void testNoDataNPECalculateHat() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateHat();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-    }
-
-    /** */
-    public void testNoDataNPESSTO() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.calculateTotalSumOfSquares();
-            fail("Expected NullPointerException was not caught.");
-        }
-        catch (NullPointerException e) {
-            return;
-        }
-        fail("Expected NullPointerException was not caught.");
-    }
-
-    /** */
-    public void testMathIllegalArgumentException() {
-        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
-        OLSMultipleLinearRegression mdl = new OLSMultipleLinearRegression();
-
-        try {
-            mdl.validateSampleData(new SparseDistributedMatrix(1, 2), new SparseDistributedVector(1));
-            fail("Expected MathIllegalArgumentException was not caught.");
-        }
-        catch (MathIllegalArgumentException e) {
-            return;
-        }
-        fail("Expected MathIllegalArgumentException was not caught.");
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelTest.java
deleted file mode 100644
index 74d5524..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelTest.java
+++ /dev/null
@@ -1,53 +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;
-
-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 OLSMultipleLinearRegressionModel}.
- */
-public class OLSMultipleLinearRegressionModelTest {
-    /** */
-    @Test
-    public void testPerfectFit() {
-        Vector val = new DenseLocalOnHeapVector(new double[] {11.0, 12.0, 13.0, 14.0, 15.0, 16.0});
-
-        double[] data = new double[] {
-            0, 0, 0, 0, 0, 0, // IMPL NOTE values in this row are later replaced (with 1.0)
-            0, 2.0, 0, 0, 0, 0,
-            0, 0, 3.0, 0, 0, 0,
-            0, 0, 0, 4.0, 0, 0,
-            0, 0, 0, 0, 5.0, 0,
-            0, 0, 0, 0, 0, 6.0};
-
-        final int nobs = 6, nvars = 5;
-
-        OLSMultipleLinearRegressionTrainer trainer
-            = new OLSMultipleLinearRegressionTrainer(0, nobs, nvars, new DenseLocalOnHeapMatrix(1, 1));
-
-        OLSMultipleLinearRegressionModel mdl = trainer.train(data);
-
-        TestUtils.assertEquals(new double[] {0d, 0d, 0d, 0d, 0d, 0d},
-            val.minus(mdl.apply(val)).getStorage().data(), 1e-13);
-    }
-}


[06/15] ignite git commit: ignite-7217 Custom thread pools monitoring added

Posted by sb...@apache.org.
ignite-7217 Custom thread pools monitoring added

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/df3c4df3
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/df3c4df3
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/df3c4df3

Branch: refs/heads/ignite-zk
Commit: df3c4df3d895c138434a97dbc5a1a580ead5799d
Parents: c7b4201
Author: Stanislav Lukyanov <st...@gmail.com>
Authored: Thu Dec 28 17:21:47 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Dec 28 17:27:34 2017 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 499 +++++++++----------
 .../apache/ignite/internal/GridMBeansTest.java  |  86 ++++
 .../internal/GridNodeMetricsLogSelfTest.java    |  29 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 4 files changed, 336 insertions(+), 280 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/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 e0c1730..c62d59c 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
@@ -37,10 +37,12 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.Date;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.ListIterator;
 import java.util.Map;
 import java.util.Properties;
+import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ThreadPoolExecutor;
@@ -88,7 +90,6 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.CollectionConfiguration;
-import org.apache.ignite.configuration.ConnectorConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.MemoryConfiguration;
@@ -282,6 +283,10 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     @GridToStringExclude
     private GridKernalContextImpl ctx;
 
+    /** Helper that registers MBeans */
+    @GridToStringExclude
+    private final MBeansManager mBeansMgr = new MBeansManager();
+
     /** Configuration. */
     private IgniteConfiguration cfg;
 
@@ -293,50 +298,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     /** */
     private String igniteInstanceName;
 
-    /** */
-    @GridToStringExclude
-    private ObjectName kernalMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName locNodeMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName allNodesMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName pubExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName sysExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName mgmtExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName p2PExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName restExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName qryExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName schemaExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName stripedExecSvcMBean;
-
     /** Kernal start timestamp. */
     private long startTime = U.currentTimeMillis();
 
@@ -753,7 +714,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         IgniteStripedThreadPoolExecutor callbackExecSvc,
         ExecutorService qryExecSvc,
         ExecutorService schemaExecSvc,
-        Map<String, ? extends ExecutorService> customExecSvcs,
+        @Nullable final Map<String, ? extends ExecutorService> customExecSvcs,
         GridAbsClosure errHnd
     )
         throws IgniteCheckedException
@@ -1083,12 +1044,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 reconnectState.waitFirstReconnect();
 
             // Register MBeans.
-            registerKernalMBean();
-            registerClusterMetricsMBeans();
-            registerExecutorMBeans(execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc, restExecSvc, qryExecSvc,
-                schemaExecSvc);
-
-            registerStripedExecutorMBean(stripedExecSvc);
+            mBeansMgr.registerAllMBeans(utilityCachePool, execSvc, svcExecSvc, sysExecSvc, stripedExecSvc, p2pExecSvc,
+                mgmtExecSvc, igfsExecSvc, dataStreamExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
+                qryExecSvc, schemaExecSvc, customExecSvcs);
 
             // Lifecycle bean notifications.
             notifyLifecycleBeans(AFTER_NODE_START);
@@ -1227,34 +1185,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 // No-op.
                             }
 
-                            int pubPoolActiveThreads = 0;
-                            int pubPoolIdleThreads = 0;
-                            int pubPoolQSize = 0;
-
-                            if (execSvc instanceof ThreadPoolExecutor) {
-                                ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
-
-                                int poolSize = exec.getPoolSize();
-
-                                pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                                pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
-                                pubPoolQSize = exec.getQueue().size();
-                            }
-
-                            int sysPoolActiveThreads = 0;
-                            int sysPoolIdleThreads = 0;
-                            int sysPoolQSize = 0;
-
-                            if (sysExecSvc instanceof ThreadPoolExecutor) {
-                                ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
-
-                                int poolSize = exec.getPoolSize();
-
-                                sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                                sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
-                                sysPoolQSize = exec.getQueue().size();
-                            }
-
                             int loadedPages = 0;
 
                             Collection<DataRegion> policies = ctx.cache().context().database().dataRegions();
@@ -1278,11 +1208,20 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                                 dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
                                 "    ^-- Non heap [used=" + dblFmt.format(nonHeapUsedInMBytes) + "MB, free=" +
                                 dblFmt.format(freeNonHeapPct) + "%, comm=" + dblFmt.format(nonHeapCommInMBytes) + "MB]" + NL +
-                                "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
-                                pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
-                                "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
-                                sysPoolIdleThreads + ", qSize=" + sysPoolQSize + "]" + NL +
-                                "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
+                                "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]" + NL +
+                                "    ^-- " + createExecutorDescription("Public thread pool", execSvc) + NL +
+                                "    ^-- " + createExecutorDescription("System thread pool", sysExecSvc);
+
+                            if (customExecSvcs != null) {
+                                StringBuilder customSvcsMsg = new StringBuilder();
+
+                                for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet()) {
+                                    customSvcsMsg.append(NL).append("    ^-- ")
+                                        .append(createExecutorDescription(entry.getKey(), entry.getValue()));
+                                }
+
+                                msg = msg + customSvcsMsg;
+                            }
 
                             if (log.isInfoEnabled())
                                 log.info(msg);
@@ -1324,6 +1263,30 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /**
+     * Create description of an executor service for logging.
+     *
+     * @param execSvcName name of the service
+     * @param execSvc service to create a description for
+     */
+    private String createExecutorDescription(String execSvcName, ExecutorService execSvc) {
+        int poolActiveThreads = 0;
+        int poolIdleThreads = 0;
+        int poolQSize = 0;
+
+        if (execSvc instanceof ThreadPoolExecutor) {
+            ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+            int poolSize = exec.getPoolSize();
+
+            poolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+            poolIdleThreads = poolSize - poolActiveThreads;
+            poolQSize = exec.getQueue().size();
+        }
+
+        return execSvcName + " [active=" + poolActiveThreads + ", idle=" + poolIdleThreads + ", qSize=" + poolQSize + "]";
+    }
+
+    /**
      * Create Hadoop component.
      *
      * @return Non-null Hadoop component: workable or no-op.
@@ -1677,183 +1640,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerKernalMBean() throws IgniteCheckedException {
-        if(U.IGNITE_MBEANS_DISABLED)
-            return;
-
-        try {
-            kernalMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Kernal",
-                getClass().getSimpleName(),
-                this,
-                IgniteMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered kernal MBean: " + kernalMBean);
-        }
-        catch (JMException e) {
-            kernalMBean = null;
-
-            throw new IgniteCheckedException("Failed to register kernal MBean.", e);
-        }
-    }
-
-    /**
-     * Register instance of ClusterMetricsMBean.
-     *
-     * @param mbean MBean instance to register.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private ObjectName registerClusterMetricsMBean(ClusterMetricsMXBean mbean) throws IgniteCheckedException {
-        if(U.IGNITE_MBEANS_DISABLED)
-            return null;
-
-        ObjectName objName;
-
-        try {
-            objName = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Kernal",
-                mbean.getClass().getSimpleName(),
-                mbean,
-                ClusterMetricsMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered MBean: " + objName);
-
-            return objName;
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register MBean: " + mbean.getClass().getSimpleName(), e);
-        }
-    }
-
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerClusterMetricsMBeans() throws IgniteCheckedException {
-        locNodeMBean = registerClusterMetricsMBean(new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery()));
-        allNodesMBean = registerClusterMetricsMBean(new ClusterMetricsMXBeanImpl(cluster()));
-    }
-
-    /**
-     * @param execSvc Public executor service.
-     * @param sysExecSvc System executor service.
-     * @param p2pExecSvc P2P executor service.
-     * @param mgmtExecSvc Management executor service.
-     * @param restExecSvc Query executor service.
-     * @param schemaExecSvc Schema executor service.
-     * @throws IgniteCheckedException If failed.
-     */
-    private void registerExecutorMBeans(ExecutorService execSvc,
-        ExecutorService sysExecSvc,
-        ExecutorService p2pExecSvc,
-        ExecutorService mgmtExecSvc,
-        ExecutorService restExecSvc,
-        ExecutorService qryExecSvc,
-        ExecutorService schemaExecSvc
-    ) throws IgniteCheckedException {if(U.IGNITE_MBEANS_DISABLED)
-            return;
-        pubExecSvcMBean = registerExecutorMBean(execSvc, "GridExecutionExecutor");
-        sysExecSvcMBean = registerExecutorMBean(sysExecSvc, "GridSystemExecutor");
-        mgmtExecSvcMBean = registerExecutorMBean(mgmtExecSvc, "GridManagementExecutor");
-        p2PExecSvcMBean = registerExecutorMBean(p2pExecSvc, "GridClassLoadingExecutor");
-        qryExecSvcMBean = registerExecutorMBean(qryExecSvc, "GridQueryExecutor");
-        schemaExecSvcMBean = registerExecutorMBean(schemaExecSvc, "GridSchemaExecutor");
-
-        ConnectorConfiguration clientCfg = cfg.getConnectorConfiguration();
-
-        if (clientCfg != null)
-            restExecSvcMBean = registerExecutorMBean(restExecSvc, "GridRestExecutor");
-    }
-
-    /**
-     * @param exec Executor service to register.
-     * @param name Property name for executor.
-     * @return Name for created MBean.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException {
-        assert exec != null;
-        assert !U.IGNITE_MBEANS_DISABLED;
-
-        try {
-            ObjectName res = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Thread Pools",
-                name,
-                new ThreadPoolMXBeanAdapter(exec),
-                ThreadPoolMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered executor service MBean: " + res);
-
-            return res;
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name +
-                ", exec=" + exec + ']', e);
-        }
-    }
-
-    /**
-     * @param stripedExecSvc Executor service.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private void registerStripedExecutorMBean(StripedExecutor stripedExecSvc) throws IgniteCheckedException {
-        if (stripedExecSvc == null || U.IGNITE_MBEANS_DISABLED)
-            return;
-
-        String name = "StripedExecutor";
-
-        try {
-            stripedExecSvcMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getIgniteInstanceName(),
-                "Thread Pools",
-                name,
-                new StripedExecutorMXBeanAdapter(stripedExecSvc),
-                StripedExecutorMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered executor service MBean: " + stripedExecSvcMBean);
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register executor service MBean [name="
-                + name + ", exec=" + stripedExecSvc + ']', e);
-        }
-    }
-
-    /**
-     * Unregisters given mbean.
-     *
-     * @param mbean MBean to unregister.
-     * @return {@code True} if successfully unregistered, {@code false} otherwise.
-     */
-    private boolean unregisterMBean(@Nullable ObjectName mbean) {
-        if (mbean == null)
-            return true;
-
-        assert !U.IGNITE_MBEANS_DISABLED;
-
-        try {
-            cfg.getMBeanServer().unregisterMBean(mbean);
-
-            if (log.isDebugEnabled())
-                log.debug("Unregistered MBean: " + mbean);
-
-            return true;
-        }
-        catch (JMException e) {
-            U.error(log, "Failed to unregister MBean.", e);
-
-            return false;
-        }
-    }
-
     /**
      * @param mgr Manager to start.
      * @throws IgniteCheckedException Throw in case of any errors.
@@ -2292,20 +2078,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 cache.blockGateways();
 
             // Unregister MBeans.
-            if (!(
-                unregisterMBean(pubExecSvcMBean) &
-                    unregisterMBean(sysExecSvcMBean) &
-                    unregisterMBean(mgmtExecSvcMBean) &
-                    unregisterMBean(p2PExecSvcMBean) &
-                    unregisterMBean(kernalMBean) &
-                    unregisterMBean(locNodeMBean) &
-                    unregisterMBean(allNodesMBean) &
-                    unregisterMBean(restExecSvcMBean) &
-                    unregisterMBean(qryExecSvcMBean) &
-                    unregisterMBean(schemaExecSvcMBean) &
-                    unregisterMBean(stripedExecSvcMBean)
-            ))
-                errOnStop = false;
+            if (!mBeansMgr.unregisterAllMBeans())
+                errOnStop = true;
 
             // Stop components in reverse order.
             for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious(); ) {
@@ -4193,6 +3967,179 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * Class that registers and unregisters MBeans for kernal.
+     */
+    private class MBeansManager {
+        /** MBean names stored to be unregistered later. */
+        private final Set<ObjectName> mBeanNames = new HashSet<>();
+
+        /**
+         * Registers all kernal MBeans (for kernal, metrics, thread pools).
+         *
+         * @param utilityCachePool Utility cache pool
+         * @param execSvc Executor service
+         * @param sysExecSvc System executor service
+         * @param stripedExecSvc Striped executor
+         * @param p2pExecSvc P2P executor service
+         * @param mgmtExecSvc Management executor service
+         * @param igfsExecSvc IGFS executor service
+         * @param dataStreamExecSvc data stream executor service
+         * @param restExecSvc Reset executor service
+         * @param affExecSvc Affinity executor service
+         * @param idxExecSvc Indexing executor service
+         * @param callbackExecSvc Callback executor service
+         * @param qryExecSvc Query executor service
+         * @param schemaExecSvc Schema executor service
+         * @param customExecSvcs Custom named executors
+         *
+         * @throws IgniteCheckedException if fails to register any of the MBeans
+         */
+        private void registerAllMBeans(
+            ExecutorService utilityCachePool,
+            final ExecutorService execSvc,
+            final ExecutorService svcExecSvc,
+            final ExecutorService sysExecSvc,
+            final StripedExecutor stripedExecSvc,
+            ExecutorService p2pExecSvc,
+            ExecutorService mgmtExecSvc,
+            ExecutorService igfsExecSvc,
+            StripedExecutor dataStreamExecSvc,
+            ExecutorService restExecSvc,
+            ExecutorService affExecSvc,
+            @Nullable ExecutorService idxExecSvc,
+            IgniteStripedThreadPoolExecutor callbackExecSvc,
+            ExecutorService qryExecSvc,
+            ExecutorService schemaExecSvc,
+            @Nullable final Map<String, ? extends ExecutorService> customExecSvcs
+        ) throws IgniteCheckedException {
+            if (U.IGNITE_MBEANS_DISABLED)
+                return;
+
+            // Kernal
+            registerMBean("Kernal", IgniteKernal.class.getSimpleName(), IgniteKernal.this, IgniteMXBean.class);
+
+            // Metrics
+            ClusterMetricsMXBean locMetricsBean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery());
+            registerMBean("Kernal", locMetricsBean.getClass().getSimpleName(), locMetricsBean, ClusterMetricsMXBean.class);
+            ClusterMetricsMXBean metricsBean = new ClusterMetricsMXBeanImpl(cluster());
+            registerMBean("Kernal", metricsBean.getClass().getSimpleName(), metricsBean, ClusterMetricsMXBean.class);
+
+            // Executors
+            registerExecutorMBean("GridUtilityCacheExecutor", utilityCachePool);
+            registerExecutorMBean("GridExecutionExecutor", execSvc);
+            registerExecutorMBean("GridServicesExecutor", svcExecSvc);
+            registerExecutorMBean("GridSystemExecutor", sysExecSvc);
+            registerExecutorMBean("GridClassLoadingExecutor", p2pExecSvc);
+            registerExecutorMBean("GridManagementExecutor", mgmtExecSvc);
+            registerExecutorMBean("GridIgfsExecutor", igfsExecSvc);
+            registerExecutorMBean("GridDataStreamExecutor", dataStreamExecSvc);
+            registerExecutorMBean("GridAffinityExecutor", affExecSvc);
+            registerExecutorMBean("GridCallbackExecutor", callbackExecSvc);
+            registerExecutorMBean("GridQueryExecutor", qryExecSvc);
+            registerExecutorMBean("GridSchemaExecutor", schemaExecSvc);
+
+            if (idxExecSvc != null)
+                registerExecutorMBean("GridIndexingExecutor", idxExecSvc);
+
+            if (cfg.getConnectorConfiguration() != null)
+                registerExecutorMBean("GridRestExecutor", restExecSvc);
+
+            if (stripedExecSvc != null) {
+                // striped executor uses a custom adapter
+                registerMBean("Thread Pools",
+                    "StripedExecutor",
+                    new StripedExecutorMXBeanAdapter(stripedExecSvc),
+                    StripedExecutorMXBean.class);
+            }
+
+            if (customExecSvcs != null) {
+                for (Map.Entry<String, ? extends ExecutorService> entry : customExecSvcs.entrySet())
+                    registerExecutorMBean(entry.getKey(), entry.getValue());
+            }
+        }
+
+        /**
+         * Registers a {@link ThreadPoolMXBean} for an executor.
+         *
+         * @param name name of the bean to register
+         * @param exec executor to register a bean for
+         *
+         * @throws IgniteCheckedException if registration fails.
+         */
+        private void registerExecutorMBean(String name, ExecutorService exec) throws IgniteCheckedException {
+            registerMBean("Thread Pools", name, new ThreadPoolMXBeanAdapter(exec), ThreadPoolMXBean.class);
+        }
+
+        /**
+         * Register an Ignite MBean.
+         *
+         * @param grp bean group name
+         * @param name bean name
+         * @param impl bean implementation
+         * @param itf bean interface
+         * @param <T> bean type
+         *
+         * @throws IgniteCheckedException if registration fails
+         */
+        private <T> void registerMBean(String grp, String name, T impl, Class<T> itf) throws IgniteCheckedException {
+            assert !U.IGNITE_MBEANS_DISABLED;
+
+            try {
+                ObjectName objName = U.registerMBean(
+                    cfg.getMBeanServer(),
+                    cfg.getIgniteInstanceName(),
+                    grp, name, impl, itf);
+
+                if (log.isDebugEnabled())
+                    log.debug("Registered MBean: " + objName);
+
+                mBeanNames.add(objName);
+            }
+            catch (JMException e) {
+                throw new IgniteCheckedException("Failed to register MBean " + name, e);
+            }
+        }
+
+        /**
+         * Unregisters all previously registered MBeans.
+         *
+         * @return {@code true} if all mbeans were unregistered successfully; {@code false} otherwise.
+         */
+        private boolean unregisterAllMBeans() {
+            boolean success = true;
+
+            for (ObjectName name : mBeanNames)
+                success = success && unregisterMBean(name);
+
+            return success;
+        }
+
+        /**
+         * Unregisters given MBean.
+         *
+         * @param mbean MBean to unregister.
+         * @return {@code true} if successfully unregistered, {@code false} otherwise.
+         */
+        private boolean unregisterMBean(ObjectName mbean) {
+            assert !U.IGNITE_MBEANS_DISABLED;
+
+            try {
+                cfg.getMBeanServer().unregisterMBean(mbean);
+
+                if (log.isDebugEnabled())
+                    log.debug("Unregistered MBean: " + mbean);
+
+                return true;
+            }
+            catch (JMException e) {
+                U.error(log, "Failed to unregister MBean.", e);
+
+                return false;
+            }
+        }
+    }
+
     /** {@inheritDoc} */
     @Override public void runIoTest(
         long warmup,

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
new file mode 100644
index 0000000..1c39982
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridMBeansTest.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.configuration.ExecutorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import javax.management.ObjectName;
+
+/**
+ * Tests for the standard JMX beans registered by the kernal.
+ */
+public class GridMBeansTest extends GridCommonAbstractTest {
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_0 = "Custom executor 0";
+
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_1 = "Custom executor 1";
+
+    /** Create test and auto-start the grid */
+    public GridMBeansTest() {
+        super(true);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * This implementation registers adds custom executors to the configuration.
+     */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setExecutorConfiguration(new ExecutorConfiguration(CUSTOM_EXECUTOR_0),
+            new ExecutorConfiguration(CUSTOM_EXECUTOR_1));
+
+        return cfg;
+    }
+
+    /** Check that kernal bean is available */
+    public void testKernalBeans() throws Exception {
+        checkBean("Kernal", "IgniteKernal", "InstanceName", grid().name());
+        checkBean("Kernal", "ClusterMetricsMXBeanImpl", "TotalServerNodes", 1);
+        checkBean("Kernal", "ClusterMetricsMXBeanImpl", "TotalServerNodes", 1);
+    }
+
+    /** Check that kernal bean is available */
+    public void testExecutorBeans() throws Exception {
+        // standard executors
+        checkBean("Thread Pools", "GridExecutionExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridSystemExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridManagementExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridClassLoadingExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridQueryExecutor", "Terminated", false);
+        checkBean("Thread Pools", "GridSchemaExecutor", "Terminated", false);
+        checkBean("Thread Pools", "StripedExecutor", "Terminated", false);
+
+        // custom executors
+        checkBean("Thread Pools", CUSTOM_EXECUTOR_0, "Terminated", false);
+        checkBean("Thread Pools", CUSTOM_EXECUTOR_1, "Terminated", false);
+    }
+
+    /** Checks that a bean with the specified group and name is available and has the expected attribute */
+    private void checkBean(String grp, String name, String attributeName, Object expAttributeVal) throws Exception {
+        ObjectName mBeanName = IgniteUtils.makeMBeanName(grid().name(), grp, name);
+        Object attributeVal = grid().configuration().getMBeanServer().getAttribute(mBeanName, attributeName);
+
+        assertEquals(expAttributeVal, attributeVal);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
index b6114d1..9edebd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridNodeMetricsLogSelfTest.java
@@ -21,6 +21,7 @@ package org.apache.ignite.internal;
 import java.io.StringWriter;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.ExecutorConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
@@ -35,6 +36,12 @@ import org.apache.log4j.WriterAppender;
 @SuppressWarnings({"ProhibitedExceptionDeclared"})
 @GridCommonTest(group = "Kernal")
 public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_0 = "Custom executor 0";
+
+    /** Executor name for setExecutorConfiguration */
+    private static final String CUSTOM_EXECUTOR_1 = "Custom executor 1";
+
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
@@ -42,6 +49,9 @@ public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
 
         cfg.setMetricsLogFrequency(1000);
 
+        cfg.setExecutorConfiguration(new ExecutorConfiguration(CUSTOM_EXECUTOR_0),
+            new ExecutorConfiguration(CUSTOM_EXECUTOR_1));
+
         return cfg;
     }
 
@@ -85,9 +95,20 @@ public class GridNodeMetricsLogSelfTest extends GridCommonAbstractTest {
 
         Logger.getRootLogger().removeAppender(app);
 
-        assert fullLog.contains("Metrics for local node");
-        assert fullLog.contains("uptime=");
-        assert fullLog.contains("Non heap");
-        assert fullLog.contains("Outbound messages queue");
+        String msg = "Metrics are missing in the log or have an unexpected format";
+
+        // don't check the format strictly, but check that all expected metrics are present
+        assertTrue(msg, fullLog.contains("Metrics for local node (to disable set 'metricsLogFrequency' to 0)"));
+        assertTrue(msg, fullLog.matches("(?s).*Node \\[id=.*, name=.*, uptime=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*H/N/C \\[hosts=.*, nodes=.*, CPUs=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*CPU \\[cur=.*, avg=.*, GC=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*PageMemory \\[pages=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Heap \\[used=.*, free=.*, comm=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Non heap \\[used=.*, free=.*, comm=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Outbound messages queue \\[size=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*Public thread pool \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*System thread pool \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*" + CUSTOM_EXECUTOR_0 + " \\[active=.*, idle=.*, qSize=.*].*"));
+        assertTrue(msg, fullLog.matches("(?s).*" + CUSTOM_EXECUTOR_1 + " \\[active=.*, idle=.*, qSize=.*].*"));
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/df3c4df3/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 7cdc670..2cc2047 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
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.ClusterGroupSelfTest;
 import org.apache.ignite.internal.GridFailFastNodeFailureDetectionSelfTest;
 import org.apache.ignite.internal.GridLifecycleAwareSelfTest;
 import org.apache.ignite.internal.GridLifecycleBeanSelfTest;
+import org.apache.ignite.internal.GridMBeansTest;
 import org.apache.ignite.internal.GridNodeMetricsLogSelfTest;
 import org.apache.ignite.internal.GridProjectionForCachesSelfTest;
 import org.apache.ignite.internal.GridReduceSelfTest;
@@ -149,6 +150,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(IgniteMarshallerCacheConcurrentReadWriteTest.class);
         suite.addTestSuite(GridNodeMetricsLogSelfTest.class);
         suite.addTestSuite(GridLocalIgniteSerializationTest.class);
+        suite.addTestSuite(GridMBeansTest.class);
 
         suite.addTestSuite(IgniteExceptionInNioWorkerSelfTest.class);
         suite.addTestSuite(IgniteLocalNodeMapBeforeStartTest.class);


[12/15] ignite git commit: IGNITE-5217: Gradient descent for OLS lin reg

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegression.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegression.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegression.java
deleted file mode 100644
index aafeae8..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegression.java
+++ /dev/null
@@ -1,257 +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;
-
-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.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.SingularMatrixException;
-import org.apache.ignite.ml.math.functions.Functions;
-
-/**
- * This class is based on the corresponding class from Apache Common Math lib.
- * <p>Implements ordinary least squares (OLS) to estimate the parameters of a
- * multiple linear regression model.</p>
- *
- * <p>The regression coefficients, <code>b</code>, satisfy the normal equations:
- * <pre><code> X<sup>T</sup> X b = X<sup>T</sup> y </code></pre></p>
- *
- * <p>To solve the normal equations, this implementation uses QR decomposition
- * of the <code>X</code> matrix. (See {@link QRDecomposition} for details on the
- * decomposition algorithm.) The <code>X</code> matrix, also known as the <i>design matrix,</i>
- * has rows corresponding to sample observations and columns corresponding to independent
- * variables.  When the model is estimated using an intercept term (i.e. when
- * {@link #isNoIntercept() isNoIntercept} is false as it is by default), the <code>X</code>
- * matrix includes an initial column identically equal to 1.  We solve the normal equations
- * as follows:
- * <pre><code> X<sup>T</sup>X b = X<sup>T</sup> y
- * (QR)<sup>T</sup> (QR) b = (QR)<sup>T</sup>y
- * R<sup>T</sup> (Q<sup>T</sup>Q) R b = R<sup>T</sup> Q<sup>T</sup> y
- * R<sup>T</sup> R b = R<sup>T</sup> Q<sup>T</sup> y
- * (R<sup>T</sup>)<sup>-1</sup> R<sup>T</sup> R b = (R<sup>T</sup>)<sup>-1</sup> R<sup>T</sup> Q<sup>T</sup> y
- * R b = Q<sup>T</sup> y </code></pre></p>
- *
- * <p>Given <code>Q</code> and <code>R</code>, the last equation is solved by back-substitution.</p>
- */
-public class OLSMultipleLinearRegression extends AbstractMultipleLinearRegression {
-    /** Cached QR decomposition of X matrix */
-    private QRDSolver solver = null;
-
-    /** Singularity threshold for QR decomposition */
-    private final double threshold;
-
-    /**
-     * Create an empty OLSMultipleLinearRegression instance.
-     */
-    public OLSMultipleLinearRegression() {
-        this(0d);
-    }
-
-    /**
-     * Create an empty OLSMultipleLinearRegression instance, using the given
-     * singularity threshold for the QR decomposition.
-     *
-     * @param threshold the singularity threshold
-     */
-    public OLSMultipleLinearRegression(final double threshold) {
-        this.threshold = threshold;
-    }
-
-    /**
-     * Loads model x and y sample data, overriding any previous sample.
-     *
-     * Computes and caches QR decomposition of the X matrix.
-     *
-     * @param y the {@code n}-sized vector representing the y sample
-     * @param x the {@code n x k} matrix representing the x sample
-     * @throws MathIllegalArgumentException if the x and y array data are not compatible for the regression
-     */
-    public void newSampleData(Vector y, Matrix x) throws MathIllegalArgumentException {
-        validateSampleData(x, y);
-        newYSampleData(y);
-        newXSampleData(x);
-    }
-
-    /**
-     * {@inheritDoc}
-     * <p>This implementation computes and caches the QR decomposition of the X matrix.</p>
-     */
-    @Override public void newSampleData(double[] data, int nobs, int nvars, Matrix like) {
-        super.newSampleData(data, nobs, nvars, like);
-        QRDecomposition qr = new QRDecomposition(getX(), threshold);
-        solver = new QRDSolver(qr.getQ(), qr.getR());
-    }
-
-    /**
-     * <p>Compute the "hat" matrix.
-     * </p>
-     * <p>The hat matrix is defined in terms of the design matrix X
-     * by X(X<sup>T</sup>X)<sup>-1</sup>X<sup>T</sup>
-     * </p>
-     * <p>The implementation here uses the QR decomposition to compute the
-     * hat matrix as Q I<sub>p</sub>Q<sup>T</sup> where I<sub>p</sub> is the
-     * p-dimensional identity matrix augmented by 0's.  This computational
-     * formula is from "The Hat Matrix in Regression and ANOVA",
-     * David C. Hoaglin and Roy E. Welsch,
-     * <i>The American Statistician</i>, Vol. 32, No. 1 (Feb., 1978), pp. 17-22.
-     * </p>
-     * <p>Data for the model must have been successfully loaded using one of
-     * the {@code newSampleData} methods before invoking this method; otherwise
-     * a {@code NullPointerException} will be thrown.</p>
-     *
-     * @return the hat matrix
-     * @throws NullPointerException unless method {@code newSampleData} has been called beforehand.
-     */
-    public Matrix calculateHat() {
-        return solver.calculateHat();
-    }
-
-    /**
-     * <p>Returns the sum of squared deviations of Y from its mean.</p>
-     *
-     * <p>If the model has no intercept term, <code>0</code> is used for the
-     * mean of Y - i.e., what is returned is the sum of the squared Y values.</p>
-     *
-     * <p>The value returned by this method is the SSTO value used in
-     * the {@link #calculateRSquared() R-squared} computation.</p>
-     *
-     * @return SSTO - the total sum of squares
-     * @throws NullPointerException if the sample has not been set
-     * @see #isNoIntercept()
-     */
-    public double calculateTotalSumOfSquares() {
-        if (isNoIntercept())
-            return getY().foldMap(Functions.PLUS, Functions.SQUARE, 0.0);
-        else {
-            // TODO: IGNITE-5826, think about incremental update formula.
-            final double mean = getY().sum() / getY().size();
-            return getY().foldMap(Functions.PLUS, x -> (mean - x) * (mean - x), 0.0);
-        }
-    }
-
-    /**
-     * Returns the sum of squared residuals.
-     *
-     * @return residual sum of squares
-     * @throws SingularMatrixException if the design matrix is singular
-     * @throws NullPointerException if the data for the model have not been loaded
-     */
-    public double calculateResidualSumOfSquares() {
-        final Vector residuals = calculateResiduals();
-        // No advertised DME, args are valid
-        return residuals.dot(residuals);
-    }
-
-    /**
-     * Returns the R-Squared statistic, defined by the formula <pre>
-     * R<sup>2</sup> = 1 - SSR / SSTO
-     * </pre>
-     * where SSR is the {@link #calculateResidualSumOfSquares() sum of squared residuals}
-     * and SSTO is the {@link #calculateTotalSumOfSquares() total sum of squares}
-     *
-     * <p>If there is no variance in y, i.e., SSTO = 0, NaN is returned.</p>
-     *
-     * @return R-square statistic
-     * @throws NullPointerException if the sample has not been set
-     * @throws SingularMatrixException if the design matrix is singular
-     */
-    public double calculateRSquared() {
-        return 1 - calculateResidualSumOfSquares() / calculateTotalSumOfSquares();
-    }
-
-    /**
-     * <p>Returns the adjusted R-squared statistic, defined by the formula <pre>
-     * R<sup>2</sup><sub>adj</sub> = 1 - [SSR (n - 1)] / [SSTO (n - p)]
-     * </pre>
-     * where SSR is the {@link #calculateResidualSumOfSquares() sum of squared residuals},
-     * SSTO is the {@link #calculateTotalSumOfSquares() total sum of squares}, n is the number
-     * of observations and p is the number of parameters estimated (including the intercept).</p>
-     *
-     * <p>If the regression is estimated without an intercept term, what is returned is <pre>
-     * <code> 1 - (1 - {@link #calculateRSquared()}) * (n / (n - p)) </code>
-     * </pre></p>
-     *
-     * <p>If there is no variance in y, i.e., SSTO = 0, NaN is returned.</p>
-     *
-     * @return adjusted R-Squared statistic
-     * @throws NullPointerException if the sample has not been set
-     * @throws SingularMatrixException if the design matrix is singular
-     * @see #isNoIntercept()
-     */
-    public double calculateAdjustedRSquared() {
-        final double n = getX().rowSize();
-        if (isNoIntercept())
-            return 1 - (1 - calculateRSquared()) * (n / (n - getX().columnSize()));
-        else
-            return 1 - (calculateResidualSumOfSquares() * (n - 1)) /
-                (calculateTotalSumOfSquares() * (n - getX().columnSize()));
-    }
-
-    /**
-     * {@inheritDoc}
-     * <p>This implementation computes and caches the QR decomposition of the X matrix
-     * once it is successfully loaded.</p>
-     */
-    @Override protected void newXSampleData(Matrix x) {
-        super.newXSampleData(x);
-        QRDecomposition qr = new QRDecomposition(getX());
-        solver = new QRDSolver(qr.getQ(), qr.getR());
-    }
-
-    /**
-     * Calculates the regression coefficients using OLS.
-     *
-     * <p>Data for the model must have been successfully loaded using one of
-     * the {@code newSampleData} methods before invoking this method; otherwise
-     * a {@code NullPointerException} will be thrown.</p>
-     *
-     * @return beta
-     * @throws SingularMatrixException if the design matrix is singular
-     * @throws NullPointerException if the data for the model have not been loaded
-     */
-    @Override protected Vector calculateBeta() {
-        return solver.solve(getY());
-    }
-
-    /**
-     * <p>Calculates the variance-covariance matrix of the regression parameters.
-     * </p>
-     * <p>Var(b) = (X<sup>T</sup>X)<sup>-1</sup>
-     * </p>
-     * <p>Uses QR decomposition to reduce (X<sup>T</sup>X)<sup>-1</sup>
-     * to (R<sup>T</sup>R)<sup>-1</sup>, with only the top p rows of
-     * R included, where p = the length of the beta vector.</p>
-     *
-     * <p>Data for the model must have been successfully loaded using one of
-     * the {@code newSampleData} methods before invoking this method; otherwise
-     * a {@code NullPointerException} will be thrown.</p>
-     *
-     * @return The beta variance-covariance matrix
-     * @throws SingularMatrixException if the design matrix is singular
-     * @throws NullPointerException if the data for the model have not been loaded
-     */
-    @Override protected Matrix calculateBetaVariance() {
-        return solver.calculateBetaVariance(getX().columnSize());
-    }
-
-    /** */
-    QRDSolver solver() {
-        return solver;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModel.java
deleted file mode 100644
index b95cbf3..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModel.java
+++ /dev/null
@@ -1,77 +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;
-
-import org.apache.ignite.ml.Exportable;
-import org.apache.ignite.ml.Exporter;
-import org.apache.ignite.ml.Model;
-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;
-
-/**
- * Model for linear regression.
- */
-public class OLSMultipleLinearRegressionModel implements Model<Vector, Vector>,
-    Exportable<OLSMultipleLinearRegressionModelFormat> {
-    /** */
-    private final Matrix xMatrix;
-    /** */
-    private final QRDSolver solver;
-
-    /**
-     * Construct linear regression model.
-     *
-     * @param xMatrix See {@link QRDecomposition#QRDecomposition(Matrix)}.
-     * @param solver Linear regression solver object.
-     */
-    public OLSMultipleLinearRegressionModel(Matrix xMatrix, QRDSolver solver) {
-        this.xMatrix = xMatrix;
-        this.solver = solver;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Vector apply(Vector val) {
-        return xMatrix.times(solver.solve(val));
-    }
-
-    /** {@inheritDoc} */
-    @Override public <P> void saveModel(Exporter<OLSMultipleLinearRegressionModelFormat, P> exporter, P path) {
-        exporter.save(new OLSMultipleLinearRegressionModelFormat(xMatrix, solver), path);
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        OLSMultipleLinearRegressionModel mdl = (OLSMultipleLinearRegressionModel)o;
-
-        return xMatrix.equals(mdl.xMatrix) && solver.equals(mdl.solver);
-    }
-
-    /** {@inheritDoc} */
-    @Override public int hashCode() {
-        int res = xMatrix.hashCode();
-        res = 31 * res + solver.hashCode();
-        return res;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelFormat.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelFormat.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelFormat.java
deleted file mode 100644
index fc44968..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionModelFormat.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.regressions;
-
-import java.io.Serializable;
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.decompositions.QRDSolver;
-
-/**
- * Linear regression model representation.
- *
- * @see OLSMultipleLinearRegressionModel
- */
-public class OLSMultipleLinearRegressionModelFormat implements Serializable {
-    /** X sample data. */
-    private final Matrix xMatrix;
-
-    /** Whether or not the regression model includes an intercept.  True means no intercept. */
-    private final QRDSolver solver;
-
-    /** */
-    public OLSMultipleLinearRegressionModelFormat(Matrix xMatrix, QRDSolver solver) {
-        this.xMatrix = xMatrix;
-        this.solver = solver;
-    }
-
-    /** */
-    public OLSMultipleLinearRegressionModel getOLSMultipleLinearRegressionModel() {
-        return new OLSMultipleLinearRegressionModel(xMatrix, solver);
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTrainer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTrainer.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTrainer.java
deleted file mode 100644
index dde0aca..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/OLSMultipleLinearRegressionTrainer.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.regressions;
-
-import org.apache.ignite.ml.Trainer;
-import org.apache.ignite.ml.math.Matrix;
-
-/**
- * Trainer for linear regression.
- */
-public class OLSMultipleLinearRegressionTrainer implements Trainer<OLSMultipleLinearRegressionModel, double[]> {
-    /** */
-    private final double threshold;
-
-    /** */
-    private final int nobs;
-
-    /** */
-    private final int nvars;
-
-    /** */
-    private final Matrix like;
-
-    /**
-     * Construct linear regression trainer.
-     *
-     * @param threshold the singularity threshold for QR decomposition
-     * @param nobs number of observations (rows)
-     * @param nvars number of independent variables (columns, not counting y)
-     * @param like matrix(maybe empty) indicating how data should be stored
-     */
-    public OLSMultipleLinearRegressionTrainer(double threshold, int nobs, int nvars, Matrix like) {
-        this.threshold = threshold;
-        this.nobs = nobs;
-        this.nvars = nvars;
-        this.like = like;
-    }
-
-    /** {@inheritDoc} */
-    @Override public OLSMultipleLinearRegressionModel train(double[] data) {
-        OLSMultipleLinearRegression regression = new OLSMultipleLinearRegression(threshold);
-
-        regression.newSampleData(data, nobs, nvars, like);
-
-        return new OLSMultipleLinearRegressionModel(regression.getX(), regression.solver());
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/RegressionsErrorMessages.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/RegressionsErrorMessages.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/RegressionsErrorMessages.java
deleted file mode 100644
index 883adca..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/RegressionsErrorMessages.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.regressions;
-
-/**
- * This class contains various messages used in regressions,
- */
-public class RegressionsErrorMessages {
-    /** Constant for string indicating that sample has insufficient observed points. */
-    static final String INSUFFICIENT_OBSERVED_POINTS_IN_SAMPLE = "Insufficient observed points in sample.";
-    /** */
-    static final String NOT_ENOUGH_DATA_FOR_NUMBER_OF_PREDICTORS = "Not enough data (%d rows) for this many predictors (%d predictors)";
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModel.java
new file mode 100644
index 0000000..6586a81
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionModel.java
@@ -0,0 +1,107 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.io.Serializable;
+import java.util.Objects;
+import org.apache.ignite.ml.Exportable;
+import org.apache.ignite.ml.Exporter;
+import org.apache.ignite.ml.Model;
+import org.apache.ignite.ml.math.Vector;
+
+/**
+ * Simple linear regression model which predicts result value Y as a linear combination of input variables:
+ * Y = weights * X + intercept.
+ */
+public class LinearRegressionModel implements Model<Vector, Double>, Exportable<LinearRegressionModel>, Serializable {
+    /** */
+    private static final long serialVersionUID = -105984600091550226L;
+
+    /** Multiplier of the objects's vector required to make prediction.  */
+    private final Vector weights;
+
+    /** Intercept of the linear regression model */
+    private final double intercept;
+
+    /** */
+    public LinearRegressionModel(Vector weights, double intercept) {
+        this.weights = weights;
+        this.intercept = intercept;
+    }
+
+    /** */
+    public Vector getWeights() {
+        return weights;
+    }
+
+    /** */
+    public double getIntercept() {
+        return intercept;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Double apply(Vector input) {
+        return input.dot(weights) + intercept;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <P> void saveModel(Exporter<LinearRegressionModel, P> exporter, P path) {
+        exporter.save(this, path);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+        LinearRegressionModel mdl = (LinearRegressionModel)o;
+        return Double.compare(mdl.intercept, intercept) == 0 &&
+            Objects.equals(weights, mdl.weights);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+
+        return Objects.hash(weights, intercept);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        if (weights.size() < 10) {
+            StringBuilder builder = new StringBuilder();
+
+            for (int i = 0; i < weights.size(); i++) {
+                double nextItem = i == weights.size() - 1 ? intercept : weights.get(i + 1);
+
+                builder.append(String.format("%.4f", Math.abs(weights.get(i))))
+                    .append("*x")
+                    .append(i)
+                    .append(nextItem > 0 ? " + " : " - ");
+            }
+
+            builder.append(String.format("%.4f", Math.abs(intercept)));
+            return builder.toString();
+        }
+
+        return "LinearRegressionModel{" +
+            "weights=" + weights +
+            ", intercept=" + intercept +
+            '}';
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..5de3cda
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionQRTrainer.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/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
new file mode 100644
index 0000000..aad4c7a
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/LinearRegressionSGDTrainer.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.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;
+
+/**
+ * Linear regression trainer based on least squares loss function and gradient descent optimization algorithm.
+ */
+public class LinearRegressionSGDTrainer implements Trainer<LinearRegressionModel, Matrix> {
+    /**
+     * Gradient descent optimizer.
+     */
+    private final GradientDescent gradientDescent;
+
+    /** */
+    public LinearRegressionSGDTrainer(GradientDescent gradientDescent) {
+        this.gradientDescent = gradientDescent;
+    }
+
+    /** */
+    public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol) {
+        this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new BarzilaiBorweinUpdater())
+            .withMaxIterations(maxIterations)
+            .withConvergenceTol(convergenceTol);
+    }
+
+    /** */
+    public LinearRegressionSGDTrainer(int maxIterations, double convergenceTol, double learningRate) {
+        this.gradientDescent = new GradientDescent(new LeastSquaresGradientFunction(), new SimpleUpdater(learningRate))
+            .withMaxIterations(maxIterations)
+            .withConvergenceTol(convergenceTol);
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override public LinearRegressionModel train(Matrix data) {
+        Vector variables = gradientDescent.optimize(data, data.likeVector(data.columnSize()));
+        Vector weights = variables.viewPart(1, variables.size() - 1);
+
+        double intercept = variables.get(0);
+
+        return new LinearRegressionModel(weights, intercept);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/package-info.java
new file mode 100644
index 0000000..086a824
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/regressions/linear/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 various linear regressions.
+ */
+package org.apache.ignite.ml.regressions.linear;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java
index 37dec77..862a9c1 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/LocalModelsTest.java
@@ -28,9 +28,8 @@ import org.apache.ignite.ml.knn.models.KNNModelFormat;
 import org.apache.ignite.ml.knn.models.KNNStrategy;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.regressions.OLSMultipleLinearRegressionModel;
-import org.apache.ignite.ml.regressions.OLSMultipleLinearRegressionModelFormat;
-import org.apache.ignite.ml.regressions.OLSMultipleLinearRegressionTrainer;
+import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
+import org.apache.ignite.ml.regressions.linear.LinearRegressionModel;
 import org.apache.ignite.ml.structures.LabeledDataset;
 import org.junit.Assert;
 import org.junit.Test;
@@ -63,21 +62,16 @@ public class LocalModelsTest {
 
     /** */
     @Test
-    public void importExportOLSMultipleLinearRegressionModelTest() throws IOException {
+    public void importExportLinearRegressionModelTest() throws IOException {
         executeModelTest(mdlFilePath -> {
-            OLSMultipleLinearRegressionModel mdl = getAbstractMultipleLinearRegressionModel();
+            LinearRegressionModel model = new LinearRegressionModel(new DenseLocalOnHeapVector(new double[]{1, 2}), 3);
+            Exporter<LinearRegressionModel, String> exporter = new FileExporter<>();
+            model.saveModel(exporter, mdlFilePath);
 
-            Exporter<OLSMultipleLinearRegressionModelFormat, String> exporter = new FileExporter<>();
-
-            mdl.saveModel(exporter, mdlFilePath);
-
-            OLSMultipleLinearRegressionModelFormat load = exporter.load(mdlFilePath);
+            LinearRegressionModel load = exporter.load(mdlFilePath);
 
             Assert.assertNotNull(load);
-
-            OLSMultipleLinearRegressionModel importedMdl = load.getOLSMultipleLinearRegressionModel();
-
-            Assert.assertTrue("", mdl.equals(importedMdl));
+            Assert.assertEquals("", model, load);
 
             return null;
         });
@@ -114,24 +108,6 @@ public class LocalModelsTest {
     }
 
     /** */
-    private OLSMultipleLinearRegressionModel getAbstractMultipleLinearRegressionModel() {
-        double[] data = new double[] {
-            0, 0, 0, 0, 0, 0, // IMPL NOTE values in this row are later replaced (with 1.0)
-            0, 2.0, 0, 0, 0, 0,
-            0, 0, 3.0, 0, 0, 0,
-            0, 0, 0, 4.0, 0, 0,
-            0, 0, 0, 0, 5.0, 0,
-            0, 0, 0, 0, 0, 6.0};
-
-        final int nobs = 6, nvars = 5;
-
-        OLSMultipleLinearRegressionTrainer trainer
-            = new OLSMultipleLinearRegressionTrainer(0, nobs, nvars, new DenseLocalOnHeapMatrix(1, 1));
-
-        return trainer.train(data);
-    }
-
-    /** */
     @Test
     public void importExportKNNModelTest() throws IOException {
         executeModelTest(mdlFilePath -> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/b2060855/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
new file mode 100644
index 0000000..f6f4775
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/GradientDescentTest.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+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/b2060855/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
new file mode 100644
index 0000000..9017c43
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/optimization/util/SparseDistributedMatrixMapReducerTest.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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/b2060855/modules/ml/src/test/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegressionTest.java
deleted file mode 100644
index 6ad56a5..0000000
--- a/modules/ml/src/test/java/org/apache/ignite/ml/regressions/AbstractMultipleLinearRegressionTest.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.ml.regressions;
-
-import org.apache.ignite.ml.math.Matrix;
-import org.apache.ignite.ml.math.Vector;
-import org.apache.ignite.ml.math.exceptions.MathIllegalArgumentException;
-import org.apache.ignite.ml.math.exceptions.NullArgumentException;
-import org.apache.ignite.ml.math.impls.matrix.DenseLocalOnHeapMatrix;
-import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * This class is based on the corresponding class from Apache Common Math lib.
- * Abstract base class for implementations of {@link MultipleLinearRegression}.
- */
-public abstract class AbstractMultipleLinearRegressionTest {
-    /** */
-    protected AbstractMultipleLinearRegression regression;
-
-    /** */
-    @Before
-    public void setUp() {
-        regression = createRegression();
-    }
-
-    /** */
-    protected abstract AbstractMultipleLinearRegression createRegression();
-
-    /** */
-    protected abstract int getNumberOfRegressors();
-
-    /** */
-    protected abstract int getSampleSize();
-
-    /** */
-    @Test
-    public void canEstimateRegressionParameters() {
-        double[] beta = regression.estimateRegressionParameters();
-        Assert.assertEquals(getNumberOfRegressors(), beta.length);
-    }
-
-    /** */
-    @Test
-    public void canEstimateResiduals() {
-        double[] e = regression.estimateResiduals();
-        Assert.assertEquals(getSampleSize(), e.length);
-    }
-
-    /** */
-    @Test
-    public void canEstimateRegressionParametersVariance() {
-        Matrix var = regression.estimateRegressionParametersVariance();
-        Assert.assertEquals(getNumberOfRegressors(), var.rowSize());
-    }
-
-    /** */
-    @Test
-    public void canEstimateRegressandVariance() {
-        if (getSampleSize() > getNumberOfRegressors()) {
-            double variance = regression.estimateRegressandVariance();
-            Assert.assertTrue(variance > 0.0);
-        }
-    }
-
-    /**
-     * Verifies that newSampleData methods consistently insert unitary columns
-     * in design matrix.  Confirms the fix for MATH-411.
-     */
-    @Test
-    public void testNewSample() {
-        double[] design = new double[] {
-            1, 19, 22, 33,
-            2, 20, 30, 40,
-            3, 25, 35, 45,
-            4, 27, 37, 47
-        };
-
-        double[] y = new double[] {1, 2, 3, 4};
-
-        double[][] x = new double[][] {
-            {19, 22, 33},
-            {20, 30, 40},
-            {25, 35, 45},
-            {27, 37, 47}
-        };
-
-        AbstractMultipleLinearRegression regression = createRegression();
-        regression.newSampleData(design, 4, 3, new DenseLocalOnHeapMatrix());
-
-        Matrix flatX = regression.getX().copy();
-        Vector flatY = regression.getY().copy();
-
-        regression.newXSampleData(new DenseLocalOnHeapMatrix(x));
-        regression.newYSampleData(new DenseLocalOnHeapVector(y));
-
-        Assert.assertEquals(flatX, regression.getX());
-        Assert.assertEquals(flatY, regression.getY());
-
-        // No intercept
-        regression.setNoIntercept(true);
-        regression.newSampleData(design, 4, 3, new DenseLocalOnHeapMatrix());
-
-        flatX = regression.getX().copy();
-        flatY = regression.getY().copy();
-
-        regression.newXSampleData(new DenseLocalOnHeapMatrix(x));
-        regression.newYSampleData(new DenseLocalOnHeapVector(y));
-
-        Assert.assertEquals(flatX, regression.getX());
-        Assert.assertEquals(flatY, regression.getY());
-    }
-
-    /** */
-    @Test(expected = NullArgumentException.class)
-    public void testNewSampleNullData() {
-        double[] data = null;
-        createRegression().newSampleData(data, 2, 3, new DenseLocalOnHeapMatrix());
-    }
-
-    /** */
-    @Test(expected = MathIllegalArgumentException.class)
-    public void testNewSampleInvalidData() {
-        double[] data = new double[] {1, 2, 3, 4};
-        createRegression().newSampleData(data, 2, 3, new DenseLocalOnHeapMatrix());
-    }
-
-    /** */
-    @Test(expected = MathIllegalArgumentException.class)
-    public void testNewSampleInsufficientData() {
-        double[] data = new double[] {1, 2, 3, 4};
-        createRegression().newSampleData(data, 1, 3, new DenseLocalOnHeapMatrix());
-    }
-
-    /** */
-    @Test(expected = NullArgumentException.class)
-    public void testXSampleDataNull() {
-        createRegression().newXSampleData(null);
-    }
-
-    /** */
-    @Test(expected = NullArgumentException.class)
-    public void testYSampleDataNull() {
-        createRegression().newYSampleData(null);
-    }
-
-}


[15/15] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-zk

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-zk


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

Branch: refs/heads/ignite-zk
Commit: 4c63e4238220e8d7dcd009201b893adda8b27aed
Parents: 26ffa0d a1b1f6c
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 29 12:27:45 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 29 12:27:45 2017 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    | 198 ++--
 .../KNNClassificationExample.java               |   9 +-
 .../ml/knn/regression/KNNRegressionExample.java |  13 +-
 .../DistributedRegressionExample.java           | 149 ---
 .../DistributedRegressionModelExample.java      | 134 ---
 ...tedLinearRegressionExampleWithQRTrainer.java | 136 +++
 ...edLinearRegressionExampleWithSGDTrainer.java | 137 +++
 .../ml/regression/linear/package-info.java      |  22 +
 .../ignite/jdbc/JdbcErrorsAbstractSelfTest.java |   2 +-
 .../ignite/codegen/MessageCodeGenerator.java    |   8 +-
 .../junits/IgniteCompatibilityAbstractTest.java |   7 +-
 .../junits/IgniteCompatibilityNodeRunner.java   |  15 +-
 .../apache/ignite/internal/IgniteKernal.java    | 499 +++++-----
 .../ignite/internal/binary/BinaryContext.java   |  66 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 168 ++--
 .../processors/hadoop/HadoopClassLoader.java    |   5 +-
 .../ClientCacheConfigurationSerializer.java     | 284 ++++--
 .../processors/query/GridQueryIndexing.java     |  15 +
 .../processors/query/GridQueryProcessor.java    | 133 ++-
 .../internal/processors/query/QuerySchema.java  |  30 +-
 .../query/QueryTypeDescriptorImpl.java          |  19 +
 .../internal/processors/query/QueryUtils.java   |  69 ++
 .../SchemaAlterTableDropColumnOperation.java    |  95 ++
 .../ignite/internal/util/IgniteUtils.java       |  61 +-
 .../apache/ignite/internal/GridMBeansTest.java  |  86 ++
 .../internal/GridNodeMetricsLogSelfTest.java    |  29 +-
 ...IgniteClientCacheInitializationFailTest.java |   6 +
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 .../hadoop/HadoopTestClassLoader.java           |   5 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  33 +
 .../query/h2/ddl/DdlStatementsProcessor.java    |  55 ++
 .../processors/query/h2/opt/GridH2Table.java    |  58 ++
 .../h2/sql/GridSqlAlterTableDropColumn.java     | 113 +++
 .../query/h2/sql/GridSqlQueryParser.java        |  45 +-
 ...ynamicColumnsAbstractConcurrentSelfTest.java | 238 ++++-
 .../cache/index/DynamicColumnsAbstractTest.java |  59 +-
 .../H2DynamicColumnsAbstractBasicSelfTest.java  | 324 ++++++-
 .../IgnitePersistentStoreSchemaLoadTest.java    |   8 +-
 .../apache/ignite/ml/knn/models/KNNModel.java   |   2 +-
 .../ignite/ml/knn/models/Normalization.java     |  32 -
 .../matrix/SparseBlockDistributedMatrix.java    |   5 +-
 .../ml/optimization/BarzilaiBorweinUpdater.java |  51 ++
 .../ignite/ml/optimization/GradientDescent.java | 201 +++++
 .../ml/optimization/GradientFunction.java       |  31 +
 .../LeastSquaresGradientFunction.java           |  33 +
 .../ignite/ml/optimization/SimpleUpdater.java   |  45 +
 .../apache/ignite/ml/optimization/Updater.java  |  30 +
 .../ignite/ml/optimization/package-info.java    |  22 +
 .../util/SparseDistributedMatrixMapReducer.java |  84 ++
 .../ml/optimization/util/package-info.java      |  22 +
 .../AbstractMultipleLinearRegression.java       | 378 --------
 .../regressions/MultipleLinearRegression.java   |  71 --
 .../OLSMultipleLinearRegression.java            | 257 ------
 .../OLSMultipleLinearRegressionModel.java       |  77 --
 .../OLSMultipleLinearRegressionModelFormat.java |  46 -
 .../OLSMultipleLinearRegressionTrainer.java     |  62 --
 .../regressions/RegressionsErrorMessages.java   |  28 -
 .../linear/LinearRegressionModel.java           | 107 +++
 .../linear/LinearRegressionQRTrainer.java       |  72 ++
 .../linear/LinearRegressionSGDTrainer.java      |  67 ++
 .../ml/regressions/linear/package-info.java     |  22 +
 .../apache/ignite/ml/structures/Dataset.java    | 232 +++++
 .../apache/ignite/ml/structures/DatasetRow.java |  79 ++
 .../ignite/ml/structures/FeatureMetadata.java   |  82 ++
 .../ignite/ml/structures/LabeledDataset.java    | 338 +------
 .../structures/LabeledDatasetTestTrainPair.java |   8 +-
 .../ignite/ml/structures/LabeledVector.java     |  53 +-
 .../preprocessing/LabeledDatasetLoader.java     | 133 +++
 .../preprocessing/LabellingMachine.java         |  41 +
 .../ml/structures/preprocessing/Normalizer.java |  78 ++
 .../structures/preprocessing/package-info.java  |  22 +
 .../org/apache/ignite/ml/LocalModelsTest.java   |  40 +-
 .../org/apache/ignite/ml/knn/BaseKNNTest.java   |   3 +-
 .../ml/knn/KNNMultipleLinearRegressionTest.java |   6 +-
 .../ignite/ml/knn/LabeledDatasetTest.java       |  25 +-
 .../ignite/ml/math/ExternalizableTest.java      |  67 ++
 .../ml/optimization/GradientDescentTest.java    |  64 ++
 .../SparseDistributedMatrixMapReducerTest.java  | 135 +++
 .../AbstractMultipleLinearRegressionTest.java   | 164 ----
 ...tedBlockOLSMultipleLinearRegressionTest.java | 901 ------------------
 ...tributedOLSMultipleLinearRegressionTest.java | 903 -------------------
 .../OLSMultipleLinearRegressionModelTest.java   |  53 --
 .../OLSMultipleLinearRegressionTest.java        | 820 -----------------
 .../ml/regressions/RegressionsTestSuite.java    |  20 +-
 .../linear/ArtificialRegressionDatasets.java    | 404 +++++++++
 ...istributedLinearRegressionQRTrainerTest.java |  36 +
 ...stributedLinearRegressionSGDTrainerTest.java |  35 +
 ...istributedLinearRegressionQRTrainerTest.java |  36 +
 ...stributedLinearRegressionSGDTrainerTest.java |  35 +
 .../GenericLinearRegressionTrainerTest.java     | 206 +++++
 ...wareAbstractLinearRegressionTrainerTest.java | 124 +++
 .../linear/LinearRegressionModelTest.java       |  66 ++
 .../LocalLinearRegressionQRTrainerTest.java     |  36 +
 .../LocalLinearRegressionSGDTrainerTest.java    |  35 +
 .../resources/datasets/regression/README.md     |  98 ++
 .../resources/datasets/regression/boston.csv    | 506 +++++++++++
 .../resources/datasets/regression/diabetes.csv  | 442 +++++++++
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj  |   5 +
 .../Cache/CacheConfigurationTest.cs             |   3 +-
 .../Cache/ClientCacheConfigurationTest.cs       |  29 +-
 .../Client/Cache/CreateCacheTest.cs             |  42 +-
 .../Client/ClientTestBase.cs                    |  18 +
 .../Client/Cache/CacheClientConfiguration.cs    |   9 +-
 .../ClientConnectorConfiguration.cs             |   2 +-
 .../Cache/ClientCacheConfigurationSerializer.cs | 123 ++-
 .../ignite/tools/classgen/ClassesGenerator.java |  69 +-
 .../uri/GridUriDeploymentFileProcessor.java     |  28 +-
 modules/web-console/backend/index.js            |   6 +-
 .../backend/migrations/recreate-index.js        |  30 +
 .../jdbc/JdbcPutIndexedValue8Benchmark.java     |   3 -
 ...iteOLSMultipleLinearRegressionBenchmark.java |  67 +-
 packaging/rpm/SOURCES/name.service              |  17 +
 packaging/rpm/SOURCES/service.sh                |  28 +
 packaging/rpm/SPECS/apache-ignite.spec          | 206 +++++
 parent/pom.xml                                  |   1 +
 115 files changed, 6937 insertions(+), 5155 deletions(-)
----------------------------------------------------------------------


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


[04/15] ignite git commit: IGNITE-7205: Dataset API

Posted by sb...@apache.org.
IGNITE-7205: Dataset API

this closes #3303


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

Branch: refs/heads/ignite-zk
Commit: 429f9544a6935d2c087e0ccbfb46f65e8723b57b
Parents: ee2a6f7
Author: zaleslaw <za...@gmail.com>
Authored: Thu Dec 28 16:31:42 2017 +0300
Committer: Yury Babak <yb...@gridgain.com>
Committed: Thu Dec 28 16:31:42 2017 +0300

----------------------------------------------------------------------
 .../KNNClassificationExample.java               |   9 +-
 .../ml/knn/regression/KNNRegressionExample.java |  13 +-
 .../apache/ignite/ml/knn/models/KNNModel.java   |   2 +-
 .../ignite/ml/knn/models/Normalization.java     |  32 --
 .../apache/ignite/ml/structures/Dataset.java    | 232 +++++++++++++
 .../apache/ignite/ml/structures/DatasetRow.java |  79 +++++
 .../ignite/ml/structures/FeatureMetadata.java   |  82 +++++
 .../ignite/ml/structures/LabeledDataset.java    | 338 ++-----------------
 .../structures/LabeledDatasetTestTrainPair.java |   8 +-
 .../ignite/ml/structures/LabeledVector.java     |  53 +--
 .../preprocessing/LabeledDatasetLoader.java     | 133 ++++++++
 .../preprocessing/LabellingMachine.java         |  41 +++
 .../ml/structures/preprocessing/Normalizer.java |  78 +++++
 .../structures/preprocessing/package-info.java  |  22 ++
 .../org/apache/ignite/ml/knn/BaseKNNTest.java   |   3 +-
 .../ml/knn/KNNMultipleLinearRegressionTest.java |   6 +-
 .../ignite/ml/knn/LabeledDatasetTest.java       |  25 +-
 .../ignite/ml/math/ExternalizableTest.java      |  67 ++++
 18 files changed, 837 insertions(+), 386 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/examples/src/main/ml/org/apache/ignite/examples/ml/knn/classification/KNNClassificationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/knn/classification/KNNClassificationExample.java b/examples/src/main/ml/org/apache/ignite/examples/ml/knn/classification/KNNClassificationExample.java
index fb7eebd..efdacd7 100644
--- a/examples/src/main/ml/org/apache/ignite/examples/ml/knn/classification/KNNClassificationExample.java
+++ b/examples/src/main/ml/org/apache/ignite/examples/ml/knn/classification/KNNClassificationExample.java
@@ -30,6 +30,8 @@ import org.apache.ignite.ml.knn.models.KNNStrategy;
 import org.apache.ignite.ml.math.distances.EuclideanDistance;
 import org.apache.ignite.ml.structures.LabeledDataset;
 import org.apache.ignite.ml.structures.LabeledDatasetTestTrainPair;
+import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
+import org.apache.ignite.ml.structures.preprocessing.LabellingMachine;
 import org.apache.ignite.thread.IgniteThread;
 
 /**
@@ -71,7 +73,7 @@ public class KNNClassificationExample {
                     Path path = Paths.get(KNNClassificationExample.class.getClassLoader().getResource(KNN_IRIS_TXT).toURI());
 
                     // Read dataset from file
-                    LabeledDataset dataset = LabeledDataset.loadTxt(path, SEPARATOR, true, false);
+                    LabeledDataset dataset = LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, true, false);
 
                     // Random splitting of iris data as 70% train and 30% test datasets
                     LabeledDatasetTestTrainPair split = new LabeledDatasetTestTrainPair(dataset, 0.3);
@@ -88,10 +90,7 @@ public class KNNClassificationExample {
                     final double[] labels = test.labels();
 
                     // Save predicted classes to test dataset
-                    for (int i = 0; i < test.rowSize(); i++) {
-                        double predictedCls = knnMdl.apply(test.getRow(i).features());
-                        test.setLabel(i, predictedCls);
-                    }
+                    LabellingMachine.assignLabels(test, knnMdl);
 
                     // Calculate amount of errors on test dataset
                     int amountOfErrors = 0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/examples/src/main/ml/org/apache/ignite/examples/ml/knn/regression/KNNRegressionExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/ml/org/apache/ignite/examples/ml/knn/regression/KNNRegressionExample.java b/examples/src/main/ml/org/apache/ignite/examples/ml/knn/regression/KNNRegressionExample.java
index 6ed0dd6..31f7191 100644
--- a/examples/src/main/ml/org/apache/ignite/examples/ml/knn/regression/KNNRegressionExample.java
+++ b/examples/src/main/ml/org/apache/ignite/examples/ml/knn/regression/KNNRegressionExample.java
@@ -26,11 +26,13 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.examples.ExampleNodeStartup;
 import org.apache.ignite.examples.ml.knn.classification.KNNClassificationExample;
 import org.apache.ignite.ml.knn.models.KNNStrategy;
-import org.apache.ignite.ml.knn.models.Normalization;
 import org.apache.ignite.ml.knn.regression.KNNMultipleLinearRegression;
 import org.apache.ignite.ml.math.distances.ManhattanDistance;
 import org.apache.ignite.ml.structures.LabeledDataset;
 import org.apache.ignite.ml.structures.LabeledDatasetTestTrainPair;
+import org.apache.ignite.ml.structures.preprocessing.LabeledDatasetLoader;
+import org.apache.ignite.ml.structures.preprocessing.LabellingMachine;
+import org.apache.ignite.ml.structures.preprocessing.Normalizer;
 import org.apache.ignite.thread.IgniteThread;
 
 /**
@@ -72,10 +74,10 @@ public class KNNRegressionExample {
                     Path path = Paths.get(KNNClassificationExample.class.getClassLoader().getResource(KNN_CLEARED_MACHINES_TXT).toURI());
 
                     // Read dataset from file
-                    LabeledDataset dataset = LabeledDataset.loadTxt(path, SEPARATOR, false, false);
+                    LabeledDataset dataset = LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, false);
 
                     // Normalize dataset
-                    dataset.normalizeWith(Normalization.MINIMAX);
+                    Normalizer.normalizeWithMiniMax(dataset);
 
                     // Random splitting of iris data as 80% train and 20% test datasets
                     LabeledDatasetTestTrainPair split = new LabeledDatasetTestTrainPair(dataset, 0.2);
@@ -93,10 +95,7 @@ public class KNNRegressionExample {
                     final double[] labels = test.labels();
 
                     // Save predicted classes to test dataset
-                    for (int i = 0; i < test.rowSize(); i++) {
-                        double predictedCls = knnMdl.apply(test.getRow(i).features());
-                        test.setLabel(i, predictedCls);
-                    }
+                    LabellingMachine.assignLabels(test, knnMdl);
 
                     // Calculate mean squared error (MSE)
                     double mse = 0.0;

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/KNNModel.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/KNNModel.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/KNNModel.java
index d3dff8c..3951be4 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/KNNModel.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/KNNModel.java
@@ -96,7 +96,7 @@ public class KNNModel implements Model<Vector, Double>, Exportable<KNNModelForma
      * @return K-nearest neighbors.
      */
     protected LabeledVector[] findKNearestNeighbors(Vector v, boolean isCashedDistance) {
-        LabeledVector[] trainingData = training.data();
+        LabeledVector[] trainingData = (LabeledVector[])training.data();
 
         TreeMap<Double, Set<Integer>> distanceIdxPairs = getDistances(v, trainingData);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/Normalization.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/Normalization.java b/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/Normalization.java
deleted file mode 100644
index aa4b291..0000000
--- a/modules/ml/src/main/java/org/apache/ignite/ml/knn/models/Normalization.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.knn.models;
-
-/** This enum contains names of different normalization approaches. */
-public enum Normalization {
-    /** Minimax.
-     *
-     * x'=(x-MIN[X])/(MAX[X]-MIN[X])
-     */
-    MINIMAX,
-    /** Z normalization.
-     *
-     * x'=(x-M[X])/\sigma [X]
-     */
-    Z_NORMALIZATION
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/Dataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/Dataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/Dataset.java
new file mode 100644
index 0000000..cb50516
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/Dataset.java
@@ -0,0 +1,232 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import java.util.Arrays;
+import org.apache.ignite.ml.math.Vector;
+
+/**
+ * Class for set of vectors. This is a base class in hierarchy of datasets.
+ */
+public class Dataset<Row extends DatasetRow> implements Serializable, Externalizable {
+    /** Data to keep. */
+    protected Row[] data;
+
+    /** Metadata to identify feature. */
+    protected FeatureMetadata[] meta;
+
+    /** Amount of instances. */
+    protected int rowSize;
+
+    /** Amount of attributes in each vector. */
+    protected int colSize;
+
+    /**
+     * Default constructor (required by Externalizable).
+     */
+    public Dataset(){}
+
+    /**
+     * Creates new Dataset by given data.
+     *
+     * @param data Given data. Should be initialized with one vector at least.
+     * @param meta Feature's metadata.
+     */
+    public Dataset(Row[] data, FeatureMetadata[] meta) {
+        this.data = data;
+        this.meta = meta;
+    }
+
+    /**
+     * Creates new Dataset by given data.
+     *
+     * @param data Given data. Should be initialized with one vector at least.
+     * @param featureNames Column names.
+     * @param colSize Amount of observed attributes in each vector.
+     */
+    public Dataset(Row[] data, String[] featureNames, int colSize) {
+        this(data.length, colSize, featureNames);
+
+        assert data != null;
+
+        this.data = data;
+    }
+
+    /**
+     * Creates new Dataset by given data.
+     *
+     * @param data Should be initialized with one vector at least.
+     * @param colSize Amount of observed attributes in each vector.
+     */
+    public Dataset(Row[] data, int colSize) {
+        this(data, null, colSize);
+    }
+
+    /**
+     * Creates new Dataset and initialized with empty data structure.
+     *
+     * @param rowSize Amount of instances. Should be > 0.
+     * @param colSize Amount of attributes. Should be > 0
+     * @param featureNames Column names.
+     */
+    public Dataset(int rowSize, int colSize, String[] featureNames) {
+        assert rowSize > 0;
+        assert colSize > 0;
+
+        if (featureNames == null)
+            generateFeatureNames();
+        else {
+            assert colSize == featureNames.length;
+            convertStringNamesToFeatureMetadata(featureNames);
+        }
+
+        this.rowSize = rowSize;
+        this.colSize = colSize;
+    }
+
+    /** */
+    protected void convertStringNamesToFeatureMetadata(String[] featureNames) {
+        this.meta = new FeatureMetadata[featureNames.length];
+        for (int i = 0; i < featureNames.length; i++)
+            this.meta[i] = new FeatureMetadata(featureNames[i]);
+    }
+
+    /** */
+    protected void generateFeatureNames() {
+        String[] featureNames = new String[colSize];
+
+        for (int i = 0; i < colSize; i++)
+            featureNames[i] = "f_" + i;
+
+        convertStringNamesToFeatureMetadata(featureNames);
+    }
+
+    /**
+     * Returns feature name for column with given index.
+     *
+     * @param i The given index.
+     * @return Feature name.
+     */
+    public String getFeatureName(int i) {
+        return meta[i].name();
+    }
+
+    /** */
+    public DatasetRow[] data() {
+        return data;
+    }
+
+    /** */
+    public void setData(Row[] data) {
+        this.data = data;
+    }
+
+    /** */
+    public FeatureMetadata[] meta() {
+        return meta;
+    }
+
+    /** */
+    public void setMeta(FeatureMetadata[] meta) {
+        this.meta = meta;
+    }
+
+    /**
+     * Gets amount of attributes.
+     *
+     * @return Amount of attributes in each Labeled Vector.
+     */
+    public int colSize() {
+        return colSize;
+    }
+
+    /**
+     * Gets amount of observation.
+     *
+     * @return Amount of rows in dataset.
+     */
+    public int rowSize() {
+        return rowSize;
+    }
+
+    /**
+     * Retrieves Labeled Vector by given index.
+     *
+     * @param idx Index of observation.
+     * @return Labeled features.
+     */
+    public Row getRow(int idx) {
+        return data[idx];
+    }
+
+    /**
+     * Get the features.
+     *
+     * @param idx Index of observation.
+     * @return Vector with features.
+     */
+    public Vector features(int idx) {
+        assert idx < rowSize;
+        assert data != null;
+        assert data[idx] != null;
+
+        return data[idx].features();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        Dataset that = (Dataset)o;
+
+        return rowSize == that.rowSize && colSize == that.colSize && Arrays.equals(data, that.data) && Arrays.equals(meta, that.meta);
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        int res = Arrays.hashCode(data);
+        res = 31 * res + Arrays.hashCode(meta);
+        res = 31 * res + rowSize;
+        res = 31 * res + colSize;
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(data);
+        out.writeObject(meta);
+        out.writeInt(rowSize);
+        out.writeInt(colSize);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        data = (Row[]) in.readObject();
+        meta = (FeatureMetadata[]) in.readObject();
+        rowSize = in.readInt();
+        colSize = in.readInt();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/DatasetRow.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/DatasetRow.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/DatasetRow.java
new file mode 100644
index 0000000..3ba0cf7
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/DatasetRow.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+import org.apache.ignite.ml.math.Vector;
+
+/** Class to keep one observation in dataset. This is a base class for labeled and unlabeled rows. */
+public class DatasetRow<V extends Vector> implements Serializable, Externalizable {
+    /** Vector. */
+    protected V vector;
+
+
+    /**
+     * Default constructor (required by Externalizable).
+     */
+    public DatasetRow() {
+    }
+
+    /** */
+    public DatasetRow(V vector) {
+        this.vector = vector;
+    }
+
+    /**
+     * Get the vector.
+     *
+     * @return Vector.
+     */
+    public V features() {
+        return vector;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        DatasetRow vector1 = (DatasetRow)o;
+
+        return vector != null ? !vector.equals(vector1.vector) : vector1.vector != null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return vector != null ? vector.hashCode() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(vector);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        vector = (V)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/FeatureMetadata.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/FeatureMetadata.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/FeatureMetadata.java
new file mode 100644
index 0000000..5d07bdb
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/FeatureMetadata.java
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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;
+
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.Serializable;
+
+/** Class for feature metadata. */
+public class FeatureMetadata implements Serializable, Externalizable {
+    /** Feature name */
+    private String name;
+
+    /**
+     * Default constructor (required by Externalizable).
+     */
+    public FeatureMetadata() {
+    }
+
+    /**
+     * Creates an instance of Feature Metadata class.
+     *
+     * @param name Name.
+     */
+    public FeatureMetadata(String name) {
+        this.name = name;
+    }
+
+    /** */
+    public String name() {
+        return name;
+    }
+
+    /** */
+    public void setName(String name) {
+        this.name = name;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean equals(Object o) {
+        if (this == o)
+            return true;
+        if (o == null || getClass() != o.getClass())
+            return false;
+
+        FeatureMetadata metadata = (FeatureMetadata)o;
+
+        return name != null ? name.equals(metadata.name) : metadata.name == null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int hashCode() {
+        return name != null ? name.hashCode() : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(name);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        name = (String)in.readObject();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDataset.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDataset.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDataset.java
index ee2f442..53f74f3 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDataset.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDataset.java
@@ -17,21 +17,9 @@
 
 package org.apache.ignite.ml.structures;
 
-import java.io.IOException;
-import java.io.Serializable;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.stream.Stream;
-import org.apache.ignite.ml.knn.models.Normalization;
 import org.apache.ignite.ml.math.Vector;
 import org.apache.ignite.ml.math.exceptions.CardinalityException;
 import org.apache.ignite.ml.math.exceptions.NoDataException;
-import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException;
-import org.apache.ignite.ml.math.exceptions.knn.EmptyFileException;
-import org.apache.ignite.ml.math.exceptions.knn.FileParsingException;
 import org.apache.ignite.ml.math.exceptions.knn.NoLabelVectorException;
 import org.apache.ignite.ml.math.impls.vector.DenseLocalOnHeapVector;
 import org.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
@@ -40,50 +28,12 @@ import org.jetbrains.annotations.NotNull;
 /**
  * Class for set of labeled vectors.
  */
-public class LabeledDataset implements Serializable {
-    /** Data to keep. */
-    private final LabeledVector[] data;
-
-    /** Feature names (one name for each attribute in vector). */
-    private String[] featureNames;
-
-    /** Amount of instances. */
-    private int rowSize;
-
-    /** Amount of attributes in each vector. */
-    private int colSize;
-
+public class LabeledDataset<L, Row extends LabeledVector> extends Dataset<Row> {
     /**
-     * Creates new Labeled Dataset by given data.
-     *
-     * @param data Should be initialized with one vector at least.
-     * @param colSize Amount of observed attributes in each vector.
+     * Default constructor (required by Externalizable).
      */
-    public LabeledDataset(LabeledVector[] data, int colSize) {
-        this(data, null, colSize);
-    }
-
-    /**
-     * Creates new Labeled Dataset by given data.
-     *
-     * @param data Given data. Should be initialized with one vector at least.
-     * @param featureNames Column names.
-     * @param colSize Amount of observed attributes in each vector.
-     */
-    public LabeledDataset(LabeledVector[] data, String[] featureNames, int colSize) {
-        assert data != null;
-        assert data.length > 0;
-
-        this.data = data;
-        this.rowSize = data.length;
-        this.colSize = colSize;
-
-        if(featureNames == null) generateFeatureNames();
-        else {
-            assert colSize == featureNames.length;
-            this.featureNames = featureNames;
-        }
-
+    public LabeledDataset() {
+        super();
     }
 
     /**
@@ -116,22 +66,26 @@ public class LabeledDataset implements Serializable {
      * @param isDistributed Use distributed data structures to keep data.
      */
     public LabeledDataset(int rowSize, int colSize, String[] featureNames, boolean isDistributed){
-        assert rowSize > 0;
-        assert colSize > 0;
-
-        if(featureNames == null) generateFeatureNames();
-        else {
-            assert colSize == featureNames.length;
-            this.featureNames = featureNames;
-        }
+        super(rowSize, colSize, featureNames);
 
-        this.rowSize = rowSize;
-        this.colSize = colSize;
+        initializeDataWithLabeledVectors(rowSize, colSize, isDistributed);
+    }
 
-        data = new LabeledVector[rowSize];
+    /** */
+    private void initializeDataWithLabeledVectors(int rowSize, int colSize, boolean isDistributed) {
+        data = (Row[])new LabeledVector[rowSize];
         for (int i = 0; i < rowSize; i++)
-            data[i] = new LabeledVector(getVector(colSize, isDistributed), null);
+            data[i] = (Row)new LabeledVector(emptyVector(colSize, isDistributed), null);
+    }
 
+    /**
+     * Creates new Labeled Dataset by given data.
+     *
+     * @param data Should be initialized with one vector at least.
+     * @param colSize Amount of observed attributes in each vector.
+     */
+    public LabeledDataset(Row[] data, int colSize) {
+        super(data, colSize);
     }
 
 
@@ -154,6 +108,7 @@ public class LabeledDataset implements Serializable {
      * @param isDistributed Use distributed data structures to keep data.
      */
     public LabeledDataset(double[][] mtx, double[] lbs, String[] featureNames, boolean isDistributed) {
+        super();
         assert mtx != null;
         assert lbs != null;
 
@@ -166,14 +121,17 @@ public class LabeledDataset implements Serializable {
         this.rowSize = lbs.length;
         this.colSize = mtx[0].length;
 
-        if(featureNames == null) generateFeatureNames();
-        else this.featureNames = featureNames;
-
+        if(featureNames == null)
+            generateFeatureNames();
+        else {
+            assert colSize == featureNames.length;
+            convertStringNamesToFeatureMetadata(featureNames);
+        }
 
-        data = new LabeledVector[rowSize];
+        data = (Row[])new LabeledVector[rowSize];
         for (int i = 0; i < rowSize; i++){
 
-            data[i] = new LabeledVector(getVector(colSize, isDistributed), lbs[i]);
+            data[i] = (Row)new LabeledVector(emptyVector(colSize, isDistributed), lbs[i]);
             for (int j = 0; j < colSize; j++) {
                 try {
                     data[i].features().set(j, mtx[i][j]);
@@ -184,76 +142,6 @@ public class LabeledDataset implements Serializable {
         }
     }
 
-    /** */
-    private void generateFeatureNames() {
-        featureNames = new String[colSize];
-
-        for (int i = 0; i < colSize; i++)
-            featureNames[i] = "f_" + i;
-    }
-
-
-    /**
-     * Get vectors and their labels.
-     *
-     * @return Array of Label Vector instances.
-     */
-    public LabeledVector[] data() {
-        return data;
-    }
-
-    /**
-     * Gets amount of observation.
-     *
-     * @return Amount of rows in dataset.
-     */
-    public int rowSize(){
-        return rowSize;
-    }
-
-    /**
-     * Returns feature name for column with given index.
-     *
-     * @param i The given index.
-     * @return Feature name.
-     */
-    public String getFeatureName(int i){
-        return featureNames[i];
-    }
-
-    /**
-     * Gets amount of attributes.
-     *
-     * @return Amount of attributes in each Labeled Vector.
-     */
-    public int colSize(){
-        return colSize;
-    }
-
-    /**
-     * Retrieves Labeled Vector by given index.
-     *
-     * @param idx Index of observation.
-     * @return Labeled features.
-     */
-    public LabeledVector getRow(int idx){
-        return data[idx];
-    }
-
-    /**
-     * Get the features.
-     *
-     * @param idx Index of observation.
-     * @return Vector with features.
-     */
-    public Vector features(int idx){
-        assert idx < rowSize;
-        assert data != null;
-        assert data[idx] != null;
-
-        return data[idx].features();
-    }
-
     /**
      * Returns label if label is attached or null if label is missed.
      *
@@ -261,7 +149,7 @@ public class LabeledDataset implements Serializable {
      * @return Label.
      */
     public double label(int idx) {
-        LabeledVector labeledVector = data[idx];
+        LabeledVector labeledVector = (LabeledVector)data[idx];
 
         if(labeledVector!=null)
             return (double)labeledVector.label();
@@ -302,174 +190,10 @@ public class LabeledDataset implements Serializable {
             throw new NoLabelVectorException(idx);
     }
 
-    /**
-     * Datafile should keep class labels in the first column.
-     *
-     * @param pathToFile Path to file.
-     * @param separator Element to tokenize row on separate tokens.
-     * @param isDistributed Generates distributed dataset if true.
-     * @param isFallOnBadData Fall on incorrect data if true.
-     * @return Labeled Dataset parsed from file.
-     */
-    public static LabeledDataset loadTxt(Path pathToFile, String separator, boolean isDistributed, boolean isFallOnBadData) throws IOException {
-        Stream<String> stream = Files.lines(pathToFile);
-        List<String> list = new ArrayList<>();
-        stream.forEach(list::add);
-
-        final int rowSize = list.size();
-
-        List<Double> labels = new ArrayList<>();
-        List<Vector> vectors = new ArrayList<>();
-
-        if (rowSize > 0) {
-
-            final int colSize = getColumnSize(separator, list) - 1;
-
-            if (colSize > 0) {
-
-                for (int i = 0; i < rowSize; i++) {
-                    Double clsLb;
-
-                    String[] rowData = list.get(i).split(separator);
-
-                    try {
-                        clsLb = Double.parseDouble(rowData[0]);
-                        Vector vec = parseFeatures(pathToFile, isDistributed, isFallOnBadData, colSize, i, rowData);
-                        labels.add(clsLb);
-                        vectors.add(vec);
-                    }
-                    catch (NumberFormatException e) {
-                        if(isFallOnBadData)
-                            throw new FileParsingException(rowData[0], i, pathToFile);
-                    }
-                }
-
-                LabeledVector[] data = new LabeledVector[vectors.size()];
-                for (int i = 0; i < vectors.size(); i++)
-                    data[i] = new LabeledVector(vectors.get(i), labels.get(i));
-
-                return new LabeledDataset(data, colSize);
-            }
-            else
-                throw new NoDataException("File should contain first row with data");
-        }
-        else
-            throw new EmptyFileException(pathToFile.toString());
-    }
-
     /** */
-    @NotNull private static Vector parseFeatures(Path pathToFile, boolean isDistributed, boolean isFallOnBadData,
-        int colSize, int rowIdx, String[] rowData) {
-        final Vector vec = getVector(colSize, isDistributed);
-
-        for (int j = 0; j < colSize; j++) {
-
-            if (rowData.length == colSize + 1) {
-                double val = fillMissedData();
-
-                try {
-                    val = Double.parseDouble(rowData[j + 1]);
-                    vec.set(j, val);
-                }
-                catch (NumberFormatException e) {
-                    if(isFallOnBadData)
-                        throw new FileParsingException(rowData[j + 1], rowIdx, pathToFile);
-                    else
-                        vec.set(j,val);
-                }
-            }
-            else throw new CardinalityException(colSize + 1, rowData.length);
-        }
-        return vec;
-    }
-
-    // TODO: IGNITE-7025 add filling with mean, mode, ignoring and so on
-    /** */
-    private static double fillMissedData() {
-            return 0.0;
-    }
-
-    /** */
-    @NotNull private static Vector getVector(int size, boolean isDistributed) {
+    @NotNull public static Vector emptyVector(int size, boolean isDistributed) {
 
         if(isDistributed) return new SparseBlockDistributedVector(size);
         else return new DenseLocalOnHeapVector(size);
     }
-
-    /** */
-    private static int getColumnSize(String separator, List<String> list) {
-        String[] rowData = list.get(0).split(separator, -1); // assume that all observation has the same length as a first row
-
-        return rowData.length;
-    }
-
-    /**
-     * Scales features in dataset.
-     *
-     * @param normalization normalization approach
-     * @return Labeled dataset
-     */
-    public LabeledDataset normalizeWith(Normalization normalization) {
-        switch (normalization){
-            case MINIMAX: minMaxFeatures();
-                break;
-            case Z_NORMALIZATION: throw new UnsupportedOperationException("Z-normalization is not supported yet");
-        }
-
-        return this;
-    }
-
-    /**
-     * Complexity 2*N^2. Try to optimize.
-     */
-    private void minMaxFeatures() {
-        double[] mins = new double[colSize];
-        double[] maxs = new double[colSize];
-
-        for (int j = 0; j < colSize; j++) {
-            double maxInCurrCol = Double.MIN_VALUE;
-            double minInCurrCol = Double.MAX_VALUE;
-
-            for (int i = 0; i < rowSize; i++) {
-                double e = data[i].features().get(j);
-                maxInCurrCol = Math.max(e, maxInCurrCol);
-                minInCurrCol = Math.min(e, minInCurrCol);
-            }
-
-            mins[j] = minInCurrCol;
-            maxs[j] = maxInCurrCol;
-        }
-
-        for (int j = 0; j < colSize; j++) {
-            double div = maxs[j] - mins[j];
-
-            for (int i = 0; i < rowSize; i++) {
-                double oldVal = data[i].features().get(j);
-                double newVal = (oldVal - mins[j])/div;
-                // x'=(x-MIN[X])/(MAX[X]-MIN[X])
-                data[i].features().set(j, newVal);
-            }
-        }
-    }
-
-    /** */
-    @Override public boolean equals(Object o) {
-        if (this == o)
-            return true;
-        if (o == null || getClass() != o.getClass())
-            return false;
-
-        LabeledDataset that = (LabeledDataset)o;
-
-        return rowSize == that.rowSize && colSize == that.colSize && Arrays.equals(data, that.data) && Arrays.equals(featureNames, that.featureNames);
-    }
-
-    /** */
-    @Override public int hashCode() {
-        int res = Arrays.hashCode(data);
-        res = 31 * res + Arrays.hashCode(featureNames);
-        res = 31 * res + rowSize;
-        res = 31 * res + colSize;
-        return res;
-    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDatasetTestTrainPair.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDatasetTestTrainPair.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDatasetTestTrainPair.java
index dd3d244..baf72d8 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDatasetTestTrainPair.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledDatasetTestTrainPair.java
@@ -53,21 +53,19 @@ public class LabeledDatasetTestTrainPair implements Serializable {
 
         final TreeSet<Integer> sortedTestIndices = getSortedIndices(datasetSize, testSize);
 
-
         LabeledVector[] testVectors = new LabeledVector[testSize];
         LabeledVector[] trainVectors = new LabeledVector[trainSize];
 
-
         int datasetCntr = 0;
         int trainCntr = 0;
         int testCntr = 0;
 
         for (Integer idx: sortedTestIndices){ // guarantee order as iterator
-            testVectors[testCntr] = dataset.getRow(idx);
+            testVectors[testCntr] = (LabeledVector)dataset.getRow(idx);
             testCntr++;
 
             for (int i = datasetCntr; i < idx; i++) {
-                trainVectors[trainCntr] = dataset.getRow(i);
+                trainVectors[trainCntr] = (LabeledVector)dataset.getRow(i);
                 trainCntr++;
             }
 
@@ -75,7 +73,7 @@ public class LabeledDatasetTestTrainPair implements Serializable {
         }
         if(datasetCntr < datasetSize){
             for (int i = datasetCntr; i < datasetSize; i++) {
-                trainVectors[trainCntr] = dataset.getRow(i);
+                trainVectors[trainCntr] = (LabeledVector)dataset.getRow(i);
                 trainCntr++;
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledVector.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledVector.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledVector.java
index a4e218b..9f0a881 100644
--- a/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledVector.java
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/LabeledVector.java
@@ -17,40 +17,37 @@
 
 package org.apache.ignite.ml.structures;
 
-import java.io.Serializable;
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
 import org.apache.ignite.ml.math.Vector;
 
 /**
  * Class for vector with label.
  *
  * @param <V> Some class extending {@link Vector}.
- * @param <T> Type of label.
+ * @param <L> Type of label.
  */
-public class LabeledVector<V extends Vector, T> implements Serializable {
-    /** Vector. */
-    private final V vector;
-
+public class LabeledVector<V extends Vector, L> extends DatasetRow<V> {
     /** Label. */
-    private T lb;
+    private L lb;
 
     /**
-     * Construct labeled vector.
-     *
-     * @param vector Vector.
-     * @param lb Label.
+     * Default constructor.
      */
-    public LabeledVector(V vector, T lb) {
-        this.vector = vector;
-        this.lb = lb;
+    public LabeledVector() {
+        super();
     }
 
     /**
-     * Get the vector.
+     * Construct labeled vector.
      *
-     * @return Vector.
+     * @param vector Vector.
+     * @param lb Label.
      */
-    public V features() {
-        return vector;
+    public LabeledVector(V vector, L lb) {
+        super(vector);
+        this.lb = lb;
     }
 
     /**
@@ -58,7 +55,7 @@ public class LabeledVector<V extends Vector, T> implements Serializable {
      *
      * @return Label.
      */
-    public T label() {
+    public L label() {
         return lb;
     }
 
@@ -67,11 +64,11 @@ public class LabeledVector<V extends Vector, T> implements Serializable {
      *
      * @param lb Label.
      */
-    public void setLabel(T lb) {
+    public void setLabel(L lb) {
         this.lb = lb;
     }
 
-    /** */
+    /** {@inheritDoc} */
     @Override public boolean equals(Object o) {
         if (this == o)
             return true;
@@ -85,10 +82,22 @@ public class LabeledVector<V extends Vector, T> implements Serializable {
         return lb != null ? lb.equals(vector1.lb) : vector1.lb == null;
     }
 
-    /** */
+    /** {@inheritDoc} */
     @Override public int hashCode() {
         int res = vector != null ? vector.hashCode() : 0;
         res = 31 * res + (lb != null ? lb.hashCode() : 0);
         return res;
     }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        out.writeObject(vector);
+        out.writeObject(lb);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        vector = (V)in.readObject();
+        lb = (L)in.readObject();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabeledDatasetLoader.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabeledDatasetLoader.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabeledDatasetLoader.java
new file mode 100644
index 0000000..0faa416
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabeledDatasetLoader.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.ml.structures.preprocessing;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Stream;
+import org.apache.ignite.ml.math.Vector;
+import org.apache.ignite.ml.math.exceptions.CardinalityException;
+import org.apache.ignite.ml.math.exceptions.NoDataException;
+import org.apache.ignite.ml.math.exceptions.knn.EmptyFileException;
+import org.apache.ignite.ml.math.exceptions.knn.FileParsingException;
+import org.apache.ignite.ml.structures.LabeledDataset;
+import org.apache.ignite.ml.structures.LabeledVector;
+import org.jetbrains.annotations.NotNull;
+
+/** Data pre-processing step which loads data from different file types. */
+public class LabeledDatasetLoader {
+    /**
+     * Datafile should keep class labels in the first column.
+     *
+     * @param pathToFile Path to file.
+     * @param separator Element to tokenize row on separate tokens.
+     * @param isDistributed Generates distributed dataset if true.
+     * @param isFallOnBadData Fall on incorrect data if true.
+     * @return Labeled Dataset parsed from file.
+     */
+    public static LabeledDataset loadFromTxtFile(Path pathToFile, String separator, boolean isDistributed,
+        boolean isFallOnBadData) throws IOException {
+        Stream<String> stream = Files.lines(pathToFile);
+        List<String> list = new ArrayList<>();
+        stream.forEach(list::add);
+
+        final int rowSize = list.size();
+
+        List<Double> labels = new ArrayList<>();
+        List<Vector> vectors = new ArrayList<>();
+
+        if (rowSize > 0) {
+
+            final int colSize = getColumnSize(separator, list) - 1;
+
+            if (colSize > 0) {
+
+                for (int i = 0; i < rowSize; i++) {
+                    Double clsLb;
+
+                    String[] rowData = list.get(i).split(separator);
+
+                    try {
+                        clsLb = Double.parseDouble(rowData[0]);
+                        Vector vec = parseFeatures(pathToFile, isDistributed, isFallOnBadData, colSize, i, rowData);
+                        labels.add(clsLb);
+                        vectors.add(vec);
+                    }
+                    catch (NumberFormatException e) {
+                        if (isFallOnBadData)
+                            throw new FileParsingException(rowData[0], i, pathToFile);
+                    }
+                }
+
+                LabeledVector[] data = new LabeledVector[vectors.size()];
+                for (int i = 0; i < vectors.size(); i++)
+                    data[i] = new LabeledVector(vectors.get(i), labels.get(i));
+
+                return new LabeledDataset(data, colSize);
+            }
+            else
+                throw new NoDataException("File should contain first row with data");
+        }
+        else
+            throw new EmptyFileException(pathToFile.toString());
+    }
+
+    /** */
+    @NotNull private static Vector parseFeatures(Path pathToFile, boolean isDistributed, boolean isFallOnBadData,
+        int colSize, int rowIdx, String[] rowData) {
+        final Vector vec = LabeledDataset.emptyVector(colSize, isDistributed);
+
+        for (int j = 0; j < colSize; j++) {
+
+            if (rowData.length == colSize + 1) {
+                double val = fillMissedData();
+
+                try {
+                    val = Double.parseDouble(rowData[j + 1]);
+                    vec.set(j, val);
+                }
+                catch (NumberFormatException e) {
+                    if (isFallOnBadData)
+                        throw new FileParsingException(rowData[j + 1], rowIdx, pathToFile);
+                    else
+                        vec.set(j, val);
+                }
+            }
+            else
+                throw new CardinalityException(colSize + 1, rowData.length);
+        }
+        return vec;
+    }
+
+    // TODO: IGNITE-7025 add filling with mean, mode, ignoring and so on
+
+    /** */
+    private static double fillMissedData() {
+        return 0.0;
+    }
+
+    /** */
+    private static int getColumnSize(String separator, List<String> list) {
+        String[] rowData = list.get(0).split(separator, -1); // assume that all observation has the same length as a first row
+
+        return rowData.length;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java
new file mode 100644
index 0000000..44719cf
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/LabellingMachine.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.ml.structures.preprocessing;
+
+import org.apache.ignite.ml.knn.models.KNNModel;
+import org.apache.ignite.ml.structures.LabeledDataset;
+
+/** Data pre-processing step which assigns labels to all observations according model. */
+public class LabellingMachine {
+    /**
+     * Set labels to each observation according passed Model.
+     * <p>
+     * NOTE: In-place operation.
+     * </p>
+     * @param ds The given labeled dataset.
+     * @param knnMdl The given kNN Model.
+     * @return Dataset with predicted labels.
+     */
+    public static LabeledDataset assignLabels(LabeledDataset ds, KNNModel knnMdl) {
+        for (int i = 0; i < ds.rowSize(); i++) {
+            double predictedCls = knnMdl.apply(ds.getRow(i).features());
+            ds.setLabel(i, predictedCls);
+        }
+        return ds;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java
new file mode 100644
index 0000000..26d8bf9
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/Normalizer.java
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT 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.preprocessing;
+
+import org.apache.ignite.ml.math.exceptions.UnsupportedOperationException;
+import org.apache.ignite.ml.structures.Dataset;
+import org.apache.ignite.ml.structures.DatasetRow;
+
+/** Data pre-processing step which scales features according normalization algorithms. */
+public class Normalizer {
+    /**
+     * Scales features in dataset with MiniMax algorithm x'=(x-MIN[X])/(MAX[X]-MIN[X]). This is an in-place operation.
+     * <p>
+     * NOTE: Complexity 2*N^2.
+     * </p>
+     * @param ds The given dataset.
+     * @return Transformed dataset.
+     */
+    public static Dataset normalizeWithMiniMax(Dataset ds) {
+        int colSize = ds.colSize();
+        double[] mins = new double[colSize];
+        double[] maxs = new double[colSize];
+
+        int rowSize = ds.rowSize();
+        DatasetRow[] data = ds.data();
+        for (int j = 0; j < colSize; j++) {
+            double maxInCurrCol = Double.MIN_VALUE;
+            double minInCurrCol = Double.MAX_VALUE;
+
+            for (int i = 0; i < rowSize; i++) {
+                double e = data[i].features().get(j);
+                maxInCurrCol = Math.max(e, maxInCurrCol);
+                minInCurrCol = Math.min(e, minInCurrCol);
+            }
+
+            mins[j] = minInCurrCol;
+            maxs[j] = maxInCurrCol;
+        }
+
+        for (int j = 0; j < colSize; j++) {
+            double div = maxs[j] - mins[j];
+
+            for (int i = 0; i < rowSize; i++) {
+                double oldVal = data[i].features().get(j);
+                double newVal = (oldVal - mins[j]) / div;
+                // x'=(x-MIN[X])/(MAX[X]-MIN[X])
+                data[i].features().set(j, newVal);
+            }
+        }
+
+        return ds;
+    }
+
+    /**
+     * Scales features in dataset with Z-Normalization algorithm x'=(x-M[X])/\sigma [X]. This is an in-place operation.
+     *
+     * @param ds The given dataset.
+     * @return Transformed dataset.
+     */
+    public static Dataset normalizeWithZNormalization(Dataset ds) {
+        throw new UnsupportedOperationException("Z-normalization is not supported yet");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/package-info.java
new file mode 100644
index 0000000..c243074
--- /dev/null
+++ b/modules/ml/src/main/java/org/apache/ignite/ml/structures/preprocessing/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 main APIs for dataset pre-processing.
+ */
+package org.apache.ignite.ml.structures.preprocessing;

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/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
index 9075978..1651588 100644
--- 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
@@ -23,6 +23,7 @@ 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;
 
 /**
@@ -76,7 +77,7 @@ public class BaseKNNTest extends GridCommonAbstractTest {
         try {
             Path path = Paths.get(this.getClass().getClassLoader().getResource(rsrcPath).toURI());
             try {
-                return LabeledDataset.loadTxt(path, SEPARATOR, false, isFallOnBadData);
+                return LabeledDatasetLoader.loadFromTxtFile(path, SEPARATOR, false, isFallOnBadData);
             }
             catch (IOException e) {
                 e.printStackTrace();

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNMultipleLinearRegressionTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNMultipleLinearRegressionTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNMultipleLinearRegressionTest.java
index d973686..e5d9b13 100644
--- a/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNMultipleLinearRegressionTest.java
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/knn/KNNMultipleLinearRegressionTest.java
@@ -19,13 +19,13 @@ package org.apache.ignite.ml.knn;
 
 import org.apache.ignite.internal.util.IgniteUtils;
 import org.apache.ignite.ml.knn.models.KNNStrategy;
-import org.apache.ignite.ml.knn.models.Normalization;
 import org.apache.ignite.ml.knn.regression.KNNMultipleLinearRegression;
 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.apache.ignite.ml.math.impls.vector.SparseBlockDistributedVector;
 import org.apache.ignite.ml.structures.LabeledDataset;
+import org.apache.ignite.ml.structures.preprocessing.Normalizer;
 import org.junit.Assert;
 
 /**
@@ -115,7 +115,7 @@ public class KNNMultipleLinearRegressionTest extends BaseKNNTest {
 
         LabeledDataset training = new LabeledDataset(x, y);
 
-        final LabeledDataset normalizedTrainingDataset = training.normalizeWith(Normalization.MINIMAX);
+        final LabeledDataset normalizedTrainingDataset = (LabeledDataset)Normalizer.normalizeWithMiniMax(training);
 
         KNNMultipleLinearRegression knnMdl = new KNNMultipleLinearRegression(5, new EuclideanDistance(), KNNStrategy.SIMPLE, normalizedTrainingDataset);
         Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});
@@ -147,7 +147,7 @@ public class KNNMultipleLinearRegressionTest extends BaseKNNTest {
 
         LabeledDataset training = new LabeledDataset(x, y);
 
-        final LabeledDataset normalizedTrainingDataset = training.normalizeWith(Normalization.MINIMAX);
+        final LabeledDataset normalizedTrainingDataset = (LabeledDataset)Normalizer.normalizeWithMiniMax(training);
 
         KNNMultipleLinearRegression knnMdl = new KNNMultipleLinearRegression(5, new EuclideanDistance(), KNNStrategy.WEIGHTED, normalizedTrainingDataset);
         Vector vector = new DenseLocalOnHeapVector(new double[] {104.6, 419180, 2822, 2857, 118734, 1956});

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/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 c64a8d8..c4ae70f 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
@@ -22,6 +22,7 @@ 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;
 import org.apache.ignite.ml.math.exceptions.NoDataException;
@@ -30,9 +31,10 @@ import org.apache.ignite.ml.math.exceptions.knn.FileParsingException;
 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;
 
 /** Tests behaviour of KNNClassificationTest. */
-public class LabeledDatasetTest extends BaseKNNTest {
+public class LabeledDatasetTest extends BaseKNNTest implements ExternalizableTest<LabeledDataset> {
     /** */
     private static final String KNN_IRIS_TXT = "datasets/knn/iris.txt";
 
@@ -88,7 +90,7 @@ public class LabeledDatasetTest extends BaseKNNTest {
         assertEquals(dataset.colSize(), 2);
         assertEquals(dataset.rowSize(), 6);
 
-        final LabeledVector<Vector, Double> row = dataset.getRow(0);
+        final LabeledVector<Vector, Double> row = (LabeledVector<Vector, Double>)dataset.getRow(0);
 
         assertEquals(row.features().get(0), 1.0);
         assertEquals(row.label(), 1.0);
@@ -202,7 +204,7 @@ public class LabeledDatasetTest extends BaseKNNTest {
 
         Path path = Paths.get(this.getClass().getClassLoader().getResource(IRIS_MISSED_DATA).toURI());
 
-        LabeledDataset training = LabeledDataset.loadTxt(path, ",", false, false);
+        LabeledDataset training = LabeledDatasetLoader.loadFromTxtFile(path, ",", false, false);
 
         assertEquals(training.features(2).get(1), 0.0);
     }
@@ -263,4 +265,21 @@ public class LabeledDatasetTest extends BaseKNNTest {
         for (int i = 0; i < lbs.length; i++)
             assertEquals(lbs[i], labels[i]);
     }
+
+    @Override public void testExternalization() {
+        IgniteUtils.setCurrentIgniteName(ignite.configuration().getIgniteInstanceName());
+
+        double[][] mtx =
+            new double[][] {
+                {1.0, 1.0},
+                {1.0, 2.0},
+                {2.0, 1.0},
+                {-1.0, -1.0},
+                {-1.0, -2.0},
+                {-2.0, -1.0}};
+        double[] lbs = new double[] {1.0, 1.0, 1.0, 2.0, 2.0, 2.0};
+
+        LabeledDataset dataset = new LabeledDataset(mtx, lbs);
+        this.externalizeTest(dataset);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/429f9544/modules/ml/src/test/java/org/apache/ignite/ml/math/ExternalizableTest.java
----------------------------------------------------------------------
diff --git a/modules/ml/src/test/java/org/apache/ignite/ml/math/ExternalizableTest.java b/modules/ml/src/test/java/org/apache/ignite/ml/math/ExternalizableTest.java
new file mode 100644
index 0000000..e4080ff
--- /dev/null
+++ b/modules/ml/src/test/java/org/apache/ignite/ml/math/ExternalizableTest.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.ml.math;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.Externalizable;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import org.apache.ignite.ml.math.impls.MathTestConstants;
+import org.junit.Test;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Common test for externalization.
+ */
+public interface ExternalizableTest<T extends Externalizable> {
+    /** */
+    @SuppressWarnings("unchecked")
+    public default void externalizeTest(T initObj) {
+        T objRestored = null;
+
+        try {
+            ByteArrayOutputStream byteArrOutputStream = new ByteArrayOutputStream();
+            ObjectOutputStream objOutputStream = new ObjectOutputStream(byteArrOutputStream);
+
+            objOutputStream.writeObject(initObj);
+
+            ByteArrayInputStream byteArrInputStream = new ByteArrayInputStream(byteArrOutputStream.toByteArray());
+            ObjectInputStream objInputStream = new ObjectInputStream(byteArrInputStream);
+
+            objRestored = (T)objInputStream.readObject();
+
+            assertTrue(MathTestConstants.VAL_NOT_EQUALS, initObj.equals(objRestored));
+            assertTrue(MathTestConstants.VAL_NOT_EQUALS, Integer.compare(initObj.hashCode(), objRestored.hashCode()) == 0);
+        }
+        catch (ClassNotFoundException | IOException e) {
+            fail(e + " [" + e.getMessage() + "]");
+        }
+        finally {
+            if (objRestored != null && objRestored instanceof Destroyable)
+                ((Destroyable)objRestored).destroy();
+        }
+    }
+
+    /** */
+    @Test
+    public void testExternalization();
+}


[02/15] ignite git commit: ignite-7107 Apache Ignite RPM packages

Posted by sb...@apache.org.
ignite-7107 Apache Ignite RPM packages

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/33ad0ad8
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/33ad0ad8
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/33ad0ad8

Branch: refs/heads/ignite-zk
Commit: 33ad0ad84a25a0d65ba408accfb1429cb96ff840
Parents: 3c5d376
Author: Ivanov Petr <pi...@gridgain.com>
Authored: Thu Dec 28 16:08:21 2017 +0300
Committer: Andrey Gura <ag...@apache.org>
Committed: Thu Dec 28 16:08:21 2017 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                           | 198 +++++++++++++++++---------
 packaging/rpm/SOURCES/name.service     |  17 +++
 packaging/rpm/SOURCES/service.sh       |  28 ++++
 packaging/rpm/SPECS/apache-ignite.spec | 206 ++++++++++++++++++++++++++++
 4 files changed, 381 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/33ad0ad8/DEVNOTES.txt
----------------------------------------------------------------------
diff --git a/DEVNOTES.txt b/DEVNOTES.txt
index 340153e..6ec0f39 100644
--- a/DEVNOTES.txt
+++ b/DEVNOTES.txt
@@ -4,81 +4,132 @@ Ignite Fabric Maven Build Instructions
 
 2) Compile and install:
 
-  mvn clean install -Pall-java,all-scala,licenses -DskipTests
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests
 
-  or if you have built Apache Ignite.NET on the first step use following command:
-  (Note that 'doxygen' should be installed before running this command.)
+   or if you have built Apache Ignite.NET on the first step use following command:
+   (Note that 'doxygen' should be installed before running this command.)
 
-  mvn clean install -Pall-java,all-scala,licenses -DskipTests -DclientDocs
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests -DclientDocs
 
 3) Javadoc generation (optional):
 
-  mvn initialize -Pjavadoc
+        mvn initialize -Pjavadoc
 
 4) Assembly Apache Ignite fabric:
 
-  mvn initialize -Prelease
+        mvn initialize -Prelease
 
 Look for apache-ignite-fabric-<version>-bin.zip in ./target/bin directory.
 
+
 Ignite Fabric with LGPL Maven Build Instructions
-======================================
+================================================
 1) Optional: build Apache Ignite.NET as described at modules/platforms/dotnet/DEVNOTES.txt.
 
 2) Compile and install:
 
-  mvn clean install -Pall-java,all-scala,licenses -DskipTests
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests
 
-  or if you have built Apache Ignite.NET on the first step use following command:
-  (Note that 'doxygen' should be installed before running this command.)
+   or if you have built Apache Ignite.NET on the first step use following command:
+   (Note that 'doxygen' should be installed before running this command.)
 
-  mvn clean install -Pall-java,all-scala,licenses -DskipTests -DclientDocs
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests -DclientDocs
 
 3) Javadoc generation with LGPL (optional):
 
-  mvn initialize -Pjavadoc,lgpl
+        mvn initialize -Pjavadoc,lgpl
 
 4) Assembly Apache Ignite fabric with LGPL dependencies:
 
-  mvn initialize -Prelease,lgpl -Dignite.edition=fabric-lgpl
+        mvn initialize -Prelease,lgpl -Dignite.edition=fabric-lgpl
+
+   Look for apache-ignite-fabric-lgpl-<version>-bin.zip in ./target/bin directory.
 
-Look for apache-ignite-fabric-lgpl-<version>-bin.zip in ./target/bin directory.
 
 Ignite Hadoop Accelerator Maven Build Instructions
-============================================
+==================================================
 1) Compile and install:
 
-    mvn clean install -Pall-java,all-scala,licenses -DskipTests
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests
 
-Use 'hadoop.version' parameter to build Ignite against a specific Hadoop version.
-Use 'spark.version' parameter to build ignite-spark module for a specific Spark version. Version should be >= 2.0.0.
-For example:
+   Use 'hadoop.version' parameter to build Ignite against a specific Hadoop version.
+   Use 'spark.version' parameter to build ignite-spark module for a specific Spark version. Version should be >= 2.0.0.
+   For example:
 
-    mvn clean install -Pall-java,all-scala,licenses -DskipTests -Dhadoop.version=2.4.2 -Dspark.version=2.1.1
+        mvn clean install -Pall-java,all-scala,licenses -DskipTests -Dhadoop.version=2.4.2 -Dspark.version=2.1.1
 
 2) Assembly Hadoop Accelerator:
-  mvn initialize -Prelease -Dignite.edition=hadoop
 
-Look for apache-ignite-hadoop-<version>-bin.zip in ./target/bin directory. Resulting binary
-assembly will also include integration module for Apache Spark.
+        mvn initialize -Prelease -Dignite.edition=hadoop
+
+   Look for apache-ignite-hadoop-<version>-bin.zip in ./target/bin directory. Resulting binary
+   assembly will also include integration module for Apache Spark.
+
+   NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 
-NOTE: JDK version should be 1.7.0-* or >= 1.8.0-u40.
 
 Ignite C++ Build Instructions
+=============================
+Instructions can be found at modules/platforms/cpp/DEVNOTES.txt.
+
+
+Apache Ignite RPM Package Build Instructions
+============================================
+1) Install dependencies
+
+        yum install -y rpmdevtools rpm-build
+
+2) Create directory layout
+
+        rpmdev-setuptree
+
+3) Copy build specification and additional sources required for build
+
+        cp -rfv packaging/rpm/* ~/rpmbuild
+
+4) Copy built binary from target/bin to sources directory
+
+        cp -rfv target/bin/apache-ignite-<version>-bin.zip ~/rpmbuild/SOURCES/apache-ignite.zip
+
+5) Run build process
+
+        rpmbuild -bb ~/rpmbuild/SPECS/apache-ignite.spec
+
+   NOTE: to build also SRPM package, run with -ba flag instead
+
+   Package will be available in ~/rpmbuild/RPMS/noarch
+
+
+Install and Run Apache Ignite from RPM
 ======================================
-  Instructions can be found at modules/platforms/cpp/DEVNOTES.txt.
+1) Install package
+
+        yum localinstall -y apache-ignite-<version>.rpm
+
+2) Start Apache Ignite service
+
+        systemctl start apache-ignite@<config name>
+
+   NOTE: <config name> - relative to /etc/apache-ignite configuration file name
+
+   Example:
+
+        systemctl start apache-ignite@default-config.xml
+
+3) See logs in /var/log/apache-ignite or journalctl -xe
+
 
 Ignite Release Instructions
 ===========================
-Use your people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
-Download https://dist.apache.org/repos/dist/release/ignite/KEYS and append you key using commands:
+1) Use your people.apache.org/keys/committer/<username>.asc key to generate KEYS file.
+   Download https://dist.apache.org/repos/dist/release/ignite/KEYS and append you key using commands:
 
-gpg --list-sigs <keyname> >> KEYS
-gpg --armor --export <keyname> >> KEYS
+        gpg --list-sigs <keyname> >> KEYS
+        gpg --armor --export <keyname> >> KEYS
 
-Upload modified KEYS file.
+   Upload modified KEYS file.
 
-Specify gpg profile at settings.xml. It will be used to sign sources and artifacts.
+2) Specify gpg profile at settings.xml. It will be used to sign sources and artifacts.
 
 <profile>
    <id>gpg</id>
@@ -88,88 +139,99 @@ Specify gpg profile at settings.xml. It will be used to sign sources and artifac
    </properties>
 </profile>
 
-Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
-Maven release plugin release:prepare goal can be used to make release tag.
+   Ensure you have RELEASE (not SNAPSHOT) version at Ignite poms.
+   Maven release plugin release:prepare goal can be used to make release tag.
+
+3) Deploy Ignite release candidate to maven repository and dev-svn, make tag:
+
+   3.1) Deploy Ignite to maven repository, prepares sources and fabric edition binaries.
 
-Deploy Ignite release candidate to maven repository and dev-svn, make tag:
+                mvn deploy -Papache-release,gpg,all-java,all-scala,licenses,deploy-ignite-site -Dignite.edition=fabric -DskipTests
 
-   1) Deploy Ignite to maven repository, prepares sources and fabric edition binaries.
-      mvn deploy -Papache-release,gpg,all-java,all-scala,licenses,deploy-ignite-site -Dignite.edition=fabric -DskipTests
+   3.2) Javadoc generation:
 
-   2) Javadoc generation:
-      mvn initialize -Pjavadoc
+                mvn initialize -Pjavadoc
 
-   3) Assembly Apache Ignite Fabric:
-      mvn initialize -Prelease
+   3.3) Assembly Apache Ignite Fabric:
 
-   4) Assembly Hadoop Accelerator:
-      mvn initialize -Prelease -Dignite.edition=hadoop
+                mvn initialize -Prelease
 
-   Binary artifact name can be changed by setting additional property -Dignite.zip.pattern. Binary artifact will be
+   3.4) Assembly Hadoop Accelerator:
+
+                mvn initialize -Prelease -Dignite.edition=hadoop
+
+   NOTE: Binary artifact name can be changed by setting additional property -Dignite.zip.pattern. Binary artifact will be
    created inside /target/bin folder when release profile is used.
 
-   Sources artifact name is fixed. Sources artifact will be created inside /target dir when apache-release profile is used.
+   NOTE: Sources artifact name is fixed. Sources artifact will be created inside /target dir when apache-release profile is used.
 
-   Nexus staging (repository.apache.org) should be closed with appropriate comment contains release version and
+   NOTE: Nexus staging (repository.apache.org) should be closed with appropriate comment contains release version and
    release candidate number, for example "Apache Ignite 1.0.0-rc7", when mvn deploy finished.
 
-   Checkout https://dist.apache.org/repos/dist/dev/ignite svn. Create release candidate folder with name
-   equals to release version with "-rc*" ending, for example "1.0.0-rc7", at svn root.
-   Copy /target/site folder content to svn/ignite/<rc-version> folder and commit with appropriate comment.
+   3.5) Checkout https://dist.apache.org/repos/dist/dev/ignite svn. Create release candidate folder with name
+        equals to release version with "-rc*" ending, for example "1.0.0-rc7", at svn root.
+        Copy /target/site folder content to svn/ignite/<rc-version> folder and commit with appropriate comment.
 
-   Make appropriate git tag for release candidate, for example "ignite-X.Y.Z-rc1".
+   3.6) Make appropriate git tag for release candidate, for example "ignite-X.Y.Z-rc1".
 
-Start vote based on https://dist.apache.org/repos/dist/dev/ignite/<rc-version>.
+4) Start vote based on https://dist.apache.org/repos/dist/dev/ignite/<rc-version>.
 
-Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/ignite/<rc-version>
-to https://dist.apache.org/repos/dist/release/ignite/<version> when version accepted.
-Use svn mv ^/dev/ignite/<rc-version> ^/release/ignite/<version> command for proper moving.
+5) Release nexus staging, move binaries and sources from https://dist.apache.org/repos/dist/dev/ignite/<rc-version>
+   to https://dist.apache.org/repos/dist/release/ignite/<version> when version accepted.
+   Use svn mv ^/dev/ignite/<rc-version> ^/release/ignite/<version> command for proper moving.
 
-Make appropriate git tag for released version, for example "ignite-X.Y.Z".
+6) Make appropriate git tag for released version, for example "ignite-X.Y.Z".
 
-Send an email to dev@ignite.apache.org contains release svn url.
+7) Send an email to dev@ignite.apache.org contains release svn url.
 
 
 JCache TCK compliance
 ======================
 To test compliance with JCache TCK use:
 
-mvn test -P-release,jcache-tck -pl :ignite-core -am
+        mvn test -P-release,jcache-tck -pl :ignite-core -am
+
 
 Ignite Mesos Maven Build Instructions
-============================================
-cd to ./modules/mesos
+=====================================
 
-mvn clean package
+        cd ./modules/mesos
+        mvn clean package
 
 Look for ignite-mesos-<version>.jar in ./target directory.
 
+
 Ignite Yarn Maven Build Instructions
-============================================
-cd to ./modules/yarn
+====================================
 
-mvn clean package
+        cd ./modules/yarn
+        mvn clean package
 
 Look for ignite-yarn-<version>.jar in ./target directory.
 
+
 Run tests
-==========
+=========
 To run tests locally use:
 
-mvn clean test -U -Plgpl,examples,-clean-libs,-release -Dmaven.test.failure.ignore=true -DfailIfNoTests=false -Dtest=%TEST_PATTERN%
+        mvn clean test -U -Plgpl,examples,-clean-libs,-release -Dmaven.test.failure.ignore=true -DfailIfNoTests=false -Dtest=%TEST_PATTERN%
 
 For example, %TEST_PATTERN% can be 'org.apache.ignite.testsuites.IgniteBasicTestSuite' or 'GridCacheLocalAtomicFullApiSelfTest#testGet'
 
+
 Apache RAT Instructions
 =======================
 To check license headers use:
 
-mvn clean validate -Pcheck-licenses
+        mvn clean validate -Pcheck-licenses
 
 For more information (e.g. exclude list) see "check-licenses" profile in "parent/pom.xml".
 
 
 Useful Release Commands
-========================
-ZIP: zip -r9 apache-ignite-X.X.X-src.zip apache-ignite-X.X.X-src
-GPG: gpg --armor --output apache-ignite-X.X.X-src.zip.asc --detach-sig apache-ignite-X.X.X-src.zip
+=======================
+ZIP:
+        zip -r9 apache-ignite-X.X.X-src.zip apache-ignite-X.X.X-src
+GPG:
+        gpg --armor --output apache-ignite-X.X.X-src.zip.asc --detach-sig apache-ignite-X.X.X-src.zip
+

http://git-wip-us.apache.org/repos/asf/ignite/blob/33ad0ad8/packaging/rpm/SOURCES/name.service
----------------------------------------------------------------------
diff --git a/packaging/rpm/SOURCES/name.service b/packaging/rpm/SOURCES/name.service
new file mode 100644
index 0000000..d03a7fc
--- /dev/null
+++ b/packaging/rpm/SOURCES/name.service
@@ -0,0 +1,17 @@
+[Unit]
+Description=Apache Ignite In-Memory Computing Platform Service
+After=syslog.target network.target
+
+[Service]
+Type=forking
+User=#user#
+WorkingDirectory=/usr/share/#name#/work
+PermissionsStartOnly=true
+ExecStartPre=-/usr/bin/mkdir /var/run/#name#
+ExecStartPre=-/usr/bin/chown #user#:#user# /var/run/#name#
+ExecStartPre=-/usr/bin/env bash /usr/share/#name#/bin/service.sh set-firewall
+ExecStart=/usr/share/#name#/bin/service.sh start %i
+PIDFile=/var/run/#name#/#name#.%i.pid
+
+[Install]
+WantedBy=multi-user.target

http://git-wip-us.apache.org/repos/asf/ignite/blob/33ad0ad8/packaging/rpm/SOURCES/service.sh
----------------------------------------------------------------------
diff --git a/packaging/rpm/SOURCES/service.sh b/packaging/rpm/SOURCES/service.sh
new file mode 100644
index 0000000..4163676
--- /dev/null
+++ b/packaging/rpm/SOURCES/service.sh
@@ -0,0 +1,28 @@
+#!/usr/bin/env bash
+
+firewallCmd="firewall-cmd --permanent --direct --add-rule ipv4 filter INPUT 0"
+
+# Define function to check whether firewalld is present and started and apply firewall rules for grid nodes
+setFirewall ()
+{
+	if [[ "$(type firewall-cmd &>/dev/null; echo $?)" -eq 0 && "$(systemctl is-active firewalld)" == "active" ]]
+	then
+	    for port in s d
+	    do
+	        ${firewallCmd} -p tcp -m multiport --${port}ports 11211:11220,47500:47509,47100:47109 -j ACCEPT &>/dev/null
+	        ${firewallCmd} -p udp -m multiport --${port}ports 47400:47409 -j ACCEPT &>/dev/null
+	    done
+	    ${firewallCmd} -m pkttype --pkt-type multicast -j ACCEPT &>/dev/null
+
+	    systemctl restart firewalld
+	fi
+}
+
+case $1 in
+	start)
+		/usr/share/#name#/bin/ignite.sh /etc/#name#/$2 & echo $! >> /var/run/#name#/#name#.$2.pid
+		;;
+	set-firewall)
+		setFirewall
+		;;
+esac

http://git-wip-us.apache.org/repos/asf/ignite/blob/33ad0ad8/packaging/rpm/SPECS/apache-ignite.spec
----------------------------------------------------------------------
diff --git a/packaging/rpm/SPECS/apache-ignite.spec b/packaging/rpm/SPECS/apache-ignite.spec
new file mode 100644
index 0000000..45d72b7
--- /dev/null
+++ b/packaging/rpm/SPECS/apache-ignite.spec
@@ -0,0 +1,206 @@
+%define __jar_repack %{nil}
+%define user ignite
+
+
+#-------------------------------------------------------------------------------
+#
+# Packages' descriptions
+#
+
+Name:             apache-ignite
+Version:          2.4.0
+Release:          1%{?dist}
+Summary:          Apache Ignite In-Memory Computing Platform
+Group:            Development/System
+License:          ASL 2.0
+URL:              https://ignite.apache.org/
+Source:           %{name}.zip
+Requires:         java-1.8.0, chkconfig
+Requires(pre):    shadow-utils
+Provides:         %{name}
+AutoReq:          no
+AutoProv:         no
+BuildArch:        noarch
+%description
+Apache Ignite™ is the in-memory computing platform composed of a strongly
+consistent distributed database with powerful SQL, key-value and processing APIs
+
+
+#-------------------------------------------------------------------------------
+#
+# Prepare step: unpack sources
+#
+
+%prep
+%setup -q -c -n %{name}
+
+
+#-------------------------------------------------------------------------------
+#
+# Preinstall scripts
+# $1 can be:
+#     1 - Initial install 
+#     2 - Upgrade
+#
+
+
+#-------------------------------------------------------------------------------
+#
+# Postinstall scripts
+# $1 can be:
+#     1 - Initial installation
+#     2 - Upgrade
+#
+
+%post
+case $1 in
+    1)
+        # Add user for service operation
+        useradd -r -d %{_datadir}/%{name} -s /usr/sbin/nologin %{user}
+        # Change ownership for work and log directories
+        chown -vR %{user}:%{user} %{_sharedstatedir}/%{name} %{_var}/log/%{name}
+        # Install alternatives
+        # Commented out until ignitevisorcmd / ignitesqlline is ready to work from any user
+        #update-alternatives --install %{_bindir}/ignitevisorcmd ignitevisorcmd %{_datadir}/%{name}/bin/ignitevisorcmd.sh 0
+        #update-alternatives --auto ignitevisorcmd
+        #update-alternatives --display ignitevisorcmd
+        #update-alternatives --install %{_bindir}/ignitesqlline ignitesqlline %{_datadir}/%{name}/bin/sqlline.sh 0
+        #update-alternatives --auto ignitesqlline
+        #update-alternatives --display ignitesqlline
+        ;;
+    2)
+        :
+        ;;
+esac
+
+
+#-------------------------------------------------------------------------------
+#
+# Pre-uninstall scripts
+# $1 can be:
+#     0 - Uninstallation
+#     1 - Upgrade
+#
+
+%preun
+case $1 in
+    0)
+        # Remove alternatives
+        # Commented out until ignitevisorcmd / ignitesqlline is ready to work from any user
+        #update-alternatives --remove ignitevisorcmd /usr/share/%{name}/bin/ignitevisorcmd.sh
+        #update-alternatives --display ignitevisorcmd || true
+        #update-alternatives --remove ignitesqlline /usr/share/%{name}/bin/sqlline.sh
+        #update-alternatives --display ignitesqlline || true
+        ;;
+    1)
+        :
+        ;;
+esac
+
+
+#-------------------------------------------------------------------------------
+#
+# Post-uninstall scripts
+# $1 can be:
+#     0 - Uninstallation
+#     1 - Upgrade
+#
+
+%postun
+case $1 in
+    0)
+        # Remove user
+        userdel %{user}
+        # Remove service PID directory
+        rm -rfv /var/run/%{name}
+        # Remove firewalld rules if firewalld is installed and running
+        if [[ "$(type firewall-cmd &>/dev/null; echo $?)" -eq 0 && "$(systemctl is-active firewalld)" == "active" ]]
+        then
+            for port in s d
+            do
+                firewall-cmd --permanent --direct --remove-rule ipv4 filter INPUT 0 -p tcp -m multiport --${port}ports 11211:11220,47500:47509,47100:47109 -j ACCEPT &>/dev/null
+                firewall-cmd --permanent --direct --remove-rule ipv4 filter INPUT 0 -p udp -m multiport --${port}ports 47400:47409 -j ACCEPT &>/dev/null
+            done
+            firewall-cmd --permanent --direct --remove-rule ipv4 filter INPUT 0 -m pkttype --pkt-type multicast -j ACCEPT &>/dev/null
+            systemctl restart firewalld
+        fi
+        ;;
+    1)
+        :
+        ;;
+esac
+
+
+#-------------------------------------------------------------------------------
+#
+# Prepare packages' layout
+#
+
+%install
+cd $(ls)
+
+# Create base directory structure
+mkdir -p %{buildroot}%{_datadir}/%{name}
+mkdir -p %{buildroot}%{_libdir}/%{name}
+mkdir -p %{buildroot}%{_datadir}/doc/%{name}-%{version}/bin
+mkdir -p %{buildroot}%{_var}/log/%{name}
+mkdir -p %{buildroot}%{_sharedstatedir}/%{name}
+mkdir -p %{buildroot}%{_sysconfdir}/systemd/system
+mkdir -p %{buildroot}%{_bindir}
+
+# Copy nessessary files and remove *.bat files
+cp -rf benchmarks bin platforms %{buildroot}%{_datadir}/%{name}
+cp -rf docs/* examples %{buildroot}%{_datadir}/doc/%{name}-%{version}
+mv -f %{buildroot}%{_datadir}/%{name}/bin/ignitevisorcmd.sh %{buildroot}%{_datadir}/doc/%{name}-%{version}/bin/
+find %{buildroot}%{_datadir}/%{name}/ -name *.bat -exec rm -rf {} \;
+
+# Copy libs to /usr/lib and map them to IGNITE_HOME
+cp -rf libs/* %{buildroot}%{_libdir}/%{name}
+ln -sf %{_libdir}/%{name} %{buildroot}%{_datadir}/%{name}/libs
+
+# Setup configuration
+cp -rf config %{buildroot}%{_sysconfdir}/%{name}
+ln -sf %{_sysconfdir}/%{name} %{buildroot}%{_datadir}/%{name}/config
+
+# Setup systemctl service
+cp -rf %{_sourcedir}/name.service %{buildroot}%{_sysconfdir}/systemd/system/%{name}@.service
+cp -rf %{_sourcedir}/service.sh %{buildroot}%{_datadir}/%{name}/bin/
+chmod +x %{buildroot}%{_datadir}/%{name}/bin/service.sh
+for file in %{buildroot}%{_sysconfdir}/systemd/system/%{name}@.service %{buildroot}%{_datadir}/%{name}/bin/service.sh
+do
+    sed -i -r -e "s|#name#|%{name}|g" \
+              -e "s|#user#|%{user}|g" \
+        ${file}
+done
+
+# Map work and log directories
+ln -sf %{_sharedstatedir}/%{name} %{buildroot}%{_datadir}/%{name}/work
+ln -sf %{_var}/log/%{name} %{buildroot}%{_sharedstatedir}/%{name}/log
+
+
+#-------------------------------------------------------------------------------
+#
+# Package file list check
+#
+%files
+%dir %{_datadir}/%{name}
+%dir %{_sysconfdir}/%{name}
+%dir %{_sharedstatedir}/%{name}
+%dir %{_var}/log/%{name}
+
+%{_datadir}/%{name}/benchmarks
+%{_datadir}/%{name}/bin
+%{_datadir}/%{name}/config
+%{_datadir}/%{name}/libs
+%{_datadir}/%{name}/platforms
+%{_datadir}/%{name}/work
+%{_libdir}/%{name}
+%{_sysconfdir}/systemd/system/%{name}@.service
+%{_sharedstatedir}/%{name}/log
+
+%config(noreplace) %{_sysconfdir}/%{name}/*
+
+%doc %{name}-*/README.txt
+%doc %{name}-*/NOTICE
+%doc %{name}-*/RELEASE_NOTES.txt
+%license %{name}-*/LICENSE