You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2016/03/28 15:09:51 UTC

[1/9] ignite git commit: IGNITE-2874: .NET: Fixed IDataStreamer performance regression caused by incorrect topology size calculation. This closes #572.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1786 63ed28519 -> e2be09eb3


IGNITE-2874: .NET: Fixed IDataStreamer performance regression caused by incorrect topology size calculation. This closes #572.


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

Branch: refs/heads/ignite-1786
Commit: fb9e9b78520a6258422e9e2f498f5e9331ae5197
Parents: 166bce8
Author: Pavel Tupitsyn <pt...@gridgain.com>
Authored: Fri Mar 25 16:59:05 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 25 16:59:05 2016 +0300

----------------------------------------------------------------------
 .../datastreamer/PlatformDataStreamer.java      |   7 +-
 .../platform/PlatformAttributeNodeFilter.java   |  31 ++++++
 .../Apache.Ignite.Core.Tests.csproj             |   5 +
 .../Config/cache-local-node.xml                 |  65 ++++++++++++
 .../Dataload/DataStreamerTestTopologyChange.cs  | 104 +++++++++++++++++++
 .../Impl/Datastream/DataStreamerImpl.cs         |   9 +-
 6 files changed, 215 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index 07ef4f2..78d5d86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -162,10 +162,11 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
 
         GridDiscoveryManager discoMgr = platformCtx.kernalContext().discovery();
 
-        long topVer = discoMgr.topologyVersion();
-        int topSize = discoMgr.cacheNodes(cacheName, new AffinityTopologyVersion(topVer)).size();
+        AffinityTopologyVersion topVer = discoMgr.topologyVersionEx();
 
-        platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer, topSize);
+        int topSize = discoMgr.cacheNodes(cacheName, topVer).size();
+
+        platformCtx.gateway().dataStreamerTopologyUpdate(ptr, topVer.topologyVersion(), topSize);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/core/src/test/java/org/apache/ignite/platform/PlatformAttributeNodeFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/platform/PlatformAttributeNodeFilter.java b/modules/core/src/test/java/org/apache/ignite/platform/PlatformAttributeNodeFilter.java
new file mode 100644
index 0000000..bbd08a4
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/platform/PlatformAttributeNodeFilter.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.platform;
+
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.lang.IgnitePredicate;
+
+/**
+ * Node filter that allows nodes with an attribute.
+ */
+public class PlatformAttributeNodeFilter implements IgnitePredicate<ClusterNode> {
+    /** {@inheritDoc} */
+    @Override public boolean apply(ClusterNode node) {
+        return node.attributes().containsKey("platformAttributeNode");
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index 7cc9296..38ddd4d 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -114,6 +114,7 @@
     <Compile Include="Compute\TaskAdapterTest.cs" />
     <Compile Include="Compute\TaskResultTest.cs" />
     <Compile Include="Dataload\DataStreamerTest.cs" />
+    <Compile Include="Dataload\DataStreamerTestTopologyChange.cs" />
     <Compile Include="DataStructures\AtomicLongTest.cs" />
     <Compile Include="DataStructures\AtomicReferenceTest.cs" />
     <Compile Include="DataStructures\AtomicSequenceTest.cs" />
@@ -188,6 +189,10 @@
     <Content Include="Config\cache-binarizables.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>
+    <Content Include="Config\cache-local-node.xml">
+      <SubType>Designer</SubType>
+      <CopyToOutputDirectory>Always</CopyToOutputDirectory>
+    </Content>
     <Content Include="Config\cache-query-continuous.xml">
       <CopyToOutputDirectory>Always</CopyToOutputDirectory>
     </Content>

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-local-node.xml
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-local-node.xml b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-local-node.xml
new file mode 100644
index 0000000..d6e4dd2
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Config/cache-local-node.xml
@@ -0,0 +1,65 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+-->
+
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd
+        http://www.springframework.org/schema/util
+        http://www.springframework.org/schema/util/spring-util.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="localHost" value="127.0.0.1"/>
+        <property name="connectorConfiguration"><null/></property>
+        
+        <property name="userAttributes">
+            <map>
+                <entry key="platformAttributeNode" value="true"/>
+            </map>
+        </property>
+        
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="cache"/>
+                    <property name="nodeFilter">
+                        <bean class="org.apache.ignite.platform.PlatformAttributeNodeFilter" />
+                    </property>
+                </bean>
+            </list>
+        </property>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <!-- In distributed environment, replace with actual host IP address. -->
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs
new file mode 100644
index 0000000..c1f2c53
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Dataload/DataStreamerTestTopologyChange.cs
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Tests.Dataload
+{
+    using System;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Streamer test with topology change.
+    /// </summary>
+    public class DataStreamerTestTopologyChange
+    {
+        /// <summary>
+        /// Tests the streamer on a node without cache.
+        /// </summary>
+        [Test]
+        public void TestNoCacheNode()
+        {
+            const string cacheName = "cache";
+
+            var cacheNodeCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+            {
+                SpringConfigUrl = @"Config\cache-local-node.xml",
+                GridName = "cacheGrid"
+            };
+
+            using (var gridNoCache = Ignition.Start(TestUtils.GetTestConfiguration()))
+            {
+                Assert.Throws<ArgumentException>(() => gridNoCache.GetCache<int, int>(cacheName));
+
+                var gridWithCache = Ignition.Start(cacheNodeCfg);
+
+                var streamer = gridNoCache.GetDataStreamer<int, int>(cacheName);
+
+                streamer.AddData(1, 2);
+                streamer.Flush();
+
+                Ignition.Stop(gridWithCache.Name, true);
+
+                Thread.Sleep(500);  // Wait for node to stop
+
+                var task = streamer.AddData(2, 3);
+                streamer.Flush();
+
+                AssertThrowsCacheStopped(task);
+            }
+        }
+
+        /// <summary>
+        /// Streamer test with destroyed cache.
+        /// </summary>
+        [Test]
+        public void TestDestroyCache()
+        {
+            const string cacheName = "cache";
+
+            using (var grid = Ignition.Start(TestUtils.GetTestConfiguration()))
+            {
+                grid.CreateCache<int, int>(cacheName);
+
+                var streamer = grid.GetDataStreamer<int, int>(cacheName);
+
+                var task = streamer.AddData(1, 2);
+                streamer.Flush();
+                task.Wait();
+
+                grid.DestroyCache(cacheName);
+
+                task = streamer.AddData(2, 3);
+                streamer.Flush();
+
+                AssertThrowsCacheStopped(task);
+            }
+        }
+
+        /// <summary>
+        /// Asserts that cache stopped error is thrown.
+        /// </summary>
+        private static void AssertThrowsCacheStopped(Task task)
+        {
+            var ex = Assert.Throws<AggregateException>(task.Wait);
+            Assert.IsTrue(ex.InnerException.Message.Contains(
+                "Failed to find server node for cache " +
+                "(all affinity nodes have left the grid or cache was stopped):"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fb9e9b78/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
index 6066504..74261d3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Datastream/DataStreamerImpl.cs
@@ -19,6 +19,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
 {
     using System;
     using System.Collections.Generic;
+    using System.Diagnostics;
     using System.Diagnostics.CodeAnalysis;
     using System.Threading;
     using System.Threading.Tasks;
@@ -86,7 +87,7 @@ namespace Apache.Ignite.Core.Impl.Datastream
         private long _topVer;
 
         /** Topology size. */
-        private int _topSize;
+        private int _topSize = 1;
         
         /** Buffer send size. */
         private volatile int _bufSndSize;
@@ -568,9 +569,9 @@ namespace Apache.Ignite.Core.Impl.Datastream
                 if (_topVer < topVer)
                 {
                     _topVer = topVer;
-                    _topSize = topSize;
+                    _topSize = topSize > 0 ? topSize : 1;  // Do not set to 0 to avoid 0 buffer size.
 
-                    _bufSndSize = topSize * UU.DataStreamerPerNodeBufferSizeGet(Target);
+                    _bufSndSize = _topSize * UU.DataStreamerPerNodeBufferSizeGet(Target);
                 }
             }
             finally
@@ -590,6 +591,8 @@ namespace Apache.Ignite.Core.Impl.Datastream
         {
             int bufSndSize0 = _bufSndSize;
 
+            Debug.Assert(bufSndSize0 > 0);
+
             while (true)
             {
                 var batch0 = _batch;


[4/9] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-1786
Commit: 2cfe0cb5a0d84caede8b528b0a515ab4ba92bbc6
Parents: b4f0dab ddd9f0f
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Mar 25 17:14:03 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 25 17:14:03 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/offheap/unsafe/GridUnsafeMap.java       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[7/9] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-1786
Commit: f03b6c30a55249d8e3824f66df3aa811f994f122
Parents: 2987728 4df7418
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Mar 28 14:33:17 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Mar 28 14:33:17 2016 +0300

----------------------------------------------------------------------
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   2 +-
 .../ignite/internal/binary/BinaryUtils.java     |  16 --
 .../binary/builder/BinaryObjectBuilderImpl.java | 107 +++++++-----
 .../managers/communication/GridIoManager.java   |  43 ++++-
 .../processors/cache/GridCacheAdapter.java      |   2 +-
 .../datastreamer/DataStreamProcessor.java       |   8 +-
 .../datastreamer/DataStreamerImpl.java          |  37 +++-
 .../datastreamer/PlatformDataStreamer.java      |   7 +-
 .../util/offheap/unsafe/GridUnsafeMap.java      |   4 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  | 144 ++++++++++++++--
 ...naryObjectBuilderDefaultMappersSelfTest.java |   2 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  33 +++-
 .../igfs/IgfsPrimaryMultiNodeSelfTest.java      |  28 ++++
 ...maryRelaxedConsistencyMultiNodeSelfTest.java |  28 ++++
 .../IgfsPrimaryRelaxedConsistencySelfTest.java  |  28 ++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        |  28 ----
 .../platform/PlatformAttributeNodeFilter.java   |  31 ++++
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  12 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   3 +-
 modules/platforms/cpp/common/src/java.cpp       |  74 ++++++--
 .../Apache.Ignite.Core.Tests.csproj             |  15 ++
 .../Binary/BinaryBuilderSelfTest.cs             |  60 ++++++-
 .../Query/CacheQueriesCodeConfigurationTest.cs  |  16 +-
 .../Config/Apache.Ignite.exe.config.test2       |  58 +++++++
 .../Config/Apache.Ignite.exe.config.test3       |  34 ++++
 .../Config/cache-local-node.xml                 |  65 +++++++
 .../Config/ignite-dotnet-cfg.xml                |  52 ++++++
 .../Dataload/DataStreamerTestTopologyChange.cs  | 104 ++++++++++++
 .../Examples/ExamplesTest.cs                    |   3 +-
 .../Apache.Ignite.Core.Tests/ExecutableTest.cs  |  89 +++++++---
 .../IgniteConfigurationSerializerTest.cs        |   3 +-
 .../Apache.Ignite.Core.Tests/JavaHomeTest.cs    |  69 ++++++++
 .../Cache/Configuration/QueryEntity.cs          |   9 +-
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |  10 +-
 .../IgniteConfigurationSection.xsd              |   8 +
 .../Impl/Datastream/DataStreamerImpl.cs         |   9 +-
 .../Apache.Ignite.Core/Impl/IgniteUtils.cs      |  34 ++++
 .../dotnet/Apache.Ignite/Apache.Ignite.csproj   |   2 +-
 .../Config/AppSettingsConfigurator.cs           |  97 +++--------
 .../Apache.Ignite/Config/ArgsConfigurator.cs    | 145 ++--------------
 .../dotnet/Apache.Ignite/Config/Configurator.cs | 168 +++++++++++++++++++
 .../Apache.Ignite/Config/IConfigurator.cs       |  34 ----
 .../dotnet/Apache.Ignite/IgniteRunner.cs        |  24 +--
 .../Apache.Ignite/Service/IgniteService.cs      |  21 +--
 .../Compute/ClosureExample.cs                   |   2 +-
 .../Compute/TaskExample.cs                      |   2 +-
 .../Datagrid/ContinuousQueryExample.cs          |   2 +-
 .../Datagrid/DataStreamerExample.cs             |   2 +-
 .../Datagrid/PutGetExample.cs                   |   2 +-
 .../Datagrid/QueryExample.cs                    |   2 +-
 .../Datagrid/StoreExample.cs                    |   2 +-
 .../Datagrid/TransactionExample.cs              |   2 +-
 .../Events/EventsExample.cs                     |   2 +-
 .../Messaging/MessagingExample.cs               |   2 +-
 .../Misc/LifecycleExample.cs                    |   2 +-
 .../Services/ServicesExample.cs                 |   2 +-
 parent/pom.xml                                  |   2 +
 57 files changed, 1329 insertions(+), 463 deletions(-)
----------------------------------------------------------------------



[5/9] ignite git commit: Support optional IO policy resolver in DataStreamer.

Posted by vo...@apache.org.
Support optional IO policy resolver in DataStreamer.


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

Branch: refs/heads/ignite-1786
Commit: 4df74187072d839bdd09eb3b6163e2612fadc1c3
Parents: 2cfe0cb
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 28 12:24:16 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 28 12:24:16 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   | 43 +++++++++++++++++---
 .../processors/cache/GridCacheAdapter.java      |  2 +-
 .../datastreamer/DataStreamProcessor.java       |  8 +++-
 .../datastreamer/DataStreamerImpl.java          | 37 ++++++++++++++++-
 4 files changed, 81 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4df74187/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 4577dc8..4bc2eea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -115,6 +115,9 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /** Direct protocol version. */
     public static final byte DIRECT_PROTO_VER = 2;
 
+    /** Current IO policy. */
+    private static final ThreadLocal<Byte> CUR_PLC = new ThreadLocal<>();
+
     /** Listeners by topic. */
     private final ConcurrentMap<Object, GridMessageListener> lsnrMap = new ConcurrentHashMap8<>();
 
@@ -764,7 +767,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
                     assert obj != null;
 
-                    lsnr.onMessage(nodeId, obj);
+                    invokeListener(msg.policy(), lsnr, nodeId, obj);
                 }
                 finally {
                     threadProcessingMessage(false);
@@ -841,7 +844,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         assert obj != null;
 
-        lsnr.onMessage(nodeId, obj);
+        invokeListener(msg.policy(), lsnr, nodeId, obj);
     }
 
     /**
@@ -1197,6 +1200,38 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     }
 
     /**
+     * Invoke message listener.
+     *
+     * @param plc Policy.
+     * @param lsnr Listener.
+     * @param nodeId Node ID.
+     * @param msg Message.
+     */
+    private void invokeListener(Byte plc, GridMessageListener lsnr, UUID nodeId, Object msg) {
+        Byte oldPlc = CUR_PLC.get();
+
+        boolean change = F.eq(oldPlc, plc);
+
+        if (change)
+            CUR_PLC.set(plc);
+
+        try {
+            lsnr.onMessage(nodeId, msg);
+        }
+        finally {
+            if (change)
+                CUR_PLC.set(oldPlc);
+        }
+    }
+
+    /**
+     * @return Current IO policy
+     */
+    @Nullable public static Byte currentPolicy() {
+        return CUR_PLC.get();
+    }
+
+    /**
      * @param node Destination node.
      * @param topic Topic to send the message to.
      * @param topicOrd GridTopic enumeration ordinal.
@@ -2418,9 +2453,7 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
             for (GridTuple3<GridIoMessage, Long, IgniteRunnable> t = msgs.poll(); t != null; t = msgs.poll()) {
                 try {
-                    lsnr.onMessage(
-                        nodeId,
-                        t.get1().message());
+                    invokeListener(plc, lsnr, nodeId, t.get1().message());
                 }
                 finally {
                     if (t.get3() != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/4df74187/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 86036ac..a1f0f28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -5920,7 +5920,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 ctx.toCacheObject(val),
                 ttl,
                 0,
-                ver);
+                ver.conflictVersion());
 
             e.prepareDirectMarshal(ctx.cacheObjectContext());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/4df74187/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index d899c67..c7c1f5e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -24,6 +24,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoManager;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.processors.GridProcessorAdapter;
@@ -339,7 +340,12 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
         DataStreamerResponse res = new DataStreamerResponse(reqId, errBytes, forceLocDep);
 
         try {
-            ctx.io().send(nodeId, resTopic, res, PUBLIC_POOL);
+            Byte plc = GridIoManager.currentPolicy();
+
+            if (plc == null)
+                plc = PUBLIC_POOL;
+
+            ctx.io().send(nodeId, resTopic, res, plc);
         }
         catch (IgniteCheckedException e) {
             if (ctx.discovery().alive(nodeId))

http://git-wip-us.apache.org/repos/asf/ignite/blob/4df74187/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 7564376..4599060 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
@@ -91,6 +92,7 @@ import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
@@ -109,6 +111,9 @@ import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUB
  */
 @SuppressWarnings("unchecked")
 public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed {
+    /** Default policy reoslver. */
+    private static final DefaultIoPolicyResolver DFLT_IO_PLC_RSLVR = new DefaultIoPolicyResolver();
+
     /** Isolated receiver. */
     private static final StreamReceiver ISOLATED_UPDATER = new IsolatedUpdater();
 
@@ -118,6 +123,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /** */
     private byte[] updaterBytes;
 
+    /** IO policy resovler for data load request. */
+    private IgniteClosure<ClusterNode, Byte> ioPlcRslvr = DFLT_IO_PLC_RSLVR;
+
     /** Max remap count before issuing an error. */
     private static final int DFLT_MAX_REMAP_CNT = 32;
 
@@ -602,6 +610,13 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     }
 
     /**
+     * @param ioPlcRslvr IO policy resolver.
+     */
+    public void ioPolicyResolver(IgniteClosure<ClusterNode, Byte> ioPlcRslvr) {
+        this.ioPlcRslvr = ioPlcRslvr;
+    }
+
+    /**
      * @param entries Entries.
      * @param resFut Result future.
      * @param activeKeys Active keys.
@@ -1257,7 +1272,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             IgniteInternalFuture<Object> fut;
 
-            if (isLocNode) {
+            Byte plc = ioPlcRslvr.apply(node);
+
+            if (plc == null)
+                plc = PUBLIC_POOL;
+
+            if (isLocNode && plc == GridIoPolicy.PUBLIC_POOL) {
                 fut = ctx.closure().callLocalSafe(
                     new DataStreamerUpdateJob(ctx, log, cacheName, entries, false, skipStore, keepBinary, rcvr), false);
 
@@ -1355,7 +1375,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     topVer);
 
                 try {
-                    ctx.io().send(node, TOPIC_DATASTREAM, req, PUBLIC_POOL);
+                    ctx.io().send(node, TOPIC_DATASTREAM, req, plc);
 
                     if (log.isDebugEnabled())
                         log.debug("Sent request to node [nodeId=" + node.id() + ", req=" + req + ']');
@@ -1620,4 +1640,17 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             }
         }
     }
+
+    /**
+     * Default IO policy resolver.
+     */
+    private static class DefaultIoPolicyResolver implements IgniteClosure<ClusterNode, Byte> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public Byte apply(ClusterNode gridNode) {
+            return PUBLIC_POOL;
+        }
+    }
 }


[6/9] ignite git commit: Query Suite separation : 1,2,3

Posted by vo...@apache.org.
Query Suite separation : 1,2,3


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

Branch: refs/heads/ignite-1786
Commit: 29877281aed4d3d79133279d8aae9bb4b5d9992d
Parents: 34a9b66
Author: Anton Vinogradov <av...@apache.org>
Authored: Mon Mar 28 14:13:20 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Mon Mar 28 14:32:13 2016 +0300

----------------------------------------------------------------------
 .../IgniteBinaryCacheQueryTestSuite2.java       |  39 +++++
 .../IgniteBinaryCacheQueryTestSuite3.java       |  39 +++++
 .../IgniteCacheQuerySelfTestSuite.java          | 156 -------------------
 .../IgniteCacheQuerySelfTestSuite2.java         | 109 +++++++++++++
 .../IgniteCacheQuerySelfTestSuite3.java         | 117 ++++++++++++++
 5 files changed, 304 insertions(+), 156 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/29877281/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
new file mode 100644
index 0000000..5c9c733
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite2.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Cache query suite with binary marshaller.
+ */
+public class IgniteBinaryCacheQueryTestSuite2 extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+        TestSuite suite = IgniteCacheQuerySelfTestSuite2.suite();
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/29877281/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
new file mode 100644
index 0000000..f9e9af2
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite3.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.testframework.config.GridTestProperties;
+
+/**
+ * Cache query suite with binary marshaller.
+ */
+public class IgniteBinaryCacheQueryTestSuite3 extends TestSuite {
+    /**
+     * @return Suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        GridTestProperties.setProperty(GridTestProperties.MARSH_CLASS_NAME, BinaryMarshaller.class.getName());
+
+        TestSuite suite = IgniteCacheQuerySelfTestSuite3.suite();
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/29877281/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 0aa3560..04885ce 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -18,17 +18,8 @@
 package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
-import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest;
 import org.apache.ignite.internal.processors.cache.GridCacheReduceQueryMultithreadedSelfTest;
@@ -36,12 +27,9 @@ import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuery
 import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapTieredMultithreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePartitionedQueryMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest;
@@ -51,78 +39,22 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheQueryMultiThreaded
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheQueryOffheapMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryMultiThreadedSelfTest;
-import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheClientQueryReplicatedNodeRestartSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeFailTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNodeRestartSelfTest2;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
 import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedAtomicTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedTxTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedAtomicTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedTxTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryFilterTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxOffheapTieredTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOperationP2PTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapTieredTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapValuesTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryConcurrentTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionAtomicOneNodeTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionTxOneNodeTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicOneNodeTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapTieredTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapValuesTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
-import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryReconnectTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
-import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest;
 import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest;
-import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
 import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest;
-import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
-import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
 
 /**
  * Test suite for cache queries.
@@ -178,94 +110,6 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class);
         suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class);
 
-        // Scan queries.
-        suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
-
-        // Fields queries.
-        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
-        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
-        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
-
-        // Continuous queries.
-        suite.addTestSuite(GridCacheContinuousQueryLocalSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryTxOffheapTieredTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryTxOffheapValuesTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapTieredTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapValuesTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedTxOneNodeTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicOneNodeTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionTxOneNodeTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryPartitionAtomicOneNodeTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryReconnectTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
-        suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
-        suite.addTestSuite(CacheContinuousQueryCounterPartitionedAtomicTest.class);
-        suite.addTestSuite(CacheContinuousQueryCounterPartitionedTxTest.class);
-        suite.addTestSuite(CacheContinuousQueryCounterReplicatedAtomicTest.class);
-        suite.addTestSuite(CacheContinuousQueryCounterReplicatedTxTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
-        suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
-        suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
-        suite.addTestSuite(CacheContinuousQueryFactoryFilterTest.class);
-        suite.addTestSuite(GridCacheContinuousQueryConcurrentTest.class);
-        suite.addTestSuite(CacheContinuousQueryOperationP2PTest.class);
-        suite.addTestSuite(CacheContinuousBatchAckTest.class);
-        suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);
-
-        // Reduce fields queries.
-        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
-        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
-
-        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
-
-        suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);
-
-        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
-
-        suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class);
-
-        // Ignite cache and H2 comparison.
-        suite.addTestSuite(BaseH2CompareQueryTest.class);
-        suite.addTestSuite(H2CompareBigQueryTest.class);
-
-        // Cache query metrics.
-        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
-        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
-
-        // Unmarshalling query test.
-        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
-        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
-
-        // Other.
-        suite.addTestSuite(CacheQueryNewClientSelfTest.class);
-
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/29877281/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
new file mode 100644
index 0000000..e9fcb79
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite2.java
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheQueryOffheapEvictDataLostTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.CacheScanPartitionQueryFallbackSelfTest;
+import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest;
+import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.query.GridCacheSwapScanQuerySelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.reducefields.GridCacheReduceFieldsQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest;
+import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest;
+import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest;
+
+/**
+ * Test suite for cache queries.
+ */
+public class IgniteCacheQuerySelfTestSuite2 extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Cache Queries Test Suite 2");
+
+        // Scan queries.
+        suite.addTestSuite(CacheScanPartitionQueryFallbackSelfTest.class);
+
+        // Fields queries.
+        suite.addTestSuite(SqlFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class);
+        suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class);
+        suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class);
+
+        // Reduce fields queries.
+        suite.addTestSuite(GridCacheReduceFieldsQueryLocalSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheReduceFieldsQueryReplicatedSelfTest.class);
+
+        suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class);
+
+        suite.addTestSuite(GridCacheSwapScanQuerySelfTest.class);
+
+        suite.addTestSuite(GridOrderedMessageCancelSelfTest.class);
+
+        suite.addTestSuite(CacheQueryOffheapEvictDataLostTest.class);
+
+        // Ignite cache and H2 comparison.
+        suite.addTestSuite(BaseH2CompareQueryTest.class);
+        suite.addTestSuite(H2CompareBigQueryTest.class);
+
+        // Cache query metrics.
+        suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class);
+        suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class);
+
+        // Unmarshalling query test.
+        suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class);
+        suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class);
+
+        // Other.
+        suite.addTestSuite(CacheQueryNewClientSelfTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/29877281/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
new file mode 100644
index 0000000..3487421
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite3.java
@@ -0,0 +1,117 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.TestSuite;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchAckTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousBatchForceServerModeAckTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedAtomicTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterPartitionedTxTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedAtomicTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryCounterReplicatedTxTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFactoryFilterTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverAtomicReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryFailoverTxSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryOperationP2PTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryRandomOperationsTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicNearEnabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicOffheapValuesTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryConcurrentTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryLocalSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionAtomicOneNodeTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionTxOneNodeTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedOnlySelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryPartitionedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicOneNodeTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedAtomicSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedP2PDisabledSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryReplicatedTxOneNodeTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapTieredTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxOffheapValuesTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.GridCacheContinuousQueryTxSelfTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientReconnectTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryClientTxReconnectTest;
+import org.apache.ignite.internal.processors.cache.query.continuous.IgniteCacheContinuousQueryReconnectTest;
+
+/**
+ * Test suite for cache queries.
+ */
+public class IgniteCacheQuerySelfTestSuite3 extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Cache Queries Test Suite 3");
+
+        // Continuous queries.
+        suite.addTestSuite(GridCacheContinuousQueryLocalSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryLocalAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedOnlySelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionedP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxOffheapTieredTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryTxOffheapValuesTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicNearEnabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicP2PDisabledSelfTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapTieredTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryAtomicOffheapValuesTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedTxOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryReplicatedAtomicOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionTxOneNodeTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryPartitionAtomicOneNodeTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryReconnectTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientReconnectTest.class);
+        suite.addTestSuite(IgniteCacheContinuousQueryClientTxReconnectTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxReplicatedSelfTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterPartitionedAtomicTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterPartitionedTxTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterReplicatedAtomicTest.class);
+        suite.addTestSuite(CacheContinuousQueryCounterReplicatedTxTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverAtomicPrimaryWriteOrderOffheapTieredTest.class);
+        suite.addTestSuite(CacheContinuousQueryFailoverTxOffheapTieredTest.class);
+        suite.addTestSuite(CacheContinuousQueryRandomOperationsTest.class);
+        suite.addTestSuite(CacheContinuousQueryFactoryFilterTest.class);
+        suite.addTestSuite(GridCacheContinuousQueryConcurrentTest.class);
+        suite.addTestSuite(CacheContinuousQueryOperationP2PTest.class);
+        suite.addTestSuite(CacheContinuousBatchAckTest.class);
+        suite.addTestSuite(CacheContinuousBatchForceServerModeAckTest.class);
+
+        return suite;
+    }
+}


[9/9] ignite git commit: Merge branch 'master' into ignite-1786

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


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

Branch: refs/heads/ignite-1786
Commit: e2be09eb3bf9648e70b14d90ceaf8f03e3666e60
Parents: 63ed285 9cd2f09
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Mon Mar 28 16:09:43 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Mon Mar 28 16:09:43 2016 +0300

----------------------------------------------------------------------
 .../managers/communication/GridIoManager.java   |  43 ++++-
 .../processors/cache/GridCacheAdapter.java      |   2 +-
 .../datastreamer/DataStreamProcessor.java       |   8 +-
 .../datastreamer/DataStreamerImpl.java          |  37 ++++-
 .../datastreamer/PlatformDataStreamer.java      |   7 +-
 .../util/offheap/unsafe/GridUnsafeMap.java      |   4 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  33 +++-
 .../igfs/IgfsPrimaryMultiNodeSelfTest.java      |  28 ++++
 ...maryRelaxedConsistencyMultiNodeSelfTest.java |  28 ++++
 .../IgfsPrimaryRelaxedConsistencySelfTest.java  |  28 ++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        |  28 ----
 .../platform/PlatformAttributeNodeFilter.java   |  31 ++++
 .../IgniteCacheDataStructuresSelfTestSuite.java |   8 +-
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |  12 +-
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |   3 +-
 .../IgniteBinaryCacheQueryTestSuite2.java       |  39 +++++
 .../IgniteBinaryCacheQueryTestSuite3.java       |  39 +++++
 .../IgniteCacheQuerySelfTestSuite.java          | 156 -------------------
 .../IgniteCacheQuerySelfTestSuite2.java         | 109 +++++++++++++
 .../IgniteCacheQuerySelfTestSuite3.java         | 117 ++++++++++++++
 .../Apache.Ignite.Core.Tests.csproj             |   5 +
 .../Config/cache-local-node.xml                 |  65 ++++++++
 .../Dataload/DataStreamerTestTopologyChange.cs  | 104 +++++++++++++
 .../Impl/Datastream/DataStreamerImpl.cs         |   9 +-
 24 files changed, 732 insertions(+), 211 deletions(-)
----------------------------------------------------------------------



[8/9] ignite git commit: Added semaphore tests to data structures suite

Posted by vo...@apache.org.
Added semaphore tests to data structures suite


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

Branch: refs/heads/ignite-1786
Commit: 9cd2f09021c735468896bfdfc29d4410cb48f123
Parents: f03b6c3
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Mar 28 15:57:16 2016 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Mar 28 15:57:16 2016 +0300

----------------------------------------------------------------------
 .../testsuites/IgniteCacheDataStructuresSelfTestSuite.java   | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9cd2f090/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
index 519d3c1..f008a29 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheDataStructuresSelfTestSuite.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.local.GridCach
 import org.apache.ignite.internal.processors.cache.datastructures.local.GridCacheLocalSetSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.IgniteLocalAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.local.IgniteLocalCountDownLatchSelfTest;
+import org.apache.ignite.internal.processors.cache.datastructures.local.IgniteLocalSemaphoreSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedAtomicOffheapQueueApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedAtomicOffheapQueueCreateMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.GridCachePartitionedAtomicOffheapQueueMultiNodeSelfTest;
@@ -66,6 +67,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.partitioned.Gr
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedCountDownLatchSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedQueueNoBackupsTest;
+import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedSemaphoreSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.partitioned.IgnitePartitionedSetNoBackupsSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedAtomicReferenceApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedAtomicStampedApiSelfTest;
@@ -78,6 +80,7 @@ import org.apache.ignite.internal.processors.cache.datastructures.replicated.Gri
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.GridCacheReplicatedSetSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.IgniteReplicatedAtomicLongApiSelfTest;
 import org.apache.ignite.internal.processors.cache.datastructures.replicated.IgniteReplicatedCountDownLatchSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheAtomicReplicatedNodeRestartSelfTest;
 
 /**
  * Test suite for cache data structures.
@@ -102,6 +105,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheLocalOffheapQueueApiSelfTest.class));
         suite.addTest(new TestSuite(GridCacheLocalAtomicQueueApiSelfTest.class));
         suite.addTest(new TestSuite(IgniteLocalCountDownLatchSelfTest.class));
+        suite.addTest(new TestSuite(IgniteLocalSemaphoreSelfTest.class));
 
         suite.addTest(new TestSuite(GridCacheReplicatedSequenceApiSelfTest.class));
         suite.addTest(new TestSuite(GridCacheReplicatedSequenceMultiNodeSelfTest.class));
@@ -111,6 +115,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCacheReplicatedSetSelfTest.class));
         suite.addTest(new TestSuite(GridCacheReplicatedDataStructuresFailoverSelfTest.class));
         suite.addTest(new TestSuite(IgniteReplicatedCountDownLatchSelfTest.class));
+        suite.addTest(new TestSuite(IgniteCacheAtomicReplicatedNodeRestartSelfTest.class));
 
         suite.addTest(new TestSuite(GridCachePartitionedSequenceApiSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedSequenceMultiNodeSelfTest.class));
@@ -131,6 +136,7 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridCachePartitionedAtomicSetSelfTest.class));
         suite.addTest(new TestSuite(IgnitePartitionedCountDownLatchSelfTest.class));
         suite.addTest(new TestSuite(IgniteDataStructureWithJobTest.class));
+        suite.addTest(new TestSuite(IgnitePartitionedSemaphoreSelfTest.class));
 
         suite.addTest(new TestSuite(GridCachePartitionedSetFailoverSelfTest.class));
         suite.addTest(new TestSuite(GridCachePartitionedOffheapSetFailoverSelfTest.class));
@@ -170,4 +176,4 @@ public class IgniteCacheDataStructuresSelfTestSuite extends TestSuite {
 
         return suite;
     }
-}
\ No newline at end of file
+}


[2/9] ignite git commit: Fixed IGNITE-2863 - Memory leak in GridUnsafeMap destruct method (author Krome Plasma, remi dot malnar at some dot invalid dot address)

Posted by vo...@apache.org.
Fixed IGNITE-2863 - Memory leak in GridUnsafeMap destruct method (author Krome Plasma, remi dot malnar at some dot invalid dot address)


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

Branch: refs/heads/ignite-1786
Commit: ddd9f0ff66bee1eaf957ea63698d3c2b8c330832
Parents: fb9e9b7
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Mar 25 17:05:13 2016 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Mar 25 17:05:13 2016 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/offheap/unsafe/GridUnsafeMap.java       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ddd9f0ff/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
index 3f58447..5c9951b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMap.java
@@ -740,7 +740,7 @@ public class GridUnsafeMap implements GridOffHeapMap {
                 if (tblAddr == 0)
                     return;
 
-                for (long binAddr = tblAddr; binAddr < memCap; binAddr += 8) {
+                for (long binAddr = tblAddr, tblEnd = (tblAddr + memCap); binAddr < tblEnd; binAddr += 8) {
                     long entryAddr = Bin.first(binAddr, mem);
 
                     if (entryAddr == 0)
@@ -1817,4 +1817,4 @@ public class GridUnsafeMap implements GridOffHeapMap {
             return len == keyBytes.length && GridUnsafeMemory.compare(ptr + HEADER_SIZE, keyBytes);
         }
     }
-}
\ No newline at end of file
+}


[3/9] ignite git commit: IGNITE-2830: IGFS: Added multi-node tests. This closes #564.

Posted by vo...@apache.org.
IGNITE-2830: IGFS: Added multi-node tests. This closes #564.


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

Branch: refs/heads/ignite-1786
Commit: b4f0dab089b0acae1542b7a397bfd3f7ce2e29e6
Parents: fb9e9b7
Author: iveselovskiy <iv...@gridgain.com>
Authored: Fri Mar 25 17:13:49 2016 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Mar 25 17:13:49 2016 +0300

----------------------------------------------------------------------
 .../processors/igfs/IgfsAbstractSelfTest.java   | 33 ++++++++++++++++----
 .../igfs/IgfsPrimaryMultiNodeSelfTest.java      | 28 +++++++++++++++++
 ...maryRelaxedConsistencyMultiNodeSelfTest.java | 28 +++++++++++++++++
 .../IgfsPrimaryRelaxedConsistencySelfTest.java  | 28 +++++++++++++++++
 .../igfs/IgfsPrimaryRelaxedSelfTest.java        | 28 -----------------
 .../ignite/testsuites/IgniteIgfsTestSuite.java  | 12 +++++--
 .../ignite/igfs/Hadoop1DualAbstractTest.java    |  3 +-
 7 files changed, 123 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index 2f6c7bf..748a771 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -55,6 +55,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.Nullable;
@@ -189,6 +190,9 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** Memory mode. */
     protected final CacheMemoryMode memoryMode;
 
+    /** Ignite nodes of cluster, excluding the secondary file system node, if any. */
+    protected Ignite[] nodes;
+
     static {
         PRIMARY_REST_CFG = new IgfsIpcEndpointConfiguration();
 
@@ -233,13 +237,20 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
-     * @return Use optimzied marshaller flag.
+     * @return Use optimized marshaller flag.
      */
     protected boolean useOptimizedMarshaller() {
         return false;
     }
 
     /**
+     * @return Amount of nodes to start.
+     */
+    protected int nodeCount() {
+        return 1;
+    }
+
+    /**
      * Data chunk.
      *
      * @param len Length.
@@ -258,9 +269,17 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         igfsSecondaryFileSystem = createSecondaryFileSystemStack();
 
-        Ignite ignite = startGridWithIgfs("ignite", "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG);
+        TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+        nodes = new Ignite[nodeCount()];
+
+        for (int i = 0; i < nodes.length; i++) {
+            String nodeName = i == 0 ? "ignite" : "ignite" + i;
+
+            nodes[i] = startGridWithIgfs(nodeName, "igfs", mode, igfsSecondaryFileSystem, PRIMARY_REST_CFG, ipFinder);
+        }
 
-        igfs = (IgfsImpl) ignite.fileSystem("igfs");
+        igfs = (IgfsImpl) nodes[0].fileSystem("igfs");
     }
 
     /**
@@ -271,7 +290,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      */
     protected IgfsSecondaryFileSystem createSecondaryFileSystemStack() throws Exception {
         Ignite igniteSecondary = startGridWithIgfs("ignite-secondary", "igfs-secondary", PRIMARY, null,
-            SECONDARY_REST_CFG);
+            SECONDARY_REST_CFG, new TcpDiscoveryVmIpFinder(true));
 
         IgfsEx secondaryIgfsImpl = (IgfsEx) igniteSecondary.fileSystem("igfs-secondary");
 
@@ -300,12 +319,14 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @param mode IGFS mode.
      * @param secondaryFs Secondary file system (optional).
      * @param restCfg Rest configuration string (optional).
+     * @param ipFinder IP finder.
      * @return Started grid instance.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
     protected Ignite startGridWithIgfs(String gridName, String igfsName, IgfsMode mode,
-        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg) throws Exception {
+        @Nullable IgfsSecondaryFileSystem secondaryFs, @Nullable IgfsIpcEndpointConfiguration restCfg,
+        TcpDiscoveryIpFinder ipFinder) throws Exception {
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 
         igfsCfg.setDataCacheName("dataCache");
@@ -347,7 +368,7 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
+        discoSpi.setIpFinder(ipFinder);
 
         prepareCacheConfigurations(dataCacheCfg, metaCacheCfg);
 

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

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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.java
deleted file mode 100644
index 6691df5..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsPrimaryRelaxedSelfTest.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.internal.processors.igfs;
-
-/**
- * Tests for PRIMARY mode and relaxed consistency model.
- */
-public class IgfsPrimaryRelaxedSelfTest extends IgfsPrimarySelfTest {
-    /** {@inheritDoc} */
-    @Override protected boolean relaxedConsistency() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
index 25c54e4..0047edb 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteIgfsTestSuite.java
@@ -38,11 +38,13 @@ import org.apache.ignite.internal.processors.igfs.IgfsMetaManagerSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsMetricsSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsModeResolverSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsModesSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryMultiNodeSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsOneClientNodeTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapTieredSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOffheapValuesSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimaryOptimziedMarshallerSelfTest;
-import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest;
+import org.apache.ignite.internal.processors.igfs.IgfsPrimaryRelaxedConsistencySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsPrimarySelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorSelfTest;
 import org.apache.ignite.internal.processors.igfs.IgfsProcessorValidationSelfTest;
@@ -70,10 +72,16 @@ public class IgniteIgfsTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite FS Test Suite For Platform Independent Tests");
 
         suite.addTest(new TestSuite(IgfsPrimarySelfTest.class));
-        suite.addTest(new TestSuite(IgfsPrimaryRelaxedSelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryMultiNodeSelfTest.class));
+
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedConsistencySelfTest.class));
+        suite.addTest(new TestSuite(IgfsPrimaryRelaxedConsistencyMultiNodeSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsPrimaryOptimziedMarshallerSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsPrimaryOffheapTieredSelfTest.class));
         suite.addTest(new TestSuite(IgfsPrimaryOffheapValuesSelfTest.class));
+
         suite.addTest(new TestSuite(IgfsDualSyncSelfTest.class));
         suite.addTest(new TestSuite(IgfsDualAsyncSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/b4f0dab0/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
index 10b1bcd..f4c1cb7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/Hadoop1DualAbstractTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.hadoop.fs.CachingHadoopFileSystemFactory;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
 import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.processors.igfs.IgfsDualAbstractSelfTest;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 
 import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.IGFS_SCHEME;
 import static org.apache.ignite.igfs.HadoopSecondaryFileSystemConfigurationTest.SECONDARY_CFG_PATH;
@@ -93,7 +94,7 @@ public abstract class Hadoop1DualAbstractTest extends IgfsDualAbstractSelfTest {
      * @throws IOException On failure.
      */
     protected void startUnderlying() throws Exception {
-        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG);
+        startGridWithIgfs(GRID_NAME, IGFS_NAME, PRIMARY, null, SECONDARY_REST_CFG, new TcpDiscoveryVmIpFinder(true));
     }
 
     /**