You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by nt...@apache.org on 2017/03/21 11:41:24 UTC

[01/12] ignite git commit: IGNITE-4671 - FairAffinityFunction fails on node restart with backupFilter set and no backups

Repository: ignite
Updated Branches:
  refs/heads/master 92beda22f -> 19ba8f477


IGNITE-4671 - FairAffinityFunction fails on node restart with backupFilter set and no backups


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

Branch: refs/heads/master
Commit: 9df5e94d5cf14ddd55e29b81989177a7798f7e1a
Parents: caa3acb
Author: dkarachentsev <dk...@gridgain.com>
Authored: Tue Feb 21 15:34:59 2017 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Tue Feb 21 15:34:59 2017 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  33 ++---
 .../affinity/fair/FairAffinityNodesRestart.java | 130 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 3 files changed, 149 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
index cf1cb02..7acb5b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
@@ -911,37 +911,37 @@ public class FairAffinityFunction implements AffinityFunction {
             if (exclNeighbors)
                 return allowNeighbors || !neighborsContainPartition(node, part);
             else if (affinityBackupFilter != null) {
-                List<ClusterNode> assigment = assignments.get(part);
+                List<ClusterNode> assignment = assignments.get(part);
 
-                assert assigment.size() > 0;
+                if (assignment.isEmpty())
+                    return true;
 
                 List<ClusterNode> newAssignment;
 
                 if (tier == 0) {
-                    for (int t = 1; t < assigment.size(); t++) {
-                        newAssignment = new ArrayList<>(assigment.size() - 1);
+                    for (int t = 1; t < assignment.size(); t++) {
+                        newAssignment = new ArrayList<>(assignment.size() - 1);
 
                         newAssignment.add(node);
 
                         if (t != 1)
-                            newAssignment.addAll(assigment.subList(1, t));
+                            newAssignment.addAll(assignment.subList(1, t));
 
-                        if (t + 1 < assigment.size())
-                            newAssignment.addAll(assigment.subList(t + 1, assigment.size()));
+                        if (t + 1 < assignment.size())
+                            newAssignment.addAll(assignment.subList(t + 1, assignment.size()));
 
-                        if (!affinityBackupFilter.apply(assigment.get(t), newAssignment))
+                        if (!affinityBackupFilter.apply(assignment.get(t), newAssignment))
                             return false;
-
                     }
 
                     return true;
                 }
-                else if (tier < assigment.size()) {
-                    newAssignment = new ArrayList<>(assigment.size() - 1);
+                else if (tier < assignment.size()) {
+                    newAssignment = new ArrayList<>(assignment.size() - 1);
 
                     int i = 0;
 
-                    for (ClusterNode assignmentNode: assigment) {
+                    for (ClusterNode assignmentNode: assignment) {
                         if (i != tier)
                             newAssignment.add(assignmentNode);
 
@@ -949,17 +949,18 @@ public class FairAffinityFunction implements AffinityFunction {
                     }
                 }
                 else
-                    newAssignment = assigment;
+                    newAssignment = assignment;
 
                 return affinityBackupFilter.apply(node, newAssignment);
             }
             else if (backupFilter != null) {
                 if (tier == 0) {
-                    List<ClusterNode> assigment = assignments.get(part);
+                    List<ClusterNode> assignment = assignments.get(part);
 
-                    assert assigment.size() > 0;
+                    if (assignment.isEmpty())
+                        return true;
 
-                    List<ClusterNode> backups = assigment.subList(1, assigment.size());
+                    List<ClusterNode> backups = assignment.subList(1, assignment.size());
 
                     return !F.exist(backups, new IgnitePredicate<ClusterNode>() {
                         @Override public boolean apply(ClusterNode n) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
new file mode 100644
index 0000000..37f1bfb
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/FairAffinityNodesRestart.java
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.cache.affinity.fair;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.P2;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Tests that FairAffinityFunction doesn't throw exception on nodes restart,
+ * with backup filter set and 0 cache backups.
+ */
+public class FairAffinityNodesRestart extends GridCommonAbstractTest {
+    /** */
+    private final static P2<ClusterNode, ClusterNode> BACKUP_FILTER = new P2<ClusterNode, ClusterNode>() {
+        @Override public boolean apply(ClusterNode node, ClusterNode node2) {
+            return true;
+        }
+    };
+
+    /** */
+    private final static P2<ClusterNode, List<ClusterNode>> AFF_BACKUP_FILTER = new P2<ClusterNode, List<ClusterNode>>() {
+        @Override public boolean apply(ClusterNode node, List<ClusterNode> nodes) {
+            return true;
+        }
+    };
+
+    /** */
+    private boolean affBackup;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg = new CacheConfiguration("fair-cache");
+
+        FairAffinityFunction aff = new FairAffinityFunction(32);
+
+        if (!affBackup)
+            aff.setBackupFilter(BACKUP_FILTER);
+        else
+            aff.setAffinityBackupFilter(AFF_BACKUP_FILTER);
+
+        ccfg.setAffinity(aff);
+        ccfg.setBackups(0);
+        ccfg.setAtomicityMode(CacheAtomicityMode.ATOMIC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @param idx Node index.
+     * @return Future.
+     */
+    private IgniteInternalFuture<IgniteEx> startAsyncGrid(final int idx) {
+        return GridTestUtils.runAsync(new Callable<IgniteEx>() {
+            @Override public IgniteEx call() throws Exception {
+                return startGrid(idx);
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBackupFilter() throws Exception {
+        affBackup = false;
+
+        check();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityBackupFilter() throws Exception {
+        affBackup = true;
+
+        check();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void check() throws Exception {
+        for (int i = 0; i < 2; i++) {
+            IgniteInternalFuture<IgniteEx> fut0 = startAsyncGrid(0);
+            IgniteInternalFuture<IgniteEx> fut1 = startAsyncGrid(1);
+            IgniteInternalFuture<IgniteEx> fut2 = startAsyncGrid(2);
+
+            IgniteEx ignite = fut0.get();
+            fut1.get();
+            fut2.get();
+
+            IgniteCache<Integer, String> cache = ignite.cache("fair-cache");
+
+            for (int j = 0; j < 100; j++)
+                cache.put(i, String.valueOf(i));
+
+            stopGrid(0);
+            stopGrid(1);
+            stopGrid(2);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9df5e94d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 092d95e..a24f020 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cache.affinity.AffinityHistoryCleanupTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityDynamicCacheSelfTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionNodesSelfTest;
 import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionSelfTest;
+import org.apache.ignite.cache.affinity.fair.FairAffinityNodesRestart;
 import org.apache.ignite.cache.affinity.local.LocalAffinityFunctionTest;
 import org.apache.ignite.cache.store.GridCacheBalancingStoreSelfTest;
 import org.apache.ignite.cache.store.GridCacheLoadOnlyStoreAdapterSelfTest;
@@ -231,6 +232,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(FairAffinityFunctionNodesSelfTest.class);
         suite.addTestSuite(FairAffinityFunctionSelfTest.class);
         suite.addTestSuite(FairAffinityDynamicCacheSelfTest.class);
+        suite.addTestSuite(FairAffinityNodesRestart.class);
         suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class);
         suite.addTestSuite(IgniteCacheAffinitySelfTest.class);
         suite.addTestSuite(AffinityClientNodeSelfTest.class);


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

Posted by nt...@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/19ba8f47
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/19ba8f47
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/19ba8f47

Branch: refs/heads/master
Commit: 19ba8f477b07074439718d4e75cddefe4bf6c71a
Parents: a140a83 92beda2
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 21 14:33:23 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 21 14:33:23 2017 +0300

----------------------------------------------------------------------
 .gitignore                                      |    2 +-
 assembly/release-fabric-base.xml                |    6 +
 config/hadoop/default-config.xml                |   29 -
 examples/config/filesystem/example-igfs.xml     |   27 -
 .../aop/aspectj/GridifyAspectJAspect.java       |   15 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |   15 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |   15 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |   15 +-
 .../spring/GridifySetToValueSpringAspect.java   |   15 +-
 .../gridify/aop/spring/GridifySpringAspect.java |   15 +-
 .../apache/ignite/gridify/AbstractAopTest.java  |    6 +-
 .../ignite/gridify/GridifySetToSetTarget.java   |   18 +-
 .../gridify/GridifySetToSetTargetInterface.java |   18 +-
 .../ignite/gridify/GridifySetToValueTarget.java |   16 +-
 .../GridifySetToValueTargetInterface.java       |   16 +-
 .../ignite/gridify/NonSpringAopSelfTest.java    |    2 +-
 .../ignite/gridify/SpringAopSelfTest.java       |    2 +-
 .../apache/ignite/gridify/TestAopTarget.java    |   16 +-
 .../ignite/gridify/TestAopTargetInterface.java  |   16 +-
 .../gridify/hierarchy/GridifyHierarchyTest.java |    2 +-
 .../ignite/gridify/hierarchy/SuperTarget.java   |    4 +-
 .../apache/ignite/p2p/P2PGridifySelfTest.java   |    8 +-
 .../singlesplit/SingleSplitsLoadTest.java       |    2 +-
 .../gridify/GridifySingleSplitLoadTest.java     |    6 +-
 .../org/test/gridify/ExternalAopTarget.java     |   16 +-
 .../gridify/ExternalNonSpringAopSelfTest.java   |    6 +-
 .../spi/checkpoint/s3/S3CheckpointSpi.java      |    4 +-
 .../s3/S3CheckpointManagerSelfTest.java         |    6 +-
 .../jmh/cache/JmhCacheAbstractBenchmark.java    |    6 +-
 .../internal/TaskEventSubjectIdSelfTest.java    |    4 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |    4 +-
 .../client/ClientDefaultCacheSelfTest.java      |    4 +-
 .../internal/client/ClientStartNodeTask.java    |   14 +-
 .../internal/client/ClientStopNodeTask.java     |    2 +-
 .../ClientTcpSslAuthenticationSelfTest.java     |    4 +-
 ...skExecutionAfterTopologyRestartSelfTest.java |    4 +-
 .../internal/client/ClientTestRestServer.java   |    6 +-
 .../ClientAbstractMultiNodeSelfTest.java        |   17 +-
 .../integration/ClientAbstractSelfTest.java     |    4 +-
 .../integration/ClientPreferDirectSelfTest.java |    4 +-
 .../client/router/ClientFailedInitSelfTest.java |    4 +-
 .../client/router/RouterFactorySelfTest.java    |    4 +-
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   16 +-
 ...BinaryMarshallerInsertStatementSelfTest.java |    4 +-
 ...cBinaryMarshallerMergeStatementSelfTest.java |    4 +-
 .../jdbc2/JdbcComplexQuerySelfTest.java         |    4 +-
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |    4 +-
 .../jdbc2/JdbcDistributedJoinsQueryTest.java    |    4 +-
 .../internal/jdbc2/JdbcEmptyCacheSelfTest.java  |    4 +-
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |    4 +-
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |    4 +-
 .../internal/jdbc2/JdbcNoDefaultCacheTest.java  |    4 +-
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |    4 +-
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |    4 +-
 .../internal/jdbc2/JdbcStatementSelfTest.java   |    4 +-
 .../rest/AbstractRestProcessorSelfTest.java     |    4 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../JettyRestProcessorAbstractSelfTest.java     |   29 +-
 .../rest/JettyRestProcessorSignedSelfTest.java  |    4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |    4 +-
 .../rest/RestMemcacheProtocolSelfTest.java      |    4 +-
 .../rest/RestProcessorMultiStartSelfTest.java   |    4 +-
 .../rest/RestProcessorStartSelfTest.java        |    8 +-
 .../rest/TaskCommandHandlerSelfTest.java        |    4 +-
 .../rest/protocols/tcp/MockNioSession.java      |    5 +-
 .../tcp/redis/RedisProtocolSelfTest.java        |    4 +-
 .../ignite/jdbc/JdbcComplexQuerySelfTest.java   |    4 +-
 .../ignite/jdbc/JdbcConnectionSelfTest.java     |    6 +-
 .../ignite/jdbc/JdbcEmptyCacheSelfTest.java     |    4 +-
 .../ignite/jdbc/JdbcLocalCachesSelfTest.java    |    4 +-
 .../ignite/jdbc/JdbcMetadataSelfTest.java       |    4 +-
 .../ignite/jdbc/JdbcNoDefaultCacheTest.java     |    4 +-
 .../jdbc/JdbcPreparedStatementSelfTest.java     |    4 +-
 .../ignite/jdbc/JdbcResultSetSelfTest.java      |    4 +-
 .../ignite/jdbc/JdbcStatementSelfTest.java      |    4 +-
 .../src/test/resources/spring-server-node.xml   |    2 +-
 .../test/resources/spring-server-ssl-node.xml   |    2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   15 +-
 .../apache/ignite/IgniteSystemProperties.java   |    8 +
 .../main/java/org/apache/ignite/Ignition.java   |   43 +-
 .../org/apache/ignite/IgnitionListener.java     |    2 +-
 .../apache/ignite/binary/BinaryRawReader.java   |   13 +
 .../apache/ignite/binary/BinaryRawWriter.java   |   13 +
 .../org/apache/ignite/binary/BinaryReader.java  |   15 +
 .../org/apache/ignite/binary/BinaryWriter.java  |   15 +
 .../org/apache/ignite/cache/CacheManager.java   |    2 +-
 .../apache/ignite/cache/CacheTypeMetadata.java  |    4 +-
 .../org/apache/ignite/cache/QueryEntity.java    |    4 +-
 .../org/apache/ignite/cache/QueryIndex.java     |   25 +-
 .../ignite/cache/query/SqlFieldsQuery.java      |    4 +-
 .../org/apache/ignite/cache/query/SqlQuery.java |    6 +-
 .../apache/ignite/cache/query/TextQuery.java    |    4 +-
 .../apache/ignite/compute/gridify/Gridify.java  |   10 +
 .../ignite/compute/gridify/GridifySetToSet.java |   10 +
 .../compute/gridify/GridifySetToValue.java      |   10 +
 .../configuration/CacheConfiguration.java       |   65 +-
 .../configuration/FileSystemConfiguration.java  |   75 +-
 .../configuration/HadoopConfiguration.java      |    2 +-
 .../configuration/IgniteConfiguration.java      |  154 +-
 .../apache/ignite/hadoop/HadoopInputSplit.java  |   54 +
 .../org/apache/ignite/hadoop/HadoopJob.java     |   74 +
 .../ignite/hadoop/HadoopMapReducePlan.java      |   80 +
 .../ignite/hadoop/HadoopMapReducePlanner.java   |   40 +
 .../igfs/IgfsGroupDataBlocksKeyMapper.java      |   17 +-
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   28 +
 .../local/LocalIgfsSecondaryFileSystem.java     |  132 +-
 .../apache/ignite/internal/GridComponent.java   |   48 +-
 .../apache/ignite/internal/GridDiagnostic.java  |   16 +-
 .../ignite/internal/GridJobSiblingImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |   35 +-
 .../ignite/internal/GridKernalContextImpl.java  |   64 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   14 +-
 .../apache/ignite/internal/GridLoggerProxy.java |   25 +-
 .../ignite/internal/GridPluginComponent.java    |   21 +-
 .../org/apache/ignite/internal/GridTopic.java   |    3 +
 .../ignite/internal/IgniteComputeImpl.java      |   15 +-
 .../apache/ignite/internal/IgniteKernal.java    |   85 +-
 .../ignite/internal/IgniteNodeAttributes.java   |   10 +-
 .../ignite/internal/IgniteTransactionsEx.java   |    8 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |  304 +-
 .../ignite/internal/MappingStoreTask.java       |   59 +
 .../internal/MarshallerContextAdapter.java      |  211 --
 .../ignite/internal/MarshallerContextImpl.java  |  694 +++--
 .../internal/MarshallerMappingFileStore.java    |  174 ++
 .../ignite/internal/MarshallerPlatformIds.java  |   30 +
 .../internal/binary/BinaryClassDescriptor.java  |  103 +-
 .../ignite/internal/binary/BinaryContext.java   |   21 +-
 .../internal/binary/BinaryFieldAccessor.java    |   79 +-
 .../binary/BinaryMetadataCollector.java         |   11 +
 .../internal/binary/BinaryObjectImpl.java       |   50 +-
 .../binary/BinaryObjectOffheapImpl.java         |    9 +
 .../internal/binary/BinaryReaderExImpl.java     |  344 ++-
 .../binary/BinarySerializedFieldComparator.java |    3 +
 .../ignite/internal/binary/BinaryUtils.java     |  147 +-
 .../ignite/internal/binary/BinaryWriteMode.java |    6 +
 .../internal/binary/BinaryWriterExImpl.java     |   82 +-
 .../internal/binary/GridBinaryMarshaller.java   |    8 +-
 .../binary/builder/BinaryBuilderReader.java     |   37 +
 .../GridClientConnectionManagerAdapter.java     |    2 +-
 .../connection/GridClientNioTcpConnection.java  |    2 +-
 .../GridClientOptimizedMarshaller.java          |   16 +-
 .../client/router/impl/GridTcpRouterImpl.java   |    4 +-
 .../internal/cluster/ClusterGroupAdapter.java   |   10 +-
 .../cluster/ClusterNodeLocalMapImpl.java        |    2 +-
 .../internal/cluster/IgniteClusterImpl.java     |    2 +-
 .../igfs/common/IgfsHandshakeRequest.java       |   17 -
 .../internal/igfs/common/IgfsMarshaller.java    |    6 +-
 .../ignite/internal/jdbc2/JdbcConnection.java   |    4 +-
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |    4 +-
 .../internal/managers/GridManagerAdapter.java   |   23 +-
 .../checkpoint/GridCheckpointManager.java       |    4 +-
 .../managers/communication/GridIoManager.java   |  116 +-
 .../communication/GridIoMessageFactory.java     |   38 +-
 .../managers/communication/GridIoPolicy.java    |   13 +-
 .../deployment/GridDeploymentCommunication.java |    6 +-
 .../discovery/GridDiscoveryManager.java         |   70 +-
 .../eventstorage/GridEventStorageManager.java   |    8 +-
 .../failover/GridFailoverContextImpl.java       |   11 -
 .../managers/failover/GridFailoverManager.java  |    3 -
 .../processors/GridProcessorAdapter.java        |   21 +-
 .../affinity/GridAffinityProcessor.java         |    2 +-
 .../processors/cache/CacheObjectContext.java    |    3 +-
 .../cache/CachePartitionExchangeWorkerTask.java |   29 +
 .../internal/processors/cache/CacheType.java    |    8 +-
 .../processors/cache/GridCacheAdapter.java      |  102 +-
 .../processors/cache/GridCacheAtomicFuture.java |    5 +-
 .../cache/GridCacheClearAllRunnable.java        |    4 +-
 .../processors/cache/GridCacheContext.java      |   11 +-
 .../cache/GridCacheDeploymentManager.java       |    8 +-
 .../processors/cache/GridCacheEventManager.java |    4 +-
 .../cache/GridCacheEvictionManager.java         |    9 +-
 .../processors/cache/GridCacheGateway.java      |    3 +-
 .../processors/cache/GridCacheIoManager.java    |   95 +-
 .../processors/cache/GridCacheMapEntry.java     |   12 +-
 .../processors/cache/GridCacheMessage.java      |   17 +-
 .../processors/cache/GridCacheMvccManager.java  |   50 +-
 .../GridCachePartitionExchangeManager.java      |  179 +-
 .../processors/cache/GridCacheProcessor.java    |  320 +-
 .../processors/cache/GridCacheProxyImpl.java    |    6 +-
 .../processors/cache/GridCacheReturn.java       |    6 +-
 .../cache/GridCacheSharedContext.java           |   17 +-
 .../cache/GridCacheSharedTtlCleanupManager.java |    2 +-
 .../processors/cache/GridCacheSwapManager.java  |    4 +-
 .../processors/cache/GridCacheTtlManager.java   |    4 +-
 .../processors/cache/GridCacheUtils.java        |   19 +-
 .../cache/GridDeferredAckMessageSender.java     |   17 +-
 .../processors/cache/IgniteCacheProxy.java      |    4 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +-
 .../processors/cache/KeyCacheObjectImpl.java    |   74 +-
 .../distributed/GridCacheCommittedTxInfo.java   |  117 -
 .../distributed/GridDistributedBaseMessage.java |    2 +-
 .../GridDistributedCacheAdapter.java            |    2 +-
 .../distributed/GridDistributedLockRequest.java |   26 +-
 .../GridDistributedTxFinishRequest.java         |  217 +-
 .../GridDistributedTxFinishResponse.java        |   75 +-
 .../distributed/GridDistributedTxMapping.java   |   45 +-
 .../GridDistributedTxPrepareRequest.java        |  205 +-
 .../GridDistributedTxPrepareResponse.java       |   76 +-
 .../GridDistributedTxRemoteAdapter.java         |   59 +-
 .../GridDistributedUnlockRequest.java           |   18 +-
 .../dht/GridClientPartitionTopology.java        |   15 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   10 +-
 .../distributed/dht/GridDhtLockRequest.java     |   16 +-
 .../dht/GridDhtPartitionTopology.java           |    9 +
 .../dht/GridDhtPartitionTopologyImpl.java       |   36 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   57 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   47 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  251 +-
 .../dht/GridDhtTxFinishResponse.java            |   70 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  160 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   28 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  195 +-
 .../dht/GridDhtTxPrepareRequest.java            |  138 +-
 .../dht/GridDhtTxPrepareResponse.java           |  117 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   15 +-
 .../GridDhtAtomicAbstractUpdateFuture.java      |  298 +-
 .../GridDhtAtomicAbstractUpdateRequest.java     |  392 ++-
 .../dht/atomic/GridDhtAtomicCache.java          |  896 +++---
 .../GridDhtAtomicDeferredUpdateResponse.java    |   68 +-
 .../dht/atomic/GridDhtAtomicNearResponse.java   |  314 ++
 .../atomic/GridDhtAtomicSingleUpdateFuture.java |  101 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |  277 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   89 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |  325 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |  124 +-
 ...idNearAtomicAbstractSingleUpdateRequest.java |  481 +--
 .../GridNearAtomicAbstractUpdateFuture.java     |  468 ++-
 .../GridNearAtomicAbstractUpdateRequest.java    |  480 ++-
 .../GridNearAtomicCheckUpdateRequest.java       |  175 ++
 .../atomic/GridNearAtomicFullUpdateRequest.java |  487 +--
 ...GridNearAtomicSingleUpdateFilterRequest.java |   23 +-
 .../GridNearAtomicSingleUpdateFuture.java       |  617 ++--
 ...GridNearAtomicSingleUpdateInvokeRequest.java |   37 +-
 .../GridNearAtomicSingleUpdateRequest.java      |   65 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  962 +++---
 .../atomic/GridNearAtomicUpdateResponse.java    |  378 +--
 .../dht/atomic/NearCacheUpdates.java            |  335 +++
 .../distributed/dht/atomic/UpdateErrors.java    |  222 ++
 .../dht/colocated/GridDhtColocatedCache.java    |    8 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   25 +-
 .../dht/preloader/GridDhtPartitionDemander.java |    9 +-
 .../GridDhtPartitionsExchangeFuture.java        |    8 +-
 .../distributed/near/GridNearAtomicCache.java   |   27 +-
 .../distributed/near/GridNearGetRequest.java    |   55 +-
 .../distributed/near/GridNearLockFuture.java    |   24 +-
 .../distributed/near/GridNearLockRequest.java   |  220 +-
 .../distributed/near/GridNearLockResponse.java  |   12 +-
 ...arOptimisticSerializableTxPrepareFuture.java |   72 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   41 +-
 ...ridNearOptimisticTxPrepareFutureAdapter.java |   13 +
 .../GridNearPessimisticTxPrepareFuture.java     |   67 +-
 .../near/GridNearSingleGetRequest.java          |   46 +-
 .../near/GridNearTransactionalCache.java        |    6 +-
 .../near/GridNearTxFinishFuture.java            |  176 +-
 .../near/GridNearTxFinishRequest.java           |  174 +-
 .../near/GridNearTxFinishResponse.java          |   36 +-
 .../cache/distributed/near/GridNearTxLocal.java | 2706 ++++++++++++++++-
 .../near/GridNearTxPrepareFutureAdapter.java    |   22 +-
 .../near/GridNearTxPrepareRequest.java          |  254 +-
 .../near/GridNearTxPrepareResponse.java         |   86 +-
 .../distributed/near/GridNearTxRemote.java      |    4 +-
 .../distributed/near/GridNearUnlockRequest.java |   20 +-
 .../distributed/near/IgniteTxMappingsImpl.java  |    4 +-
 .../near/IgniteTxMappingsSingleImpl.java        |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |    4 +-
 .../cache/query/GridCacheQueryManager.java      |   64 +-
 .../continuous/CacheContinuousQueryHandler.java |    2 +-
 .../store/GridCacheStoreManagerAdapter.java     |  144 +-
 .../cache/store/GridCacheWriteBehindStore.java  |   16 +-
 .../cache/transactions/IgniteInternalTx.java    |   80 +-
 .../transactions/IgniteTransactionsImpl.java    |   12 +-
 .../cache/transactions/IgniteTxAdapter.java     |  165 +-
 .../cache/transactions/IgniteTxEntry.java       |   48 +-
 .../cache/transactions/IgniteTxHandler.java     |  157 +-
 .../IgniteTxImplicitSingleStateImpl.java        |    4 +-
 .../transactions/IgniteTxLocalAdapter.java      | 2799 ++----------------
 .../cache/transactions/IgniteTxLocalEx.java     |  145 +-
 .../cache/transactions/IgniteTxManager.java     |  299 +-
 .../cache/transactions/IgniteTxRemoteEx.java    |   11 +
 .../IgniteTxRemoteStateAdapter.java             |    2 +-
 .../cache/transactions/IgniteTxState.java       |    2 +-
 .../cache/transactions/IgniteTxStateImpl.java   |    4 +-
 .../transactions/TransactionProxyImpl.java      |   13 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |   22 +-
 .../processors/clock/GridClockServer.java       |    2 +-
 .../clock/GridClockSyncProcessor.java           |    4 +-
 .../processors/closure/AffinityTask.java        |    6 -
 .../closure/GridClosureProcessor.java           |   43 +-
 .../processors/cluster/ClusterProcessor.java    |   55 +-
 .../processors/cluster/GridUpdateNotifier.java  |   14 +-
 .../continuous/GridContinuousProcessor.java     |  148 +-
 .../datastreamer/DataStreamProcessor.java       |   63 +-
 .../datastreamer/DataStreamerImpl.java          |   39 +-
 .../datastructures/DataStructuresProcessor.java |   34 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   18 +-
 .../GridCacheAtomicReferenceImpl.java           |    6 +-
 .../GridCacheAtomicSequenceImpl.java            |    4 +-
 .../GridCacheAtomicStampedImpl.java             |    6 +-
 .../GridCacheCountDownLatchImpl.java            |    6 +-
 .../datastructures/GridCacheLockImpl.java       |   11 +-
 .../datastructures/GridCacheSemaphoreImpl.java  |   18 +-
 .../GridTransactionalCacheQueueImpl.java        |   10 +-
 .../processors/hadoop/HadoopDefaultJobInfo.java |    9 +-
 .../processors/hadoop/HadoopFileBlock.java      |    1 +
 .../processors/hadoop/HadoopInputSplit.java     |   54 -
 .../internal/processors/hadoop/HadoopJob.java   |  107 -
 .../internal/processors/hadoop/HadoopJobEx.java |  140 +
 .../processors/hadoop/HadoopJobInfo.java        |   54 +-
 .../processors/hadoop/HadoopMapReducePlan.java  |   80 -
 .../hadoop/HadoopMapReducePlanner.java          |   40 -
 .../processors/hadoop/HadoopTaskContext.java    |    6 +-
 .../processors/hadoop/HadoopTaskInfo.java       |    1 +
 .../hadoop/counter/HadoopCounterWriter.java     |    4 +-
 .../processors/igfs/IgfsBaseBlockKey.java       |   42 +
 .../internal/processors/igfs/IgfsBlockKey.java  |   26 +-
 .../processors/igfs/IgfsBlockLocationImpl.java  |   55 +
 .../internal/processors/igfs/IgfsContext.java   |    8 +-
 .../processors/igfs/IgfsDataManager.java        |   86 +-
 .../internal/processors/igfs/IgfsFileImpl.java  |    5 +-
 .../igfs/IgfsFragmentizerManager.java           |    4 +-
 .../internal/processors/igfs/IgfsImpl.java      |   58 +-
 .../processors/igfs/IgfsIpcHandler.java         |    6 +-
 .../processors/igfs/IgfsKernalContextAware.java |   32 -
 .../processors/igfs/IgfsMetaManager.java        |  102 +-
 .../processors/igfs/IgfsNoopProcessor.java      |    2 +-
 .../internal/processors/igfs/IgfsProcessor.java |  132 +-
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   10 +-
 .../igfs/IgfsSecondaryFileSystemV2.java         |   40 -
 .../internal/processors/igfs/IgfsServer.java    |    6 +-
 .../processors/igfs/IgfsServerManager.java      |    2 +-
 .../processors/igfs/IgfsThreadFactory.java      |    6 +-
 .../internal/processors/igfs/IgfsUtils.java     |  320 +-
 .../igfs/client/IgfsClientAbstractCallable.java |   23 +-
 .../igfs/client/IgfsClientAffinityCallable.java |    6 +-
 .../igfs/client/IgfsClientDeleteCallable.java   |    6 +-
 .../igfs/client/IgfsClientExistsCallable.java   |    5 +-
 .../igfs/client/IgfsClientInfoCallable.java     |    5 +-
 .../client/IgfsClientListFilesCallable.java     |    5 +-
 .../client/IgfsClientListPathsCallable.java     |    5 +-
 .../igfs/client/IgfsClientMkdirsCallable.java   |    6 +-
 .../igfs/client/IgfsClientRenameCallable.java   |    6 +-
 .../igfs/client/IgfsClientSetTimesCallable.java |    6 +-
 .../igfs/client/IgfsClientSizeCallable.java     |    5 +-
 .../igfs/client/IgfsClientSummaryCallable.java  |    5 +-
 .../igfs/client/IgfsClientUpdateCallable.java   |    6 +-
 .../meta/IgfsClientMetaIdsForPathCallable.java  |    5 +-
 .../meta/IgfsClientMetaInfoForPathCallable.java |    5 +-
 .../meta/IgfsClientMetaUnlockCallable.java      |    8 +-
 .../local/LocalFileSystemBlockKey.java          |  103 +
 .../local/LocalFileSystemIgfsFile.java          |   10 +-
 .../LocalFileSystemPositionedReadable.java      |   65 +
 .../secondary/local/LocalFileSystemUtils.java   |   25 +
 ...fsSecondaryFileSystemPositionedReadable.java |   65 -
 .../processors/job/GridJobProcessor.java        |    6 +-
 .../internal/processors/job/GridJobWorker.java  |    4 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    3 +-
 .../marshaller/ClientRequestFuture.java         |  183 ++
 .../GridMarshallerMappingProcessor.java         |  363 +++
 .../processors/marshaller/MappedName.java       |   63 +
 .../marshaller/MappingAcceptedMessage.java      |   71 +
 .../marshaller/MappingExchangeResult.java       |   96 +
 .../marshaller/MappingProposedMessage.java      |  137 +
 .../marshaller/MappingUpdatedListener.java      |   29 +
 .../marshaller/MarshallerMappingItem.java       |   99 +
 .../marshaller/MarshallerMappingTransport.java  |  212 ++
 .../MissingMappingRequestMessage.java           |  146 +
 .../MissingMappingResponseMessage.java          |  169 ++
 .../internal/processors/odbc/OdbcProcessor.java |    6 +-
 .../processors/odbc/OdbcRequestHandler.java     |    2 +
 .../platform/PlatformAbstractTarget.java        |    8 +
 .../platform/PlatformAsyncResult.java           |   41 +
 .../platform/PlatformContextImpl.java           |   10 +-
 .../processors/platform/PlatformIgnition.java   |   34 +-
 .../platform/PlatformNoopProcessor.java         |    5 +
 .../platform/PlatformPluginExtension.java       |   39 +
 .../processors/platform/PlatformProcessor.java  |    7 +
 .../platform/PlatformProcessorImpl.java         |   73 +-
 .../processors/platform/PlatformTarget.java     |   10 +
 .../platform/PlatformTargetProxy.java           |    9 +
 .../platform/PlatformTargetProxyImpl.java       |   39 +
 .../cache/PlatformCacheEntryProcessorImpl.java  |    6 +-
 .../callback/PlatformCallbackGateway.java       |  119 +
 .../platform/callback/PlatformCallbackOp.java   |   21 +
 .../platform/compute/PlatformAbstractJob.java   |    2 +-
 .../plugin/PlatformPluginProcessor.java         |   46 +
 .../cache/PlatformCachePluginConfiguration.java |   58 +
 .../cache/PlatformCachePluginProvider.java      |  123 +
 .../utils/PlatformConfigurationUtils.java       |  154 +-
 .../platform/utils/PlatformUtils.java           |   56 +-
 .../processors/plugin/CachePluginManager.java   |    8 +-
 .../plugin/IgnitePluginProcessor.java           |   84 +-
 .../internal/processors/pool/PoolProcessor.java |   15 +-
 .../processors/port/GridPortProcessor.java      |    2 +-
 .../query/GridQueryIndexDescriptor.java         |    4 +-
 .../processors/query/GridQueryIndexType.java    |   32 -
 .../processors/query/GridQueryIndexing.java     |   16 +-
 .../processors/query/GridQueryProcessor.java    | 2270 +-------------
 .../processors/query/GridQueryProperty.java     |   21 +-
 .../query/QueryIndexDescriptorImpl.java         |  102 +
 .../processors/query/QueryTypeCandidate.java    |   66 +
 .../query/QueryTypeDescriptorImpl.java          |  337 +++
 .../processors/query/QueryTypeIdKey.java        |   94 +
 .../processors/query/QueryTypeNameKey.java      |   68 +
 .../internal/processors/query/QueryUtils.java   | 1003 +++++++
 .../query/property/QueryBinaryProperty.java     |  267 ++
 .../query/property/QueryClassProperty.java      |  130 +
 .../query/property/QueryFieldAccessor.java      |   67 +
 .../query/property/QueryMethodsAccessor.java    |   82 +
 .../query/property/QueryPropertyAccessor.java   |   53 +
 .../property/QueryReadOnlyMethodsAccessor.java  |   71 +
 .../processors/resource/GridResourceIoc.java    |    6 +-
 .../resource/GridResourceProcessor.java         |   32 +-
 .../processors/rest/GridRestProcessor.java      |    8 +-
 .../handlers/task/GridTaskCommandHandler.java   |    4 +-
 .../version/GridVersionCommandHandler.java      |    2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    2 +-
 .../service/GridServiceProcessor.java           |   16 +-
 .../session/GridTaskSessionProcessor.java       |    2 +-
 .../processors/task/GridTaskProcessor.java      |    4 +-
 .../processors/task/GridTaskWorker.java         |   13 +-
 .../timeout/GridTimeoutProcessor.java           |    6 +-
 .../suggestions/GridPerformanceSuggestions.java |    6 +-
 .../ignite/internal/util/IgniteUtils.java       |  103 +-
 .../ignite/internal/util/StripedExecutor.java   |   38 +-
 .../util/future/GridCompoundFuture.java         |   11 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    7 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   22 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |    7 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   23 +-
 .../internal/util/nio/GridNioCodecFilter.java   |    9 +-
 .../util/nio/GridNioEmbeddedFuture.java         |    7 +
 .../ignite/internal/util/nio/GridNioFilter.java |   12 +-
 .../internal/util/nio/GridNioFilterAdapter.java |    7 +-
 .../internal/util/nio/GridNioFilterChain.java   |   15 +-
 .../util/nio/GridNioFinishedFuture.java         |    5 -
 .../ignite/internal/util/nio/GridNioFuture.java |    7 -
 .../internal/util/nio/GridNioFutureImpl.java    |   18 +-
 .../ignite/internal/util/nio/GridNioServer.java |  131 +-
 .../internal/util/nio/GridNioSession.java       |    7 +-
 .../internal/util/nio/GridNioSessionImpl.java   |    9 +-
 .../util/nio/GridNioSessionMetaKey.java         |    5 +-
 .../util/nio/GridShmemCommunicationClient.java  |    6 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   13 +-
 .../internal/util/nio/SessionWriteRequest.java  |    7 -
 .../internal/util/nio/ssl/GridNioSslFilter.java |   12 +-
 .../util/nio/ssl/GridNioSslHandler.java         |   29 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../ignite/internal/util/worker/GridWorker.java |   22 +-
 .../internal/visor/igfs/VisorIgfsEndpoint.java  |   26 +-
 .../visor/node/VisorBasicConfiguration.java     |   12 +-
 .../visor/node/VisorIgfsConfiguration.java      |    4 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    8 +-
 .../node/VisorNodeDataCollectorJobResult.java   |   16 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |    2 +-
 .../node/VisorNodeDataCollectorTaskResult.java  |   10 +-
 .../java/org/apache/ignite/lang/IgniteUuid.java |   25 +-
 .../ignite/marshaller/MarshallerContext.java    |   23 +-
 .../optimized/OptimizedMarshallerUtils.java     |   12 +-
 .../apache/ignite/mxbean/IgnitionMXBean.java    |   14 +-
 ...PlatformCachePluginConfigurationClosure.java |   31 +
 ...mCachePluginConfigurationClosureFactory.java |   37 +
 .../PlatformPluginConfigurationClosure.java     |   30 +
 ...atformPluginConfigurationClosureFactory.java |   37 +
 .../ignite/resources/FileSystemResource.java    |   62 +
 .../java/org/apache/ignite/spi/IgniteSpi.java   |    6 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   15 +-
 .../org/apache/ignite/spi/IgniteSpiThread.java  |    6 +-
 .../ignite/spi/IgniteSpiThreadFactory.java      |   10 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |    4 +-
 .../spi/checkpoint/jdbc/JdbcCheckpointSpi.java  |    2 +-
 .../spi/checkpoint/noop/NoopCheckpointSpi.java  |    2 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   12 +-
 .../sharedfs/SharedFsTimeoutTask.java           |    6 +-
 .../fifoqueue/FifoQueueCollisionSpi.java        |    4 +-
 .../jobstealing/JobStealingCollisionSpi.java    |    4 +-
 .../spi/collision/noop/NoopCollisionSpi.java    |    2 +-
 .../PriorityQueueCollisionSpi.java              |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   22 +-
 .../deployment/local/LocalDeploymentSpi.java    |    4 +-
 .../ignite/spi/discovery/DiscoveryDataBag.java  |  299 ++
 .../spi/discovery/DiscoverySpiDataExchange.java |   15 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |   28 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  100 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   79 +-
 .../tcp/internal/DiscoveryDataPacket.java       |  345 +++
 .../TcpDiscoveryJoinRequestMessage.java         |   18 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   10 +-
 .../messages/TcpDiscoveryNodeAddedMessage.java  |   78 +-
 .../memory/MemoryEventStorageSpi.java           |    4 +-
 .../ignite/spi/failover/FailoverContext.java    |   10 -
 .../spi/failover/always/AlwaysFailoverSpi.java  |    4 +-
 .../jobstealing/JobStealingFailoverSpi.java     |    4 +-
 .../spi/failover/never/NeverFailoverSpi.java    |    4 +-
 .../spi/indexing/noop/NoopIndexingSpi.java      |    2 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |    4 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |    4 +-
 .../WeightedRandomLoadBalancingSpi.java         |    4 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   10 +-
 .../spi/swapspace/noop/NoopSwapSpaceSpi.java    |    2 +-
 .../ignite/startup/BasicWarmupClosure.java      |    2 +-
 .../startup/cmdline/CommandLineStartup.java     |    6 +-
 .../org/apache/ignite/stream/StreamVisitor.java |    2 +-
 .../ignite/stream/socket/SocketStreamer.java    |    6 +-
 .../thread/IgniteStripedThreadPoolExecutor.java |    6 +-
 .../org/apache/ignite/thread/IgniteThread.java  |   74 +-
 .../ignite/thread/IgniteThreadFactory.java      |   18 +-
 .../ignite/thread/IgniteThreadPoolExecutor.java |   12 +-
 .../resources/META-INF/classnames.properties    |    5 +-
 .../test/config/default-spring-url-testing.xml  |    2 +-
 modules/core/src/test/config/igfs-loopback.xml  |   27 -
 modules/core/src/test/config/igfs-shmem.xml     |   27 -
 .../core/src/test/config/load/dsi-load-base.xml |    2 +-
 .../test/config/loaders/grid-cfg-2-grids.xml    |    4 +-
 .../core/src/test/config/loaders/grid-cfg.xml   |    2 +-
 .../config/websession/example-cache-client.xml  |    2 +-
 .../test/config/websession/example-cache2.xml   |    2 +-
 .../test/config/websession/spring-cache-1.xml   |    2 +-
 .../test/config/websession/spring-cache-2.xml   |    2 +-
 .../test/config/websession/spring-cache-3.xml   |    2 +-
 .../org.apache.ignite.plugin.PluginProvider     |    1 +
 ...atformCachePluginConfigurationClosureFactory |    1 +
 ...rm.PlatformPluginConfigurationClosureFactory |    1 +
 .../GridCacheAffinityBackupsSelfTest.java       |    4 +-
 .../ignite/IgniteCacheAffinitySelfTest.java     |    6 +-
 .../cache/IgniteWarmupClosureSelfTest.java      |    4 +-
 .../affinity/AffinityClientNodeSelfTest.java    |    6 +-
 ...ityFunctionBackupFilterAbstractSelfTest.java |    4 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    4 +-
 .../affinity/AffinityHistoryCleanupTest.java    |    4 +-
 .../fair/FairAffinityDynamicCacheSelfTest.java  |    4 +-
 .../fair/FairAffinityFunctionNodesSelfTest.java |    4 +-
 .../local/LocalAffinityFunctionTest.java        |    4 +-
 ...cheStoreSessionListenerAbstractSelfTest.java |    4 +-
 ...heStoreSessionListenerLifecycleSelfTest.java |    4 +-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |    4 +-
 .../IgniteCacheExpiryStoreLoadSelfTest.java     |    4 +-
 .../store/StoreResourceInjectionSelfTest.java   |    4 +-
 .../CacheJdbcPojoStoreAbstractSelfTest.java     |    4 +-
 ...eJdbcStoreAbstractMultithreadedSelfTest.java |    4 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |    4 +-
 .../ignite/igfs/IgfsEventsAbstractSelfTest.java |   64 +-
 .../igfs/IgfsFragmentizerAbstractSelfTest.java  |   21 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |    3 +-
 .../internal/ClusterGroupAbstractTest.java      |    4 +-
 .../internal/ClusterGroupHostsSelfTest.java     |    4 +-
 .../ignite/internal/ClusterMetricsSelfTest.java |    4 +-
 .../internal/ClusterNodeMetricsSelfTest.java    |    4 +-
 .../ComputeJobCancelWithServiceSelfTest.java    |    4 +-
 .../ignite/internal/GridAffinityMappedTest.java |   11 +-
 .../internal/GridAffinityP2PSelfTest.java       |   11 +-
 .../ignite/internal/GridAffinitySelfTest.java   |   10 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |    4 +-
 .../internal/GridCacheRecreateLockTest.java     |    2 +-
 .../internal/GridCancelUnusedJobSelfTest.java   |    6 +-
 .../GridCancelledJobsMetricsSelfTest.java       |    4 +-
 .../GridCollisionJobsContextSelfTest.java       |   10 +-
 .../internal/GridCommunicationSelfTest.java     |    2 +-
 ...omputationBinarylizableClosuresSelfTest.java |    4 +-
 .../GridContinuousJobAnnotationSelfTest.java    |    4 +-
 .../ignite/internal/GridDeploymentSelfTest.java |   16 +-
 .../internal/GridDiscoveryEventSelfTest.java    |    4 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    6 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |    2 +-
 ...ventStorageRuntimeConfigurationSelfTest.java |    4 +-
 .../internal/GridFactoryVmShutdownTest.java     |    2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    4 +-
 .../GridFailedInputParametersSelfTest.java      |    2 +-
 .../GridFailoverCustomTopologySelfTest.java     |    4 +-
 .../ignite/internal/GridFailoverSelfTest.java   |    4 +-
 .../GridFailoverTaskWithPredicateSelfTest.java  |    6 +-
 .../internal/GridFailoverTopologySelfTest.java  |    4 +-
 .../ignite/internal/GridGetOrStartSelfTest.java |    8 +-
 .../ignite/internal/GridHomePathSelfTest.java   |    8 +-
 .../GridJobCheckpointCleanupSelfTest.java       |    6 +-
 .../GridJobCollisionCancelSelfTest.java         |    4 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |    4 +-
 .../internal/GridJobStealingSelfTest.java       |    4 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |    6 +-
 .../internal/GridLifecycleAwareSelfTest.java    |    4 +-
 .../internal/GridLifecycleBeanSelfTest.java     |   16 +-
 .../GridLocalEventListenerSelfTest.java         |    6 +-
 .../internal/GridMultipleJobsSelfTest.java      |    6 +-
 .../internal/GridMultipleSpisSelfTest.java      |    4 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |    4 +-
 .../GridMultithreadedJobStealingSelfTest.java   |    4 +-
 .../ignite/internal/GridNodeLocalSelfTest.java  |    2 +-
 .../internal/GridNodeMetricsLogSelfTest.java    |    4 +-
 .../GridNodeVisorAttributesSelfTest.java        |    4 +-
 .../internal/GridNonHistoryMetricsSelfTest.java |    4 +-
 .../GridProjectionForCachesSelfTest.java        |   18 +-
 ...ectionLocalJobMultipleArgumentsSelfTest.java |    4 +-
 .../internal/GridReleaseTypeSelfTest.java       |    4 +-
 .../internal/GridRuntimeExceptionSelfTest.java  |    8 +-
 .../internal/GridSpiExceptionSelfTest.java      |    8 +-
 .../ignite/internal/GridStartStopSelfTest.java  |    6 +-
 .../internal/GridStopWithWaitSelfTest.java      |   12 +-
 .../GridTaskFailoverAffinityRunTest.java        |    6 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |    8 +-
 .../GridTaskInstanceExecutionSelfTest.java      |    2 +-
 .../internal/GridTaskJobRejectSelfTest.java     |    4 +-
 .../internal/GridTaskListenerSelfTest.java      |    2 +-
 .../internal/GridTaskMapAsyncSelfTest.java      |    6 +-
 .../internal/GridTaskResultCacheSelfTest.java   |    4 +-
 .../internal/GridTaskTimeoutSelfTest.java       |    8 +-
 .../IgniteClientReconnectAbstractTest.java      |    4 +-
 .../IgniteClientReconnectApiExceptionTest.java  |    4 +-
 .../IgniteClientReconnectCacheTest.java         |   13 +-
 ...niteClientReconnectFailoverAbstractTest.java |    4 +-
 .../IgniteClientReconnectFailoverTest.java      |    4 +-
 .../IgniteClientReconnectStreamerTest.java      |    4 +-
 .../IgniteComputeEmptyClusterGroupTest.java     |    4 +-
 .../internal/IgniteExecutorServiceTest.java     |   14 +-
 ...gniteExplicitImplicitDeploymentSelfTest.java |    4 +-
 .../IgniteLocalNodeMapBeforeStartTest.java      |    2 +-
 ...RoundRobinErrorAfterClientReconnectTest.java |    6 +-
 .../IgniteSlowClientDetectionSelfTest.java      |    7 +-
 ...UpdateNotifierPerClusterSettingSelfTest.java |    4 +-
 .../MarshallerContextLockingSelfTest.java       |   67 +-
 .../ignite/internal/TaskNodeRestartTest.java    |    4 +-
 .../internal/TestRecordingCommunicationSpi.java |   33 +-
 .../BinaryArrayIdentityResolverSelfTest.java    |    4 +-
 .../BinaryConfigurationConsistencySelfTest.java |    4 +-
 .../internal/binary/BinaryEnumsSelfTest.java    |    4 +-
 .../BinaryFieldIdentityResolverSelfTest.java    |    4 +-
 ...ryIdentityResolverConfigurationSelfTest.java |    4 +-
 .../binary/BinaryMarshallerSelfTest.java        |  119 +-
 .../BinaryObjectBuilderAdditionalSelfTest.java  |    4 +-
 ...naryObjectBuilderDefaultMappersSelfTest.java |    4 +-
 ...ilderSimpleNameLowerCaseMappersSelfTest.java |    4 +-
 .../binary/BinaryObjectExceptionSelfTest.java   |  209 ++
 .../binary/BinaryObjectToStringSelfTest.java    |    4 +-
 .../BinarySerialiedFieldComparatorSelfTest.java |    4 +-
 .../internal/binary/BinaryTreeSelfTest.java     |    2 +-
 .../binary/GridBinaryAffinityKeySelfTest.java   |    6 +-
 ...GridBinaryMarshallerCtxDisabledSelfTest.java |   42 +-
 ...aultBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 ...CaseBinaryMappersBinaryMetaDataSelfTest.java |    4 +-
 ...GridManagerLocalMessageListenerSelfTest.java |    6 +-
 .../managers/GridNoopManagerSelfTest.java       |    2 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   33 +-
 .../checkpoint/GridCheckpointTaskSelfTest.java  |    4 +-
 ...idCommunicationManagerListenersSelfTest.java |    6 +-
 .../GridCommunicationSendMessageSelfTest.java   |    8 +-
 .../communication/GridIoManagerSelfTest.java    |   28 +-
 .../IgniteCommunicationBalanceTest.java         |    4 +-
 .../communication/IgniteIoTestMessagesTest.java |    4 +-
 .../IgniteVariousConnectionNumberTest.java      |    6 +-
 .../GridDeploymentManagerStopSelfTest.java      |    2 +-
 .../GridDeploymentMessageCountSelfTest.java     |    6 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    6 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   10 +-
 .../IgniteTopologyPrintFormatSelfTest.java      |    8 +-
 .../events/GridEventStorageManagerSelfTest.java |    4 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |    4 +-
 .../GridCacheTxLoadFromStoreOnLockSelfTest.java |    4 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    4 +-
 .../BinaryObjectOffHeapUnswapTemporaryTest.java |    4 +-
 .../cache/CacheAffinityCallSelfTest.java        |    6 +-
 .../CacheAtomicSingleMessageCountSelfTest.java  |    4 +-
 .../cache/CacheClientStoreSelfTest.java         |    6 +-
 ...cheDhtLocalPartitionAfterRemoveSelfTest.java |    4 +-
 .../cache/CacheEntryProcessorCopySelfTest.java  |    4 +-
 .../cache/CacheEnumOperationsAbstractTest.java  |    4 +-
 ...CacheExchangeMessageDuplicatedStateTest.java |    6 +-
 .../cache/CacheFutureExceptionSelfTest.java     |    6 +-
 .../cache/CacheGetEntryAbstractTest.java        |    4 +-
 ...erceptorPartitionCounterLocalSanityTest.java |    4 +-
 ...torPartitionCounterRandomOperationsTest.java |    4 +-
 .../CacheMetricsForClusterGroupSelfTest.java    |    4 +-
 .../processors/cache/CacheNamesSelfTest.java    |    4 +-
 .../CacheNamesWithSpecialCharactersTest.java    |    4 +-
 .../cache/CacheNearReaderUpdateTest.java        |    4 +-
 .../cache/CacheOffheapMapEntrySelfTest.java     |    6 +-
 .../CachePutEventListenerErrorSelfTest.java     |    4 +-
 .../processors/cache/CachePutIfAbsentTest.java  |    4 +-
 .../cache/CacheReadThroughRestartSelfTest.java  |    6 +-
 .../cache/CacheRebalancingSelfTest.java         |   20 +-
 .../CacheSerializableTransactionsTest.java      |    4 +-
 .../CacheStartupInDeploymentModesTest.java      |    4 +-
 .../cache/CacheStopAndDestroySelfTest.java      |    6 +-
 .../CacheStoreUsageMultinodeAbstractTest.java   |   10 +-
 .../cache/CacheSwapUnswapGetTest.java           |    4 +-
 .../CacheSwapUnswapGetTestSmallQueueSize.java   |    4 +-
 .../processors/cache/CacheTxFastFinishTest.java |    4 +-
 .../CacheTxNotAllowReadFromBackupTest.java      |    4 +-
 .../processors/cache/CrossCacheLockTest.java    |    6 +-
 .../cache/CrossCacheTxRandomOperationsTest.java |    6 +-
 .../EntryVersionConsistencyReadThroughTest.java |    4 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 ...ridCacheAbstractByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   16 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   20 +-
 .../GridCacheAbstractLocalStoreSelfTest.java    |   24 +-
 .../GridCacheAbstractRemoveFailureTest.java     |    6 +-
 .../cache/GridCacheAbstractSelfTest.java        |   10 +-
 .../cache/GridCacheAbstractTxReadTest.java      |    8 +-
 ...acheAbstractUsersAffinityMapperSelfTest.java |    4 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |    4 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |    6 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |    4 +-
 ...eAtomicEntryProcessorDeploymentSelfTest.java |    4 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |   26 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |    4 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |    4 +-
 ...acheBasicStoreMultithreadedAbstractTest.java |    4 +-
 .../cache/GridCacheClearAllSelfTest.java        |    4 +-
 .../cache/GridCacheClearLocallySelfTest.java    |   10 +-
 .../cache/GridCacheClearSelfTest.java           |    4 +-
 .../cache/GridCacheConcurrentMapSelfTest.java   |    4 +-
 .../cache/GridCacheConcurrentMapTest.java       |    4 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   18 +-
 .../GridCacheConditionalDeploymentSelfTest.java |    4 +-
 ...idCacheConfigurationConsistencySelfTest.java |    4 +-
 ...ridCacheConfigurationValidationSelfTest.java |   46 +-
 .../GridCacheDaemonNodeAbstractSelfTest.java    |    4 +-
 .../cache/GridCacheDeploymentSelfTest.java      |   22 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   12 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |    4 +-
 .../GridCacheFinishPartitionsSelfTest.java      |    4 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |    4 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |    4 +-
 .../cache/GridCacheIncrementTransformTest.java  |    4 +-
 .../GridCacheInterceptorAbstractSelfTest.java   |    8 +-
 .../cache/GridCacheIteratorPerformanceTest.java |    4 +-
 .../cache/GridCacheKeyCheckSelfTest.java        |    4 +-
 .../processors/cache/GridCacheLeakTest.java     |    4 +-
 .../cache/GridCacheLifecycleAwareSelfTest.java  |    4 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |    4 +-
 .../GridCacheMarshallerTxAbstractTest.java      |    4 +-
 .../GridCacheMarshallingNodeJoinSelfTest.java   |    4 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |    4 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    4 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |    4 +-
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |    4 +-
 ...inodeUpdateNearEnabledNoBackupsSelfTest.java |    4 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |    4 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |    4 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |    4 +-
 .../cache/GridCacheOffHeapCleanupTest.java      |    4 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |    2 +-
 .../GridCacheOffHeapTieredAbstractSelfTest.java |    4 +-
 ...heOffHeapTieredEvictionAbstractSelfTest.java |    4 +-
 .../cache/GridCacheOffheapUpdateSelfTest.java   |    4 +-
 .../GridCacheOnCopyFlagAbstractSelfTest.java    |    4 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |    4 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    4 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |    4 +-
 ...hePartitionedProjectionAffinitySelfTest.java |    4 +-
 .../cache/GridCachePartitionedWritesTest.java   |    4 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |    4 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   10 +-
 .../GridCacheQueryIndexingDisabledSelfTest.java |    4 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |    4 +-
 .../GridCacheReferenceCleanupSelfTest.java      |    4 +-
 .../cache/GridCacheReloadSelfTest.java          |    4 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |    6 +-
 .../GridCacheReturnValueTransferSelfTest.java   |    4 +-
 .../cache/GridCacheSlowTxWarnTest.java          |    4 +-
 .../processors/cache/GridCacheStopSelfTest.java |    4 +-
 ...ridCacheStoreManagerDeserializationTest.java |    6 +-
 .../cache/GridCacheStorePutxSelfTest.java       |    4 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |    4 +-
 .../cache/GridCacheSwapCleanupTest.java         |    4 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |    4 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |    4 +-
 ...ridCacheSwapSpaceSpiConsistencySelfTest.java |   10 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |    4 +-
 .../processors/cache/GridCacheTestStore.java    |    2 -
 .../GridCacheTtlManagerEvictionSelfTest.java    |    4 +-
 .../GridCacheTtlManagerNotificationTest.java    |    4 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |    4 +-
 .../GridCacheValueBytesPreloadingSelfTest.java  |   10 +-
 ...idCacheValueConsistencyAbstractSelfTest.java |    4 +-
 .../GridCacheVariableTopologySelfTest.java      |    4 +-
 .../cache/GridCacheVersionMultinodeTest.java    |    4 +-
 .../GridCacheVersionTopologyChangeTest.java     |    4 +-
 .../cache/GridLocalIgniteSerializationTest.java |   10 +-
 ...ProjectionForCachesOnDaemonNodeSelfTest.java |    4 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |   10 +-
 ...IgniteCacheBinaryEntryProcessorSelfTest.java |    6 +-
 .../IgniteCacheBinaryObjectsScanSelfTest.java   |    6 +-
 ...teCacheConfigurationDefaultTemplateTest.java |    4 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    4 +-
 .../IgniteCacheContainsKeyAbstractSelfTest.java |    8 +-
 ...niteCacheCopyOnReadDisabledAbstractTest.java |    4 +-
 .../cache/IgniteCacheCreateRestartSelfTest.java |    4 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    9 +-
 ...niteCacheEntryListenerExpiredEventsTest.java |    4 +-
 .../IgniteCacheEntryProcessorCallTest.java      |    8 +-
 .../IgniteCacheEntryProcessorNodeJoinTest.java  |    4 +-
 ...niteCacheExpireAndUpdateConsistencyTest.java |    4 +-
 ...IgniteCacheGetCustomCollectionsSelfTest.java |    4 +-
 .../cache/IgniteCacheIncrementTxTest.java       |    6 +-
 ...gniteCacheInvokeReadThroughAbstractTest.java |    4 +-
 ...gniteCacheLoadRebalanceEvictionSelfTest.java |    4 +-
 .../IgniteCacheManyAsyncOperationsTest.java     |    6 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    6 +-
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |    8 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |    4 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |    6 +-
 .../IgniteCachePartitionMapUpdateTest.java      |    4 +-
 .../cache/IgniteCachePeekModesAbstractTest.java |    8 +-
 .../cache/IgniteCachePutAllRestartTest.java     |    4 +-
 .../IgniteCachePutStackOverflowSelfTest.java    |    4 +-
 .../IgniteCacheReadThroughEvictionSelfTest.java |    2 +-
 .../IgniteCacheReadThroughStoreCallTest.java    |    4 +-
 ...iteCacheScanPredicateDeploymentSelfTest.java |    6 +-
 .../cache/IgniteCacheSerializationSelfTest.java |    6 +-
 .../cache/IgniteCacheStartStopLoadTest.java     |    4 +-
 .../cache/IgniteCacheStoreCollectionTest.java   |    4 +-
 .../IgniteCacheStoreValueAbstractTest.java      |    8 +-
 .../IgniteCacheTopologySafeGetSelfTest.java     |    4 +-
 .../cache/IgniteCacheTxPreloadNoWriteTest.java  |    4 +-
 .../cache/IgniteCachingProviderSelfTest.java    |    8 +-
 .../IgniteClientAffinityAssignmentSelfTest.java |    4 +-
 .../IgniteDaemonNodeMarshallerCacheTest.java    |    4 +-
 .../cache/IgniteDynamicCacheAndNodeStop.java    |    4 +-
 .../cache/IgniteDynamicCacheFilterTest.java     |    4 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java |    6 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |    4 +-
 ...niteDynamicCacheStartStopConcurrentTest.java |    4 +-
 ...niteDynamicCacheWithConfigStartSelfTest.java |    4 +-
 .../IgniteDynamicClientCacheStartSelfTest.java  |    8 +-
 .../cache/IgniteInternalCacheTypesTest.java     |   17 +-
 ...iteMarshallerCacheClassNameConflictTest.java |  273 ++
 ...lerCacheClientRequestsMappingOnMissTest.java |  341 +++
 ...eMarshallerCacheConcurrentReadWriteTest.java |    4 +-
 .../cache/IgniteOnePhaseCommitNearSelfTest.java |   10 +-
 .../cache/IgnitePutAllLargeBatchSelfTest.java   |   10 +-
 ...tAllUpdateNonPreloadedPartitionSelfTest.java |   10 +-
 .../IgniteStartCacheInTransactionSelfTest.java  |    4 +-
 .../cache/IgniteStaticCacheStartSelfTest.java   |    4 +-
 .../cache/IgniteSystemCacheOnClientTest.java    |   29 +-
 ...gniteTopologyValidatorAbstractCacheTest.java |   10 +-
 ...niteTopologyValidatorGridSplitCacheTest.java |    2 +-
 .../processors/cache/IgniteTxAbstractTest.java  |    4 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |    4 +-
 .../cache/IgniteTxConfigCacheSelfTest.java      |    8 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |   10 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |    4 +-
 .../cache/IgniteTxReentryAbstractSelfTest.java  |    4 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    8 +-
 .../MarshallerCacheJobRunNodeRestartTest.java   |    4 +-
 .../cache/OffHeapTieredTransactionSelfTest.java |    4 +-
 ...heapCacheMetricsForClusterGroupSelfTest.java |    4 +-
 .../cache/OffheapCacheOnClientsTest.java        |    4 +-
 ...sExchangeOnDiscoveryHistoryOverflowTest.java |    2 +-
 .../CacheKeepBinaryWithInterceptorTest.java     |    4 +-
 .../GridBinaryCacheEntryMemorySizeSelfTest.java |   21 +-
 ...ryDuplicateIndexObjectsAbstractSelfTest.java |    8 +-
 ...acheBinaryObjectUserClassloaderSelfTest.java |   10 +-
 ...naryObjectsAbstractDataStreamerSelfTest.java |    4 +-
 ...aryObjectsAbstractMultiThreadedSelfTest.java |    4 +-
 .../GridCacheBinaryObjectsAbstractSelfTest.java |    4 +-
 .../GridCacheBinaryStoreAbstractSelfTest.java   |    4 +-
 ...yStoreBinariesSimpleNameMappersSelfTest.java |    4 +-
 ...ntNodeBinaryObjectMetadataMultinodeTest.java |    4 +-
 ...CacheClientNodeBinaryObjectMetadataTest.java |    6 +-
 .../DataStreamProcessorBinarySelfTest.java      |    4 +-
 .../GridDataStreamerImplSelfTest.java           |    4 +-
 .../GridCacheAffinityRoutingBinarySelfTest.java |    4 +-
 ...sNearPartitionedByteArrayValuesSelfTest.java |    4 +-
 ...sPartitionedOnlyByteArrayValuesSelfTest.java |    4 +-
 .../dht/GridCacheMemoryModeBinarySelfTest.java  |    4 +-
 ...dCacheOffHeapTieredAtomicBinarySelfTest.java |    4 +-
 .../GridCacheOffHeapTieredBinarySelfTest.java   |    4 +-
 ...fHeapTieredEvictionAtomicBinarySelfTest.java |    4 +-
 ...acheOffHeapTieredEvictionBinarySelfTest.java |    4 +-
 ...IgniteCacheAbstractExecutionContextTest.java |    8 +-
 ...niteCacheContinuousExecutionContextTest.java |    4 +-
 ...IgniteCacheIsolatedExecutionContextTest.java |    4 +-
 ...niteCacheP2PDisableExecutionContextTest.java |    4 +-
 .../IgniteCachePrivateExecutionContextTest.java |    4 +-
 .../IgniteCacheSharedExecutionContextTest.java  |    4 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |   46 +-
 ...actQueueFailoverDataConsistencySelfTest.java |    4 +-
 .../GridCacheQueueCleanupSelfTest.java          |    4 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |    6 +-
 .../GridCacheSequenceApiSelfAbstractTest.java   |    4 +-
 .../IgniteAtomicLongApiAbstractSelfTest.java    |    4 +-
 .../IgniteAtomicsAbstractTest.java              |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |    6 +-
 .../IgniteCollectionAbstractTest.java           |    4 +-
 .../IgniteDataStructureUniqueNameTest.java      |    4 +-
 .../IgniteDataStructureWithJobTest.java         |    4 +-
 .../IgniteSemaphoreAbstractSelfTest.java        |    2 +-
 ...SemaphoreFailoverSafeReleasePermitsTest.java |    4 +-
 ...achePartitionedAtomicSequenceTxSelfTest.java |    4 +-
 ...idCachePartitionedNodeRestartTxSelfTest.java |    4 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |    4 +-
 .../CacheAbstractRestartSelfTest.java           |    6 +-
 .../distributed/CacheAffinityEarlyTest.java     |    4 +-
 ...acheAsyncOperationsFailoverAbstractTest.java |    8 +-
 .../distributed/CacheAsyncOperationsTest.java   |    6 +-
 .../CacheGetFutureHangsSelfTest.java            |    4 +-
 .../CacheGetInsideLockChangingTopologyTest.java |    4 +-
 ...ffinityAssignmentNodeJoinValidationTest.java |    4 +-
 .../CacheLateAffinityAssignmentTest.java        |   59 +-
 ...CacheLoadingConcurrentGridStartSelfTest.java |   12 +-
 .../CacheLockReleaseNodeLeaveTest.java          |    4 +-
 .../CachePutAllFailoverAbstractTest.java        |    8 +-
 .../CacheTryLockMultithreadedTest.java          |    4 +-
 ...tractDistributedByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractJobExecutionTest.java      |    8 +-
 .../GridCacheAbstractNodeRestartSelfTest.java   |    4 +-
 ...tractPartitionedByteArrayValuesSelfTest.java |    4 +-
 .../GridCacheAbstractPrimarySyncSelfTest.java   |    4 +-
 .../GridCacheBasicOpAbstractTest.java           |    4 +-
 .../GridCacheClientModesAbstractSelfTest.java   |   24 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |    4 +-
 ...acheEntrySetIterationPreloadingSelfTest.java |    4 +-
 .../distributed/GridCacheEventAbstractTest.java |    2 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |    4 +-
 .../distributed/GridCacheLockAbstractTest.java  |    4 +-
 .../distributed/GridCacheMixedModeSelfTest.java |   12 +-
 .../GridCacheMultiNodeAbstractTest.java         |    4 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |    4 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |    8 +-
 .../GridCacheNodeFailureAbstractTest.java       |    4 +-
 ...ridCachePartitionNotLoadedEventSelfTest.java |    8 +-
 ...chePartitionedReloadAllAbstractSelfTest.java |    4 +-
 .../GridCachePreloadEventsAbstractSelfTest.java |    4 +-
 .../GridCachePreloadLifecycleAbstractTest.java  |    4 +-
 ...GridCachePreloadRestartAbstractSelfTest.java |    4 +-
 .../GridCacheTransformEventSelfTest.java        |    4 +-
 ...yMetadataUpdateChangingTopologySelfTest.java |   11 +-
 ...niteBinaryMetadataUpdateNodeRestartTest.java |    4 +-
 .../distributed/IgniteCache150ClientsTest.java  |    6 +-
 ...micMessageRecoveryPairedConnectionsTest.java |    4 +-
 ...niteCacheClientNodeChangingTopologyTest.java |   22 +-
 .../IgniteCacheClientNodeConcurrentStart.java   |    6 +-
 ...teCacheClientNodePartitionsExchangeTest.java |    8 +-
 .../IgniteCacheClientReconnectTest.java         |    4 +-
 ...acheConnectionRecovery10ConnectionsTest.java |    4 +-
 .../IgniteCacheConnectionRecoveryTest.java      |    4 +-
 .../IgniteCacheCreatePutMultiNodeSelfTest.java  |    4 +-
 .../distributed/IgniteCacheCreatePutTest.java   |    4 +-
 .../distributed/IgniteCacheGetRestartTest.java  |    6 +-
 .../distributed/IgniteCacheManyClientsTest.java |    4 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |    6 +-
 ...eCacheMessageRecoveryIdleConnectionTest.java |    4 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |    4 +-
 .../IgniteCacheNearOffheapGetSelfTest.java      |   12 +-
 .../IgniteCacheNearRestartRollbackSelfTest.java |   12 +-
 .../IgniteCacheNodeJoinAbstractTest.java        |    4 +-
 .../distributed/IgniteCachePrimarySyncTest.java |    4 +-
 .../IgniteCachePutGetRestartAbstractTest.java   |   10 +-
 .../IgniteCacheReadFromBackupTest.java          |    4 +-
 .../IgniteCacheServerNodeConcurrentStart.java   |    4 +-
 .../IgniteCacheSingleGetMessageTest.java        |    4 +-
 .../IgniteCacheSizeFailoverTest.java            |    4 +-
 .../IgniteCacheSystemTransactionsSelfTest.java  |   29 +-
 .../IgniteCacheTxFairAffinityNodeJoinTest.java  |    4 +-
 .../IgniteCacheTxIteratorSelfTest.java          |    4 +-
 ...arDisabledFairAffinityPutGetRestartTest.java |    4 +-
 .../IgniteCrossCacheTxStoreSelfTest.java        |   16 +-
 .../IgniteTxCachePrimarySyncTest.java           |    9 +-
 ...teSynchronizationModesMultithreadedTest.java |    4 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |   10 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   14 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   12 +-
 .../IgniteTxPreloadAbstractTest.java            |    4 +-
 ...tPartitionedOnlyByteArrayValuesSelfTest.java |    4 +-
 ...heAbstractTransformWriteThroughSelfTest.java |    4 +-
 .../dht/GridCacheAtomicFullApiSelfTest.java     |    4 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |   41 +-
 .../dht/GridCacheColocatedDebugTest.java        |    4 +-
 ...eColocatedOptimisticTransactionSelfTest.java |    4 +-
 ...dCacheColocatedTxSingleThreadedSelfTest.java |    4 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |    4 +-
 .../dht/GridCacheDhtEntrySetSelfTest.java       |    4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    4 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |    4 +-
 .../GridCacheDhtEvictionsDisabledSelfTest.java  |    4 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |    4 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |    4 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |    9 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |    6 +-
 .../GridCacheDhtPreloadMessageCountTest.java    |    4 +-
 ...ridCacheDhtPreloadMultiThreadedSelfTest.java |   10 +-
 .../dht/GridCacheDhtPreloadOffHeapSelfTest.java |    4 +-
 .../dht/GridCacheDhtPreloadPerformanceTest.java |    4 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |    4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   16 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |    8 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |    4 +-
 .../dht/GridCacheDhtTxPreloadSelfTest.java      |    4 +-
 .../dht/GridCacheGlobalLoadTest.java            |    4 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...ePartitionedNearDisabledMetricsSelfTest.java |    8 +-
 ...idCachePartitionedPreloadEventsSelfTest.java |    4 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |    6 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |    4 +-
 ...ridCachePartitionedUnloadEventsSelfTest.java |    4 +-
 .../dht/GridCacheTxNodeFailureSelfTest.java     |   10 +-
 .../dht/GridNearCacheTxNodeFailureSelfTest.java |    4 +-
 ...gniteAtomicLongChangingTopologySelfTest.java |    4 +-
 .../IgniteCacheCommitDelayTxRecoveryTest.java   |    4 +-
 .../dht/IgniteCacheConcurrentPutGetRemove.java  |    4 +-
 .../IgniteCacheCrossCacheTxFailoverTest.java    |    6 +-
 .../dht/IgniteCacheLockFailoverSelfTest.java    |    8 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    4 +-
 ...artitionedBackupNodeFailureRecoveryTest.java |    4 +-
 ...woBackupsPrimaryNodeFailureRecoveryTest.java |    4 +-
 ...ePrimaryNodeFailureRecoveryAbstractTest.java |   13 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   41 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |    2 +-
 .../dht/IgniteCacheTxRecoveryRollbackTest.java  |  501 ++++
 .../dht/IgniteCrossCacheTxSelfTest.java         |    4 +-
 .../AtomicPutAllChangingTopologyTest.java       |    2 +-
 .../atomic/GridCacheAtomicFailoverSelfTest.java |    4 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    6 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |    4 +-
 ...AtomicPrimaryWriteOrderFailoverSelfTest.java |    4 +-
 .../atomic/IgniteCacheAtomicProtocolTest.java   |  883 ++++++
 ...tionedMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nabledMultiNodeLongTxTimeoutFullApiTest.java |    4 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...tomicClientOnlyMultiNodeFullApiSelfTest.java |    4 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 ...micFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...imaryWriteOrderMultiNodeFullApiSelfTest.java |    4 +-
 ...GridCacheAtomicMultiNodeFullApiSelfTest.java |    4 +-
 ...omicMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...eAtomicNearOnlyMultiNodeFullApiSelfTest.java |    4 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheAtomicOffHeapFullApiSelfTest.java  |    4 +-
 ...heAtomicOffHeapMultiNodeFullApiSelfTest.java |    4 +-
 ...idCacheAtomicPartitionedMetricsSelfTest.java |    4 +-
 ...derFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 .../near/GridCacheGetStoreErrorSelfTest.java    |    4 +-
 .../near/GridCacheNearEvictionSelfTest.java     |    4 +-
 .../near/GridCacheNearJobExecutionSelfTest.java |    4 +-
 .../near/GridCacheNearMetricsSelfTest.java      |    4 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |    8 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |    4 +-
 ...nlyFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   10 +-
 ...OnlyMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    4 +-
 .../GridCacheNearPartitionedClearSelfTest.java  |    4 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |    6 +-
 .../near/GridCacheNearReadersSelfTest.java      |   21 +-
 .../near/GridCacheNearTxForceKeyTest.java       |    4 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |    4 +-
 .../near/GridCacheNearTxPreloadSelfTest.java    |    4 +-
 ...AffinityExcludeNeighborsPerformanceTest.java |    4 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |    4 +-
 .../GridCachePartitionedAffinitySelfTest.java   |   20 +-
 .../near/GridCachePartitionedBasicApiTest.java  |    4 +-
 .../GridCachePartitionedBasicOpSelfTest.java    |    4 +-
 ...ePartitionedBasicStoreMultiNodeSelfTest.java |    4 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    4 +-
 ...yOnReadDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../near/GridCachePartitionedEventSelfTest.java |    2 +-
 .../GridCachePartitionedEvictionSelfTest.java   |    4 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |    4 +-
 ...GridCachePartitionedFilteredPutSelfTest.java |    4 +-
 .../GridCachePartitionedFullApiSelfTest.java    |    8 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |    4 +-
 ...LateAffDisabledMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |    4 +-
 .../near/GridCachePartitionedLockSelfTest.java  |    4 +-
 .../GridCachePartitionedMetricsSelfTest.java    |    8 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |   79 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   12 +-
 ...onedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedMultiNodeSelfTest.java  |    4 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |    4 +-
 .../near/GridCachePartitionedNestedTxTest.java  |    4 +-
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 -
 ...GridCachePartitionedNodeFailureSelfTest.java |    4 +-
 .../GridCachePartitionedNodeRestartTest.java    |    4 +-
 ...ePartitionedOptimisticTxNodeRestartTest.java |    4 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |    4 +-
 .../GridCachePartitionedStorePutSelfTest.java   |    4 +-
 ...GridCachePartitionedTxConcurrentGetTest.java |    4 +-
 ...GridCachePartitionedTxMultiNodeSelfTest.java |    4 +-
 ...CachePartitionedTxMultiThreadedSelfTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   11 +-
 ...achePartitionedTxSingleThreadedSelfTest.java |    4 +-
 .../GridCachePartitionedTxTimeoutSelfTest.java  |    4 +-
 .../near/GridCachePutArrayValueSelfTest.java    |    4 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |    4 +-
 .../near/IgniteCacheNearOnlyTxTest.java         |    6 +-
 .../near/IgniteCacheNearTxRollbackTest.java     |    4 +-
 .../near/NearCacheMultithreadedUpdateTest.java  |    4 +-
 .../near/NearCachePutAllMultinodeTest.java      |    4 +-
 .../near/NearCacheSyncUpdateTest.java           |    4 +-
 .../near/NoneRebalanceModeSelfTest.java         |    4 +-
 ...cingDelayedPartitionMapExchangeSelfTest.java |    4 +-
 .../GridCacheRebalancingAsyncSelfTest.java      |    4 +-
 .../GridCacheRebalancingOrderingTest.java       |    6 +-
 .../GridCacheRebalancingSyncCheckDataTest.java  |    4 +-
 .../GridCacheRebalancingSyncSelfTest.java       |    6 +-
 ...eRebalancingUnmarshallingFailedSelfTest.java |    4 +-
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |    4 +-
 ...tedFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...xcludeNeighborsMultiNodeFullApiSelfTest.java |    4 +-
 ...dezvousAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...stractReplicatedByteArrayValuesSelfTest.java |    4 +-
 ...ridCacheAtomicReplicatedMetricsSelfTest.java |    4 +-
 ...nedFairAffinityMultiNodeFullApiSelfTest.java |    4 +-
 ...eplicatedAtomicMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedBasicApiTest.java        |    4 +-
 .../GridCacheReplicatedBasicOpSelfTest.java     |    4 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |    4 +-
 .../GridCacheReplicatedFullApiSelfTest.java     |    8 +-
 .../GridCacheReplicatedJobExecutionTest.java    |    4 +-
 .../GridCacheReplicatedMarshallerTxTest.java    |    4 +-
 .../GridCacheReplicatedMetricsSelfTest.java     |    4 +-
 ...atedMultiNodeP2PDisabledFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedMultiNodeSelfTest.java   |    4 +-
 .../GridCacheReplicatedNodeFailureSelfTest.java |    4 +-
 .../GridCacheReplicatedTxConcurrentGetTest.java |    4 +-
 ...GridCacheReplicatedTxMultiNodeBasicTest.java |    4 +-
 ...dCacheReplicatedTxMultiThreadedSelfTest.java |    4 +-
 ...CacheReplicatedTxSingleThreadedSelfTest.java |    4 +-
 .../GridCacheReplicatedTxTimeoutSelfTest.java   |    4 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |    6 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    4 +-
 .../replicated/GridReplicatedTxPreloadTest.java |    4 +-
 .../IgniteCacheSyncRebalanceModeSelfTest.java   |    4 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |    4 +-
 ...idCacheReplicatedPreloadOffHeapSelfTest.java |    4 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |   14 +-
 ...eplicatedPreloadStartStopEventsSelfTest.java |    4 +-
 .../cache/eviction/EvictionAbstractTest.java    |    4 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |    4 +-
 ...heConcurrentEvictionConsistencySelfTest.java |    4 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |    4 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    4 +-
 .../GridCacheEmptyEntriesAbstractSelfTest.java  |    4 +-
 .../GridCacheEvictionFilterSelfTest.java        |    4 +-
 .../GridCacheEvictionLockUnlockSelfTest.java    |    4 +-
 .../GridCacheEvictionTouchSelfTest.java         |    4 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |    4 +-
 .../lru/LruNearEvictionPolicySelfTest.java      |    4 +-
 .../LruNearOnlyNearEvictionPolicySelfTest.java  |    4 +-
 .../SortedEvictionPolicyPerformanceTest.java    |    4 +-
 .../IgniteCacheClientNearCacheExpiryTest.java   |    6 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    4 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |    4 +-
 .../expiry/IgniteCacheTtlCleanupSelfTest.java   |    4 +-
 .../IgniteCacheLoadAllAbstractTest.java         |    4 +-
 ...iteCacheNoLoadPreviousValueAbstractTest.java |    8 +-
 .../IgniteCacheNoReadThroughAbstractTest.java   |    8 +-
 .../IgniteCacheNoWriteThroughAbstractTest.java  |    8 +-
 ...IgniteCacheStoreNodeRestartAbstractTest.java |    6 +-
 .../IgniteCacheStoreSessionAbstractTest.java    |    6 +-
 ...acheStoreSessionWriteBehindAbstractTest.java |    6 +-
 ...dCacheAtomicLocalMetricsNoStoreSelfTest.java |    4 +-
 .../GridCacheAtomicLocalMetricsSelfTest.java    |    4 +-
 .../local/GridCacheLocalBasicApiSelfTest.java   |    4 +-
 .../GridCacheLocalByteArrayValuesSelfTest.java  |    4 +-
 .../local/GridCacheLocalFullApiSelfTest.java    |    8 +-
 .../local/GridCacheLocalLoadAllSelfTest.java    |    4 +-
 .../local/GridCacheLocalMetricsSelfTest.java    |    8 +-
 .../GridCacheLocalTxMultiThreadedSelfTest.java  |    4 +-
 .../GridCacheLocalTxSingleThreadedSelfTest.java |    4 +-
 ...LocalCacheOffHeapAndSwapMetricsSelfTest.java |    4 +-
 .../BinaryTxCacheLocalEntriesSelfTest.java      |    8 +-
 .../GridCacheQueryTransformerSelfTest.java      |    4 +-
 .../GridCacheSwapScanQueryAbstractSelfTest.java |    4 +-
 .../cache/query/IndexingSpiQuerySelfTest.java   |    2 +-
 .../cache/query/IndexingSpiQueryTxSelfTest.java |    8 +-
 .../continuous/CacheContinuousBatchAckTest.java |   12 +-
 ...heContinuousBatchForceServerModeAckTest.java |    8 +-
 ...eContinuousQueryAsyncFilterListenerTest.java |    6 +-
 ...CacheContinuousQueryCounterAbstractTest.java |    8 +-
 ...acheContinuousQueryExecuteInPrimaryTest.java |    4 +-
 ...ContinuousQueryFailoverAbstractSelfTest.java |   35 +-
 .../CacheContinuousQueryLostPartitionTest.java  |    6 +-
 ...ontinuousQueryOperationFromCallbackTest.java |    4 +-
 .../CacheContinuousQueryOperationP2PTest.java   |    4 +-
 .../CacheContinuousQueryOrderingEventTest.java  |    4 +-
 ...acheContinuousQueryRandomOperationsTest.java |    4 +-
 .../CacheContinuousQueryVariationsTest.java     |    6 +-
 ...eEntryProcessorExternalizableFailedTest.java |    4 +-
 .../CacheEntryProcessorNonSerializableTest.java |    4 +-
 ...CacheKeepBinaryIterationSwapEnabledTest.java |    4 +-
 .../CacheKeepBinaryIterationTest.java           |    4 +-
 .../ClientReconnectContinuousQueryTest.java     |    2 +-
 ...yRemoteFilterMissingInClassPathSelfTest.java |    4 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |   12 +-
 .../GridCacheContinuousQueryConcurrentTest.java |    6 +-
 ...eContinuousQueryMultiNodesFilteringTest.java |   10 +-
 ...eContinuousQueryReplicatedTxOneNodeTest.java |    4 +-
 ...niteCacheContinuousQueryBackupQueueTest.java |    4 +-
 ...CacheContinuousQueryClientReconnectTest.java |    4 +-
 .../IgniteCacheContinuousQueryClientTest.java   |    4 +-
 ...eCacheContinuousQueryImmutableEntryTest.java |    6 +-
 ...teCacheContinuousQueryNoUnsubscribeTest.java |    4 +-
 ...IgniteCacheContinuousQueryReconnectTest.java |    4 +-
 ...BehindStorePartitionedMultiNodeSelfTest.java |    4 +-
 .../IgniteCacheWriteBehindNoUpdateSelfTest.java |    4 +-
 ...CacheClientWriteBehindStoreAbstractTest.java |   12 +-
 .../TxDeadlockDetectionNoHangsTest.java         |    4 +-
 .../transactions/TxDeadlockDetectionTest.java   |    4 +-
 ...timisticDeadlockDetectionCrossCacheTest.java |    4 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   34 +-
 ...simisticDeadlockDetectionCrossCacheTest.java |    4 +-
 .../TxPessimisticDeadlockDetectionTest.java     |    4 +-
 ...edEntryPartitionedAtomicOffHeapSelfTest.java |    4 +-
 ...PartitionedTransactionalOffHeapSelfTest.java |    4 +-
 ...nedEntryReplicatedAtomicOffHeapSelfTest.java |    4 +-
 ...yReplicatedTransactionalOffHeapSelfTest.java |    4 +-
 .../clock/GridTimeSyncProcessorSelfTest.java    |    4 +-
 .../closure/GridClosureProcessorRemoteTest.java |    2 +-
 .../closure/GridClosureProcessorSelfTest.java   |    4 +-
 .../closure/GridClosureSerializationTest.java   |    4 +-
 .../cluster/GridAddressResolverSelfTest.java    |    6 +-
 ...ComputeJobExecutionErrorToLogManualTest.java |    4 +-
 .../compute/PublicThreadpoolStarvationTest.java |    4 +-
 .../continuous/GridEventConsumeSelfTest.java    |    4 +-
 .../continuous/GridMessageListenSelfTest.java   |    4 +-
 .../IgniteNoCustomEventsOnNodeStart.java        |    4 +-
 .../DataStreamProcessorSelfTest.java            |  114 +-
 .../datastreamer/DataStreamerImplSelfTest.java  |    4 +-
 .../DataStreamerMultiThreadedSelfTest.java      |    4 +-
 .../DataStreamerMultinodeCreateCacheTest.java   |    4 +-
 .../datastreamer/DataStreamerTimeoutTest.java   |    4 +-
 .../DataStreamerUpdateAfterLoadTest.java        |    4 +-
 .../IgniteDataStreamerPerformanceTest.java      |    4 +-
 .../igfs/IgfsAbstractBaseSelfTest.java          |   23 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |   18 +-
 .../igfs/IgfsBackupFailoverSelfTest.java        |   20 +-
 ...lockMessageSystemPoolStarvationSelfTest.java |   23 +-
 ...sCachePerBlockLruEvictionPolicySelfTest.java |   22 +-
 .../processors/igfs/IgfsCacheSelfTest.java      |   23 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |    8 +-
 .../igfs/IgfsDataManagerSelfTest.java           |   37 +-
 .../igfs/IgfsDualAbstractSelfTest.java          |   26 +-
 ...SecondaryFileSystemDualAbstractSelfTest.java |    5 -
 ...fsLocalSecondaryFileSystemProxySelfTest.java |   81 +
 ...IgfsLocalSecondaryFileSystemTestAdapter.java |    7 +-
 .../processors/igfs/IgfsMaxSizeSelfTest.java    |   12 +-
 .../igfs/IgfsMetaManagerSelfTest.java           |   10 +-
 .../processors/igfs/IgfsMetricsSelfTest.java    |   20 +-
 .../processors/igfs/IgfsModesSelfTest.java      |   55 +-
 .../processors/igfs/IgfsOneClientNodeTest.java  |   17 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |   29 +-
 .../igfs/IgfsProcessorValidationSelfTest.java   |  324 +-
 ...gfsSecondaryFileSystemInjectionSelfTest.java |  268 ++
 ...IpcEndpointRegistrationAbstractSelfTest.java |   21 +-
 .../processors/igfs/IgfsSizeSelfTest.java       |   31 +-
 .../processors/igfs/IgfsStartCacheTest.java     |   30 +-
 .../processors/igfs/IgfsStreamsSelfTest.java    |   50 +-
 .../processors/igfs/IgfsTaskSelfTest.java       |   10 +-
 .../IgfsAbstractRecordResolverSelfTest.java     |   10 +-
 ...niteMessagingConfigVariationFullApiTest.java |    6 +-
 .../odbc/OdbcProcessorValidationSelfTest.java   |    2 +-
 .../service/ClosureServiceClientsNodesTest.java |   10 +-
 .../service/GridServiceClientNodeTest.java      |    4 +-
 .../GridServiceContinuousQueryRedeploy.java     |  167 --
 .../GridServiceContinuousQueryRedeployTest.java |  167 ++
 .../GridServiceProcessorAbstractSelfTest.java   |    4 +-
 ...GridServiceProxyClientReconnectSelfTest.java |    6 +-
 .../GridServiceProxyNodeStopSelfTest.java       |    4 +-
 .../GridServiceSerializationSelfTest.java       |    6 +-
 ...gniteServiceConfigVariationsFullApiTest.java |    4 +-
 ...yment2ClassLoadersDefaultMarshallerTest.java |   18 +-
 ...oymentClassLoadingDefaultMarshallerTest.java |   16 +-
 .../IgniteServiceDynamicCachesSelfTest.java     |    4 +-
 ...gniteServiceProxyTimeoutInitializedTest.java |    6 +-
 .../service/IgniteServiceReassignmentTest.java  |    4 +-
 .../ServicePredicateAccessCacheTest.java        |    4 +-
 .../internal/util/IgniteUtilsSelfTest.java      |   66 +
 .../nio/GridNioEmbeddedFutureSelfTest.java      |    2 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |   25 +-
 .../ipc/shmem/IpcSharedMemoryNodeStartup.java   |   30 +-
 .../ipc/shmem/IpcSharedMemorySpaceSelfTest.java |    6 +-
 .../ipc/shmem/IpcSharedMemoryUtilsSelfTest.java |    2 +-
 .../internal/util/nio/GridNioSelfTest.java      |    2 +-
 .../internal/util/nio/GridNioSslSelfTest.java   |    2 +-
 .../nio/IgniteExceptionInNioWorkerSelfTest.java |    6 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   12 +-
 .../loadtests/GridCacheMultiNodeLoadTest.java   |    4 +-
 .../communication/GridIoManagerBenchmark.java   |    4 +-
 .../communication/GridIoManagerBenchmark0.java  |   18 +-
 .../GridTcpCommunicationBenchmark.java          |    2 +-
 .../GridCachePartitionedAtomicLongLoadTest.java |    4 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |    4 +-
 .../GridMultiSplitsRedeployLoadTest.java        |    2 +-
 ...ridSingleSplitsNewNodesAbstractLoadTest.java |    8 +-
 .../GridSingleSplitsRedeployLoadTest.java       |    2 +-
 .../direct/session/GridSessionLoadTest.java     |    2 +-
 .../direct/stealing/GridStealingLoadTest.java   |    4 +-
 .../ignite/loadtests/job/GridJobLoadTest.java   |    6 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |    2 +-
 ...namicProxySerializationMultiJvmSelfTest.java |    4 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   15 +-
 .../marshaller/MarshallerContextSelfTest.java   |  150 +-
 .../marshaller/MarshallerContextTestImpl.java   |   33 +-
 .../OptimizedMarshallerEnumSelfTest.java        |   25 +-
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    4 +-
 .../ignite/messaging/GridMessagingSelfTest.java |    4 +-
 .../IgniteMessagingWithClientTest.java          |    6 +-
 .../p2p/GridAbstractMultinodeRedeployTest.java  |    4 +-
 .../GridP2PContinuousDeploymentSelfTest.java    |   18 +-
 .../GridP2PDifferentClassLoaderSelfTest.java    |    4 +-
 .../p2p/GridP2PDoubleDeploymentSelfTest.java    |    4 +-
 .../p2p/GridP2PHotRedeploymentSelfTest.java     |    4 +-
 .../p2p/GridP2PJobClassLoaderSelfTest.java      |    4 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |    4 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |    4 +-
 .../ignite/p2p/GridP2PNodeLeftSelfTest.java     |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    4 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |    4 +-
 .../p2p/GridP2PSameClassLoaderSelfTest.java     |    4 +-
 .../ignite/p2p/GridP2PTimeoutSelfTest.java      |    4 +-
 .../ignite/p2p/GridP2PUndeploySelfTest.java     |    6 +-
 .../platform/PlatformComputeEchoTask.java       |   18 +
 .../PlatformJavaObjectFactoryProxySelfTest.java |    2 +-
 .../ignite/platform/PlatformStopIgniteTask.java |   10 +-
 .../platform/plugin/PlatformTestPlugin.java     |   27 +
 .../plugin/PlatformTestPluginConfiguration.java |   63 +
 ...rmTestPluginConfigurationClosureFactory.java |   61 +
 .../plugin/PlatformTestPluginException.java     |   34 +
 .../plugin/PlatformTestPluginExtension.java     |   51 +
 .../plugin/PlatformTestPluginProvider.java      |   97 +
 .../plugin/PlatformTestPluginTarget.java        |  231 ++
 .../cache/PlatformGetCachePluginsTask.java      |   85 +
 .../PlatformTestCachePluginConfiguration.java   |   60 +
 ...formTestCachePluginConfigurationClosure.java |   48 +
 ...tCachePluginConfigurationClosureFactory.java |   37 +
 .../cache/PlatformTestCachePluginProvider.java  |   73 +
 ...SessionCancelSiblingsFromFutureSelfTest.java |    6 +-
 ...ridSessionCancelSiblingsFromJobSelfTest.java |    6 +-
 ...idSessionCancelSiblingsFromTaskSelfTest.java |    6 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |    2 +-
 .../GridSessionCollisionSpiSelfTest.java        |    2 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |    6 +-
 ...dSessionFutureWaitTaskAttributeSelfTest.java |    6 +-
 .../session/GridSessionJobFailoverSelfTest.java |    4 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |    6 +-
 .../ignite/session/GridSessionLoadSelfTest.java |    4 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |    6 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |    6 +-
 .../GridSessionSetJobAttributeSelfTest.java     |    6 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |    6 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |    6 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |    6 +-
 .../GridSessionWaitAttributeSelfTest.java       |    8 +-
 .../spi/GridSpiStartStopAbstractTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   10 +-
 .../CacheCheckpointSpiSecondCacheSelfTest.java  |    4 +-
 .../GridAbstractCommunicationSelfTest.java      |    4 +-
 .../communication/GridCacheMessageSelfTest.java |    6 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |    4 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |    4 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |    4 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    2 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |    2 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    4 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    4 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |    4 +-
 .../tcp/IgniteCacheSslStartStopSelfTest.java    |    4 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    4 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    9 +-
 ...gniteClientReconnectMassiveShutdownTest.java |    4 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |    6 +-
 .../tcp/TcpClientDiscoverySpiMulticastTest.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   16 +-
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |    4 +-
 .../TcpDiscoveryMarshallerCheckSelfTest.java    |    4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |    8 +-
 ...DiscoveryNodeConfigConsistentIdSelfTest.java |    6 +-
 .../TcpDiscoveryNodeConsistentIdSelfTest.java   |    4 +-
 .../discovery/tcp/TcpDiscoveryRestartTest.java  |    4 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  139 +-
 .../tcp/TcpDiscoverySnapshotHistoryTest.java    |    4 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    2 +-
 .../tcp/TcpDiscoverySpiStartStopSelfTest.java   |   11 +-
 .../tcp/TcpDiscoverySpiWildcardSelfTest.java    |    4 +-
 .../discovery/tcp/TcpDiscoverySslSelfTest.java  |    4 +-
 .../vm/TcpDiscoveryVmIpFinderSelfTest.java      |    2 +-
 .../spi/failover/GridFailoverTestContext.java   |    7 +-
 .../GridInternalTasksLoadBalancingSelfTest.java |    6 +-
 .../GridSwapSpaceSpiConsistencySelfTest.java    |   12 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |    2 +-
 .../inmemory/GridTestSwapSpaceSpi.java          |    2 +-
 .../noop/GridNoopSwapSpaceSpiSelfTest.java      |    4 +-
 .../ignite/startup/GridVmNodesStarter.java      |    4 +-
 .../stream/socket/SocketStreamerSelfTest.java   |    4 +-
 .../ignite/testframework/GridTestNode.java      |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   10 +-
 .../configvariations/ConfigFactory.java         |    8 +-
 .../ConfigVariationsFactory.java                |    6 +-
 .../testframework/junits/GridAbstractTest.java  |  213 +-
 .../junits/GridTestKernalContext.java           |   12 +-
 ...IgniteCacheConfigVariationsAbstractTest.java |   12 +-
 .../IgniteConfigVariationsAbstractTest.java     |   12 +-
 .../junits/common/GridAbstractExamplesTest.java |    2 +-
 .../junits/common/GridCommonAbstractTest.java   |   15 +-
 .../common/GridRollingRestartAbstractTest.java  |    6 +-
 .../junits/multijvm/IgniteNodeRunner.java       |   55 +-
 .../junits/multijvm/IgniteProcessProxy.java     |  109 +-
 .../junits/spi/GridSpiAbstractConfigTest.java   |    2 +-
 .../junits/spi/GridSpiAbstractTest.java         |   13 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../testsuites/IgniteBinaryBasicTestSuite.java  |    5 +
 .../IgniteBinaryObjectsTestSuite.java           |    2 +
 .../IgniteCacheFullApiSelfTestSuite.java        |    8 +-
 .../IgniteCacheRestartTestSuite2.java           |    3 +-
 .../testsuites/IgniteCacheTestSuite5.java       |    3 +
 .../IgniteCacheTxRecoverySelfTestSuite.java     |    3 +
 .../ignite/testsuites/IgniteIgfsTestSuite.java  |    3 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    4 +-
 .../ignite/thread/IgniteThreadPoolSizeTest.java |    8 -
 .../util/AttributeNodeFilterSelfTest.java       |    4 +-
 .../apache/ignite/util/GridRandomSelfTest.java  |   17 -
 .../util/GridTopologyHeapSizeSelfTest.java      |    4 +-
 .../h2/GridBinaryH2IndexingGeoSelfTest.java     |    4 +-
 modules/hadoop/pom.xml                          |    7 +
 .../fs/IgniteHadoopFileSystemCounterWriter.java |    6 +-
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java |   23 +-
 .../mapreduce/IgniteHadoopMapReducePlanner.java |  416 ---
 .../IgniteHadoopWeightedMapReducePlanner.java   |   15 +-
 .../planner/HadoopAbstractMapReducePlanner.java |  118 +
 .../planner/HadoopTestRoundRobinMrPlanner.java  |   75 +
 .../processors/hadoop/HadoopCommonUtils.java    |    1 +
 .../processors/hadoop/HadoopContext.java        |    2 +
 .../processors/hadoop/HadoopExternalSplit.java  |    1 +
 .../processors/hadoop/HadoopProcessor.java      |    4 +-
 .../processors/hadoop/HadoopSplitWrapper.java   |    1 +
 .../HadoopFileSystemCounterWriterDelegate.java  |    4 +-
 .../HadoopIgfsSecondaryFileSystemDelegate.java  |    4 +-
 .../hadoop/igfs/HadoopIgfsEndpoint.java         |   38 +-
 ...doopFileSystemCounterWriterDelegateImpl.java |    4 +-
 ...doopIgfsSecondaryFileSystemDelegateImpl.java |   61 +-
 .../hadoop/impl/igfs/HadoopIgfsInProc.java      |  190 +-
 .../hadoop/impl/igfs/HadoopIgfsOutProc.java     |   24 +-
 .../hadoop/impl/igfs/HadoopIgfsUtils.java       |    6 +
 .../hadoop/impl/igfs/HadoopIgfsWrapper.java     |  104 +-
 .../hadoop/impl/v1/HadoopV1MapTask.java         |    6 +-
 .../hadoop/impl/v1/HadoopV1ReduceTask.java      |    4 +-
 .../hadoop/impl/v1/HadoopV1Splitter.java        |    2 +-
 .../hadoop/impl/v2/HadoopV2Context.java         |    3 +-
 .../processors/hadoop/impl/v2/HadoopV2Job.java  |   25 +-
 .../hadoop/impl/v2/HadoopV2Splitter.java        |    2 +-
 .../hadoop/impl/v2/HadoopV2TaskContext.java     |    6 +-
 .../hadoop/jobtracker/HadoopJobMetadata.java    |    4 +-
 .../hadoop/jobtracker/HadoopJobTracker.java     |   32 +-
 .../planner/HadoopAbstractMapReducePlanner.java |  116 -
 .../planner/HadoopDefaultMapReducePlan.java     |    4 +-
 .../hadoop/shuffle/HadoopShuffle.java           |    9 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        |   13 +-
 .../shuffle/direct/HadoopDirectDataInput.java   |   75 +-
 .../shuffle/streams/HadoopDataInStream.java     |   34 +-
 .../shuffle/streams/HadoopOffheapBuffer.java    |   18 +
 .../HadoopEmbeddedTaskExecutor.java             |   10 +-
 .../taskexecutor/HadoopExecutorService.java     |   10 +-
 .../hadoop/taskexecutor/HadoopRunnableTask.java |    6 +-
 .../taskexecutor/HadoopTaskExecutorAdapter.java |    8 +-
 .../external/HadoopExternalTaskExecutor.java    |   18 +-
 .../child/HadoopChildProcessRunner.java         |    4 +-
 .../HadoopExternalCommunication.java            |   29 +-
 .../communication/HadoopIpcToNioAdapter.java    |    7 +-
 .../communication/HadoopMarshallerFilter.java   |   10 +-
 .../resources/META-INF/classnames.properties    |    4 +-
 .../test/config/hadoop-fs-open-test/grid-0.xml  |  126 +
 .../test/config/hadoop-fs-open-test/grid-1.xml  |  126 +
 .../test/config/hadoop-fs-open-test/grid-2.xml  |  126 +
 .../test/config/igfs-cli-config-dual-async.xml  |  135 +
 .../test/config/igfs-cli-config-dual-sync.xml   |  133 +
 .../src/test/config/igfs-cli-config-primary.xml |  124 +
 .../src/test/config/igfs-cli-config-proxy.xml   |  133 +
 .../impl/HadoopAbstractMapReduceTest.java       |   14 +-
 .../hadoop/impl/HadoopAbstractSelfTest.java     |   31 +-
 .../hadoop/impl/HadoopCommandLineTest.java      |    4 +-
 .../HadoopDefaultMapReducePlannerSelfTest.java  |  619 ----
 .../hadoop/impl/HadoopGroupingTest.java         |    4 +-
 .../hadoop/impl/HadoopJobTrackerSelfTest.java   |    5 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |    4 +-
 .../impl/HadoopNoHadoopMapReduceTest.java       |    4 +-
 .../hadoop/impl/HadoopPlannerMockJob.java       |   10 +-
 .../hadoop/impl/HadoopSortingExternalTest.java  |    8 +-
 .../hadoop/impl/HadoopSortingTest.java          |    4 +-
 .../impl/HadoopTaskExecutionSelfTest.java       |   23 +-
 .../hadoop/impl/HadoopTasksAllVersionsTest.java |   16 +-
 .../hadoop/impl/HadoopTasksV1Test.java          |    4 +-
 .../hadoop/impl/HadoopTasksV2Test.java          |    4 +-
 .../hadoop/impl/HadoopTeraSortTest.java         |    4 +-
 .../impl/HadoopTestRoundRobinMrPlanner.java     |   75 -
 .../hadoop/impl/HadoopTestTaskContext.java      |    6 +-
 .../hadoop/impl/HadoopV2JobSelfTest.java        |    6 +-
 .../hadoop/impl/HadoopValidationSelfTest.java   |    4 +-
 .../HadoopWeightedMapReducePlannerTest.java     |    6 +-
 .../HadoopClientProtocolEmbeddedSelfTest.java   |    4 +-
 ...opClientProtocolMultipleServersSelfTest.java |    4 +-
 .../client/HadoopClientProtocolSelfTest.java    |    2 +-
 .../impl/igfs/Hadoop1DualAbstractTest.java      |   13 +-
 .../impl/igfs/Hadoop1OverIgfsProxyTest.java     |   67 +
 .../igfs/HadoopFIleSystemFactorySelfTest.java   |   16 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   74 +-
 ...Igfs20FileSystemLoopbackPrimarySelfTest.java |    6 +-
 ...oopIgfs20FileSystemShmemPrimarySelfTest.java |    6 +-
 .../igfs/HadoopIgfsDualAbstractSelfTest.java    |   20 +-
 ...oopSecondaryFileSystemConfigurationTest.java |   95 +-
 .../hadoop/impl/igfs/IgfsEventsTestSuite.java   |    4 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |   14 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  208 +-
 ...opFileSystemClientBasedAbstractSelfTest.java |  193 ++
 ...pFileSystemClientBasedDualAsyncSelfTest.java |   38 +
 ...opFileSystemClientBasedDualSyncSelfTest.java |   38 +
 ...niteHadoopFileSystemClientBasedOpenTest.java |  305 ++
 ...oopFileSystemClientBasedPrimarySelfTest.java |   38 +
 ...adoopFileSystemClientBasedProxySelfTest.java |   37 +
 .../IgniteHadoopFileSystemClientSelfTest.java   |   62 +-
 ...IgniteHadoopFileSystemHandshakeSelfTest.java |  111 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   53 +-
 ...niteHadoopFileSystemLoggerStateSelfTest.java |   30 +-
 ...adoopFileSystemLoopbackAbstractSelfTest.java |    4 +-
 ...oopbackExternalToClientAbstractSelfTest.java |   61 +
 ...opbackExternalToClientDualAsyncSelfTest.java |   33 +
 ...oopbackExternalToClientDualSyncSelfTest.java |   33 +
 ...LoopbackExternalToClientPrimarySelfTest.java |   33 +
 ...emLoopbackExternalToClientProxySelfTest.java |   33 +
 ...condaryFileSystemInitializationSelfTest.java |   38 +-
 ...teHadoopFileSystemShmemAbstractSelfTest.java |    4 +-
 ...ileSystemShmemEmbeddedDualAsyncSelfTest.java |   33 -
 ...FileSystemShmemEmbeddedDualSyncSelfTest.java |   33 -
 ...pFileSystemShmemEmbeddedPrimarySelfTest.java |   33 -
 ...ileSystemShmemEmbeddedSecondarySelfTest.java |   33 -
 ...emShmemExternalToClientAbstractSelfTest.java |  106 +
 ...mShmemExternalToClientDualAsyncSelfTest.java |   33 +
 ...emShmemExternalToClientDualSyncSelfTest.java |   33 +
 ...temShmemExternalToClientPrimarySelfTest.java |   33 +
 ...ystemShmemExternalToClientProxySelfTest.java |   33 +
 .../collections/HadoopAbstractMapTest.java      |    4 +-
 .../streams/HadoopDataStreamSelfTest.java       |  177 +-
 .../HadoopExternalTaskExecutionSelfTest.java    |   16 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   25 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   17 +-
 .../HibernateReadWriteAccessStrategy.java       |   12 +-
 .../cache/hibernate/HibernateRegionFactory.java |   25 +-
 .../src/test/config/factory-cache1.xml          |    2 +-
 .../HibernateL2CacheConfigurationSelfTest.java  |   20 +-
 .../hibernate/HibernateL2CacheSelfTest.java     |   24 +-
 .../HibernateL2CacheTransactionalSelfTest.java  |    4 +-
 .../query/h2/DmlStatementsProcessor.java        |  100 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   45 +-
 .../query/h2/dml/UpdatePlanBuilder.java         |  109 +-
 .../query/h2/opt/GridH2IndexBase.java           |   10 -
 .../processors/query/h2/opt/GridH2Table.java    |   33 -
 .../query/h2/opt/GridH2TreeIndex.java           |   25 -
 .../query/h2/opt/GridLuceneIndex.java           |    4 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  |    8 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    7 +-
 ...CacheAbstractQueryDetailMetricsSelfTest.java |    4 +-
 .../CacheAbstractQueryMetricsSelfTest.java      |    4 +-
 .../CacheBinaryKeyConcurrentQueryTest.java      |    4 +-
 .../cache/CacheIndexStreamerTest.java           |    4 +-
 .../cache/CacheIndexingOffheapCleanupTest.java  |   21 -
 .../CacheOffheapBatchIndexingBaseTest.java      |    4 +-
 .../cache/CacheQueryBuildValueTest.java         |    4 +-
 .../cache/CacheQueryNewClientSelfTest.java      |    4 +-
 .../CacheRandomOperationsMultithreadedTest.java |    4 +-
 ...CacheScanPartitionQueryFallbackSelfTest.java |    6 +-
 .../cache/CacheSqlQueryValueCopySelfTest.java   |    6 +-
 .../ClientReconnectAfterClusterRestartTest.java |    6 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |    4 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |    4 +-
 .../cache/GridCacheOffHeapSelfTest.java         |    4 +-
 .../GridCacheOffheapIndexEntryEvictTest.java    |    4 +-
 .../cache/GridCacheOffheapIndexGetSelfTest.java |    4 +-
 .../GridCacheQueryIndexDisabledSelfTest.java    |    4 +-
 .../GridCacheQuerySerializationSelfTest.java    |    4 +-
 .../cache/GridCacheQuerySimpleBenchmark.java    |    4 +-
 .../processors/cache/GridCacheSwapSelfTest.java |    4 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |    4 +-
 .../IgniteBinaryObjectFieldsQuerySelfTest.java  |    6 +-
 .../IgniteBinaryObjectQueryArgumentsTest.java   |    4 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java |   25 +-
 ...niteCacheAbstractInsertSqlQuerySelfTest.java |    8 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  103 +-
 .../IgniteCacheAbstractSqlDmlQuerySelfTest.java |    6 +-
 .../IgniteCacheCollocatedQuerySelfTest.java     |    4 +-
 ...acheConfigurationPrimitiveTypesSelfTest.java |    4 +-
 .../IgniteCacheCrossCacheJoinRandomTest.java    |    4 +-
 ...acheDistributedJoinCollocatedAndNotTest.java |    4 +-
 ...acheDistributedJoinCustomAffinityMapper.java |    4 +-
 .../IgniteCacheDistributedJoinNoIndexTest.java  |    4 +-
 ...ributedJoinPartitionedAndReplicatedTest.java |    4 +-
 ...CacheDistributedJoinQueryConditionsTest.java |    4 +-
 .../cache/IgniteCacheDistributedJoinTest.java   |    4 +-
 ...cheDuplicateEntityConfigurationSelfTest.java |    4 +-
 .../IgniteCacheFieldsQueryNoDataSelfTest.java   |    4 +-
 ...teCacheFullTextQueryNodeJoiningSelfTest.java |    4 +-
 .../IgniteCacheInsertSqlQuerySelfTest.java      |   26 +-
 ...PartitionedAndReplicatedCollocationTest.java |    4 +-
 ...teCacheJoinPartitionedAndReplicatedTest.java |    4 +-
 ...IgniteCacheJoinQueryWithAffinityKeyTest.java |    4 +-
 .../cache/IgniteCacheLargeResultSelfTest.java   |    4 +-
 ...eLockPartitionOnAffinityRunAbstractTest.java |    4 +-
 ...PartitionOnAffinityRunAtomicCacheOpTest.java |    4 +-
 ...titionOnAffinityRunWithCollisionSpiTest.java |    6 +-
 .../cache/IgniteCacheMergeSqlQuerySelfTest.java |   24 +
 .../cache/IgniteCacheNoClassQuerySelfTest.java  |    4 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |    4 +-
 .../cache/IgniteCacheOffheapIndexScanTest.java  |    4 +-
 ...CacheOffheapTieredMultithreadedSelfTest.java |    4 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |    4 +-
 ...IgniteCachePrimitiveFieldsQuerySelfTest.java |    4 +-
 .../cache/IgniteCacheQueriesLoadTest1.java      |    4 +-
 .../IgniteCacheQueryH2IndexingLeakTest.java     |    4 +-
 .../cache/IgniteCacheQueryLoadSelfTest.java     |    8 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    4 +-
 ...gniteCacheSqlQueryMultiThreadedSelfTest.java |    4 +-
 .../IgniteCacheStarvationOnRebalanceTest.java   |    4 +-
 .../IgniteCacheUpdateSqlQuerySelfTest.java      |   65 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |    4 +-
 .../cache/IgniteClientReconnectQueriesTest.java |    4 +-
 .../cache/IgniteCrossCachesJoinsQueryTest.java  |    4 +-
 .../cache/IncorrectCacheTypeMetadataTest.java   |    4 +-
 .../cache/IncorrectQueryEntityTest.java         |    4 +-
 .../cache/SqlFieldsQuerySelfTest.java           |    4 +-
 ...lientQueryReplicatedNodeRestartSelfTest.java |   10 +-
 ...niteCacheDistributedQueryCancelSelfTest.java |    6 +-
 ...butedQueryStopOnCancelOrTimeoutSelfTest.java |    6 +-
 ...artitionedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...achePartitionedQueryP2PDisabledSelfTest.java |    4 +-
 ...cheQueryAbstractDistributedJoinSelfTest.java |    6 +-
 .../near/IgniteCacheQueryNodeFailTest.java      |    4 +-
 .../IgniteCacheQueryNodeRestartSelfTest.java    |    4 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   |    4 +-
 ...ReplicatedFieldsQueryP2PEnabledSelfTest.java |    4 +-
 ...CacheReplicatedQueryP2PDisabledSelfTest.java |    4 +-
 ...eCacheLocalQueryCancelOrTimeoutSelfTest.java |    4 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java     |    4 +-
 .../query/IgniteSqlQueryDedicatedPoolTest.java  |  110 +
 .../query/IgniteSqlSplitterSelfTest.java        |    4 +-
 .../query/h2/GridH2IndexRebuildTest.java        |  261 --
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   35 +-
 .../query/h2/opt/GridH2TableSelfTest.java       |   54 -
 .../h2/sql/AbstractH2CompareQueryTest.java      |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    4 +-
 .../FetchingQueryCursorStressTest.java          |    2 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |    4 +-
 .../IgniteCacheQuerySelfTestSuite.java          |    4 +-
 .../processors/cache/jta/CacheJtaManager.java   |    3 +-
 .../processors/cache/jta/CacheJtaResource.java  |    8 +-
 .../cache/AbstractCacheJtaSelfTest.java         |    8 +-
 ...CacheJtaConfigurationValidationSelfTest.java |    4 +-
 ...CacheJtaFactoryConfigValidationSelfTest.java |    4 +-
 .../cache/GridJtaLifecycleAwareSelfTest.java    |    4 +-
 .../log4j/GridLog4jCorrectFileNameTest.java     |    6 +-
 .../log4j2/GridLog4j2CorrectFileNameTest.java   |    6 +-
 .../log4j2/GridLog4j2InitializedTest.java       |    2 +-
 .../logger/log4j2/Log4j2LoggerSelfTest.java     |    6 +-
 .../log4j2/Log4j2LoggerVerboseModeSelfTest.java |    6 +-
 .../IgniteAbstractOsgiContextActivator.java     |    2 +-
 .../activators/BasicIgniteTestActivator.java    |    2 +-
 .../include/ignite/binary/binary_object.h       |   22 +-
 .../binary/include/ignite/binary/binary_type.h  |    8 +-
 .../include/ignite/impl/binary/binary_common.h  |    6 +
 .../ignite/impl/binary/binary_object_header.h   |    9 +-
 .../ignite/impl/binary/binary_object_impl.h     |   61 +-
 .../ignite/impl/binary/binary_reader_impl.h     |   29 +-
 .../ignite/impl/binary/binary_type_updater.h    |    2 +-
 .../binary/binary_array_identity_resolver.cpp   |    2 +-
 .../src/impl/binary/binary_object_header.cpp    |   26 +-
 .../src/impl/binary/binary_object_impl.cpp      |   61 +-
 .../src/impl/binary/binary_type_manager.cpp     |    2 +-
 modules/platforms/cpp/common/Makefile.am        |    1 +
 .../common/include/ignite/common/concurrent.h   |   29 +-
 .../include/ignite/common/platform_utils.h      |    6 +-
 .../cpp/common/include/ignite/ignite_error.h    |    2 +-
 .../include/ignite/common/dynamic_load_os.h     |  131 +
 .../os/linux/src/common/dynamic_load_os.cpp     |   90 +
 .../os/linux/src/common/platform_utils.cpp      |   18 +-
 .../win/include/ignite/common/dynamic_load_os.h |  133 +
 .../os/win/src/common/dynamic_load_os.cpp       |  115 +
 .../common/os/win/src/common/platform_utils.cpp |   26 +-
 .../cpp/common/project/vs/common.vcxproj        |    2 +
 .../common/project/vs/common.vcxproj.filters    |    6 +
 .../platforms/cpp/common/src/common/utils.cpp   |    3 +-
 .../platforms/cpp/common/src/ignite_error.cpp   |   68 +-
 modules/platforms/cpp/core-test/Makefile.am     |    5 +-
 .../core-test/include/ignite/binary_test_defs.h |  106 +-
 .../cpp/core-test/project/vs/core-test.vcxproj  |    2 +
 .../project/vs/core-test.vcxproj.filters        |    6 +
 .../cpp/core-test/src/binary_object_test.cpp    |    6 +-
 .../src/binary_reader_writer_raw_test.cpp       |   23 +-
 .../cpp/core-test/src/cache_invoke_test.cpp     |  553 ++++
 .../platforms/cpp/core-test/src/cache_test.cpp  |    5 +-
 .../cpp/core-test/src/cluster_test.cpp          |   98 +
 .../cpp/core-test/src/interop_memory_test.cpp   |    3 +-
 .../platforms/cpp/core-test/src/test_utils.cpp  |    6 +-
 modules/platforms/cpp/core/Makefile.am          |    1 +
 modules/platforms/cpp/core/include/Makefile.am  |   10 +-
 .../cpp/core/include/ignite/cache/cache.h       |  217 +-
 .../ignite/cache/cache_entry_processor.h        |  111 +
 .../include/ignite/cache/mutable_cache_entry.h  |  176 ++
 .../include/ignite/cache/query/query_argument.h |  134 -
 .../include/ignite/cache/query/query_cursor.h   |    2 +-
 .../core/include/ignite/cache/query/query_sql.h |   16 +-
 .../ignite/cache/query/query_sql_fields.h       |   16 +-
 .../platforms/cpp/core/include/ignite/ignite.h  |   18 +
 .../cpp/core/include/ignite/ignite_binding.h    |  119 +
 .../include/ignite/ignite_binding_context.h     |   88 +
 .../core/include/ignite/ignite_configuration.h  |    2 -
 .../impl/binary/binary_type_updater_impl.h      |    2 +-
 .../impl/cache/cache_entry_processor_holder.h   |  282 ++
 .../core/include/ignite/impl/cache/cache_impl.h |   71 +-
 .../ignite/impl/cache/query/query_argument.h    |  137 +
 .../ignite/impl/cluster/cluster_group_impl.h    |   77 +
 .../include/ignite/impl/ignite_binding_impl.h   |  121 +
 .../include/ignite/impl/ignite_environment.h    |   61 +-
 .../cpp/core/include/ignite/impl/ignite_impl.h  |   50 +-
 .../ignite/impl/interop/interop_target.h        |   39 +-
 .../core/include/ignite/impl/module_manager.h   |  131 +
 .../cpp/core/include/ignite/impl/operations.h   |   62 +-
 .../platforms/cpp/core/project/vs/core.vcxproj  |   11 +-
 .../cpp/core/project/vs/core.vcxproj.filters    |   36 +-
 modules/platforms/cpp/core/src/ignite.cpp       |   12 +-
 modules/platforms/cpp/core/src/ignition.cpp     |  343 ++-
 .../impl/binary/binary_type_updater_impl.cpp    |    2 +-
 .../cpp/core/src/impl/cache/cache_impl.cpp      |   74 +-
 .../continuous/continuous_query_handle_impl.cpp |    2 +-
 .../core/src/impl/cache/query/query_impl.cpp    |    8 +-
 .../src/impl/cluster/cluster_group_impl.cpp     |   64 +
 .../cpp/core/src/impl/ignite_environment.cpp    |   92 +-
 .../platforms/cpp/core/src/impl/ignite_impl.cpp |   32 +-
 .../core/src/impl/interop/interop_target.cpp    |   51 +-
 .../src/impl/transactions/transactions_impl.cpp |   14 +-
 .../cpp/jni/include/ignite/jni/exports.h        |    2 +
 .../platforms/cpp/jni/include/ignite/jni/java.h |    6 +-
 .../cpp/jni/include/ignite/jni/utils.h          |   78 +-
 .../platforms/cpp/jni/os/linux/src/utils.cpp    |  227 +-
 modules/platforms/cpp/jni/os/win/src/utils.cpp  |  233 +-
 modules/platforms/cpp/jni/project/vs/module.def |    2 +
 modules/platforms/cpp/jni/src/exports.cpp       |    8 +
 modules/platforms/cpp/jni/src/java.cpp          |   76 +-
 .../cpp/odbc/install/install_amd64.cmd          |   10 +-
 .../platforms/cpp/odbc/install/install_x86.cmd  |    6 +-
 modules/platforms/cpp/odbc/src/utility.cpp      |   17 +-
 .../Apache.Ignite.AspNet.Tests/App.config       |    6 +-
 .../IgniteOutputCacheProviderTest.cs            |    2 +-
 .../IgniteSessionStateStoreProviderTest.cs      |    2 +-
 .../Apache.Ignite.AspNet/Impl/ConfigUtil.cs     |    4 +-
 .../Apache.Ignite.Core.Tests.csproj             |   10 +
 .../Binary/BinaryBuilderSelfTest.cs             |   22 +-
 .../Binary/BinaryReaderWriterTest.cs            |    8 +
 .../Cache/Affinity/AffinityFieldTest.cs         |    2 +-
 .../Cache/Affinity/AffinityFunctionTest.cs      |    4 +-
 .../Cache/Affinity/AffinityTest.cs              |    2 +-
 .../Cache/CacheAbstractTest.cs                  |    2 +-
 .../Cache/CacheConfigurationTest.cs             |   15 +-
 .../Cache/CacheDynamicStartTest.cs              |    2 +-
 .../Cache/CacheMetricsTest.cs                   |    2 +-
 .../Cache/CacheNearTest.cs                      |    4 +-
 .../Cache/Query/CacheLinqTest.cs                |    2 +-
 .../Cache/Query/CacheQueriesTest.cs             |    2 +-
 .../Continuous/ContinuousQueryAbstractTest.cs   |    4 +-
 .../Continuous/ContinuousQueryJavaFilterTest.cs |    2 +-
 .../Cache/Store/CacheStoreSessionTest.cs        |    2 +-
 .../Cache/Store/CacheStoreTest.cs               |    2 +-
 .../Compute/ComputeApiTest.cs                   |   87 +-
 .../Cache/Affinity/affinity-function2.xml       |    2 +-
 .../Config/Compute/compute-grid1.xml            |    2 +-
 .../Config/Compute/compute-grid2.xml            |    2 +-
 .../Config/Compute/compute-grid3.xml            |    2 +-
 .../Config/Lifecycle/lifecycle-beans.xml        |    2 +-
 .../Config/Lifecycle/lifecycle-no-beans.xml     |    2 +-
 .../Config/start-test-grid1.xml                 |    2 +-
 .../Config/start-test-grid2.xml                 |    2 +-
 .../ConsoleRedirectTest.cs                      |    4 +-
 .../Dataload/DataStreamerTest.cs                |    2 +-
 .../Dataload/DataStreamerTestTopologyChange.cs  |    2 +-
 .../Apache.Ignite.Core.Tests/ExceptionsTest.cs  |   30 +-
 .../IgniteConfigurationSectionTest.cs           |    2 +-
 .../IgniteConfigurationSerializerTest.cs        |   44 +-
 .../IgniteConfigurationTest.cs                  |   23 +-
 .../Log/CustomLoggerTest.cs                     |    2 +-
 .../Cache/CacheJavaPluginConfiguration.cs       |   45 +
 .../Plugin/Cache/CacheJavaPluginTest.cs         |  113 +
 .../Plugin/Cache/CachePlugin.cs                 |  127 +
 .../Plugin/Cache/CachePluginConfiguration.cs    |   64 +
 .../Plugin/Cache/CachePluginTest.cs             |  218 ++
 .../Plugin/PluginTest.cs                        |  342 +++
 .../Plugin/TestIgnitePlugin.cs                  |   39 +
 .../Plugin/TestIgnitePluginConfiguration.cs     |   53 +
 .../Plugin/TestIgnitePluginException.cs         |   64 +
 .../Plugin/TestIgnitePluginProvider.cs          |  153 +
 .../Apache.Ignite.Core.Tests/ReconnectTest.cs   |    2 +-
 .../SerializationTest.cs                        |    2 +-
 .../dotnet/Apache.Ignite.Core.Tests/app.config  |    4 +-
 .../Apache.Ignite.Core.Tests/custom_app.config  |    2 +-
 .../Apache.Ignite.Core.csproj                   |   20 +
 .../Cache/Configuration/CacheConfiguration.cs   |   46 +
 .../Cache/Configuration/QueryEntity.cs          |    2 +-
 .../Common/ExceptionFactory.cs                  |   31 +
 .../Apache.Ignite.Core/Common/IgniteGuid.cs     |   47 +-
 .../Apache.Ignite.Core/Common/JavaException.cs  |   62 +-
 .../Communication/Tcp/TcpCommunicationSpi.cs    |    2 +-
 .../Discovery/Tcp/TcpDiscoverySpi.cs            |    2 +-
 .../Apache.Ignite.Core/Events/CacheEvent.cs     |    2 +-
 .../Apache.Ignite.Core/Events/EventBase.cs      |    4 +-
 .../Apache.Ignite.Core/Events/JobEvent.cs       |    4 +-
 .../Apache.Ignite.Core/Events/TaskEvent.cs      |    2 +-
 .../dotnet/Apache.Ignite.Core/IIgnite.cs        |   10 +
 .../Apache.Ignite.Core/IgniteConfiguration.cs   |   78 +-
 .../IgniteConfigurationSection.xsd              |   49 +-
 .../dotnet/Apache.Ignite.Core/Ignition.cs       |    2 +-
 .../Impl/Binary/BinaryObjectBuilder.cs          |    2 +-
 .../Impl/Binary/BinaryReader.cs                 |    4 +-
 .../Impl/Binary/BinarySystemHandlers.cs         |   16 +-
 .../Impl/Binary/BinaryUtils.cs                  |   55 +-
 .../Impl/Binary/BinaryWriter.cs                 |    6 +-
 .../Impl/Binary/Marshaller.cs                   |    4 +-
 .../Common/IgniteConfigurationXmlSerializer.cs  |   40 +-
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs   |   96 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs    |   26 +-
 .../Apache.Ignite.Core/Impl/IgniteProxy.cs      |    6 +
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   99 +-
 .../Impl/Plugin/Cache/CachePluginContext.cs     |   82 +
 .../Impl/Plugin/Cache/CachePluginProcessor.cs   |   77 +
 .../Plugin/Cache/CachePluginProviderProxy.cs    |   75 +
 .../Plugin/Cache/ICachePluginProviderProxy.cs   |   52 +
 .../Impl/Plugin/IPluginProviderProxy.cs         |   74 +
 .../Impl/Plugin/PluginContext.cs                |  100 +
 .../Impl/Plugin/PluginProcessor.cs              |  359 +++
 .../Impl/Plugin/PluginProviderProxy.cs          |   90 +
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |    6 +
 .../Impl/Unmanaged/UnmanagedCallbackOp.cs       |    9 +-
 .../Impl/Unmanaged/UnmanagedCallbacks.cs        |   94 +-
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   12 +
 .../Interop/IPlatformTarget.cs                  |  103 +
 .../Cache/CachePluginProviderTypeAttribute.cs   |   52 +
 .../Plugin/Cache/ICachePluginConfiguration.cs   |   66 +
 .../Plugin/Cache/ICachePluginContext.cs         |   47 +
 .../Plugin/Cache/ICachePluginProvider.cs        |   52 +
 .../Plugin/IPluginConfiguration.cs              |   61 +
 .../Apache.Ignite.Core/Plugin/IPluginContext.cs |   76 +
 .../Plugin/IPluginProvider.cs                   |   69 +
 .../Apache.Ignite.Core/Plugin/PluginCallback.cs |   29 +
 .../Plugin/PluginNotFoundException.cs           |   65 +
 .../Plugin/PluginProviderTypeAttribute.cs       |   52 +
 .../App.config                                  |    4 +-
 .../EntityFrameworkCacheInitializationTest.cs   |    2 +-
 .../EntityFrameworkCacheTest.cs                 |    2 +-
 .../IgniteDbConfiguration.cs                    |    4 +-
 .../Misc/ClientReconnectExample.cs              |    2 +-
 .../scala/org/apache/ignite/scalar/scalar.scala |   17 +-
 .../test/resources/spring-ping-pong-partner.xml |    2 +-
 .../scalar/tests/ScalarProjectionSpec.scala     |    4 +-
 .../schedule/IgniteScheduleProcessor.java       |    2 +-
 .../processors/schedule/ScheduleFutureImpl.java |    4 +-
 .../org/apache/ignite/spark/IgniteContext.scala |    2 +-
 .../spark/JavaEmbeddedIgniteRDDSelfTest.java    |   14 +-
 .../spark/JavaStandaloneIgniteRDDSelfTest.java  |    6 +-
 .../org/apache/ignite/spark/IgniteRDDSpec.scala |   14 +-
 .../org/apache/ignite/IgniteSpringBean.java     |    2 +-
 .../ignite/cache/spring/SpringCacheManager.java |   42 +-
 .../spring/SpringTransactionManager.java        |   50 +-
 .../src/test/config/jdbc-pojo-store-builtin.xml |    2 +-
 .../src/test/config/jdbc-pojo-store-obj.xml     |    2 +-
 modules/spring/src/test/config/node1.xml        |    2 +-
 modules/spring/src/test/config/store-cache1.xml |    2 +-
 .../src/test/java/config/ignite-test-config.xml |    2 +-
 .../test/java/config/spring-transactions.xml    |    2 +-
 .../spring/GridSpringCacheManagerSelfTest.java  |    6 +-
 .../SpringCacheManagerContextInjectionTest.java |    2 +-
 .../ignite/cache/spring/spring-caching.xml      |    2 +-
 .../store/jdbc/CachePojoStoreXmlSelfTest.java   |    4 +-
 .../ignite/internal/GridFactorySelfTest.java    |  154 +-
 .../GridSpringBeanSerializationSelfTest.java    |    2 +-
 .../internal/IgniteDynamicCacheConfigTest.java  |    4 +-
 .../GridSpringResourceInjectionSelfTest.java    |    4 +-
 .../GridTransformSpringInjectionSelfTest.java   |    2 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |    6 +-
 .../org/apache/ignite/spring-injection-test.xml |    2 +-
 .../spring/injection/spring-resource.tmpl.xml   |    2 +-
 .../org/apache/ignite/spring/sprint-exclude.xml |    2 -
 .../GridSpringTransactionManagerSelfTest.java   |    6 +-
 ...gTransactionManagerContextInjectionTest.java |    2 +-
 .../spi/deployment/uri/UriDeploymentSpi.java    |    6 +-
 .../scanners/UriDeploymentScannerManager.java   |   12 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |    6 +-
 .../ignite/p2p/GridP2PDisabledSelfTest.java     |    4 +-
 .../config/VisorConfigurationCommand.scala      |    2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    6 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   10 +-
 .../ignite/visor/VisorRuntimeBaseSpec.scala     |    4 +-
 .../commands/alert/VisorAlertCommandSpec.scala  |    4 +-
 .../cache/VisorCacheClearCommandSpec.scala      |    4 +-
 .../commands/cache/VisorCacheCommandSpec.scala  |    4 +-
 .../cache/VisorCacheResetCommandSpec.scala      |    4 +-
 .../config/VisorConfigurationCommandSpec.scala  |    4 +-
 .../cswap/VisorCacheSwapCommandSpec.scala       |    4 +-
 .../disco/VisorDiscoveryCommandSpec.scala       |    4 +-
 .../events/VisorEventsCommandSpec.scala         |    4 +-
 .../visor/commands/gc/VisorGcCommandSpec.scala  |    4 +-
 .../commands/tasks/VisorTasksCommandSpec.scala  |    4 +-
 modules/web-console/frontend/app/app.js         |   10 +-
 .../app/components/input-dialog/index.js        |   24 +
 .../input-dialog/input-dialog.controller.js     |   35 +
 .../input-dialog/input-dialog.service.js        |   88 +
 .../input-dialog/input-dialog.tpl.pug           |   39 +
 .../frontend/app/modules/sql/sql.controller.js  |   84 +-
 .../frontend/app/services/Clone.service.js      |   66 -
 .../frontend/controllers/caches-controller.js   |   10 +-
 .../frontend/controllers/clusters-controller.js |    6 +-
 .../frontend/controllers/domains-controller.js  |   12 +-
 .../frontend/controllers/igfs-controller.js     |   10 +-
 .../web-console/frontend/views/sql/sql.tpl.pug  |    9 +-
 .../frontend/views/templates/clone.tpl.pug      |   39 -
 .../frontend/views/templates/message.tpl.pug    |    4 +-
 .../ignite/console/demo/AgentClusterDemo.java   |    4 +-
 .../cache/websession/WebSessionFilter.java      |   12 +-
 .../servlet/ServletContextListenerStartup.java  |   12 +-
 .../ignite/startup/servlet/ServletStartup.java  |    8 +-
 .../internal/websession/WebSessionSelfTest.java |   21 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    2 +-
 .../cache/IgniteGetAndPutBenchmark.java         |    2 +-
 .../cache/IgniteGetAndPutTxBenchmark.java       |    2 +-
 .../tcp/ipfinder/zk/ZookeeperIpFinderTest.java  |   10 +-
 1928 files changed, 40484 insertions(+), 22726 deletions(-)
----------------------------------------------------------------------



[08/12] ignite git commit: IGNITE-4717 VisorClearTask minor fix. (cherry picked from commit d4b87f4)

Posted by nt...@apache.org.
IGNITE-4717 VisorClearTask minor fix.
(cherry picked from commit d4b87f4)


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

Branch: refs/heads/master
Commit: 590b82d817595b82a0706e332c545e4746fafbc2
Parents: 3310f10
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Mar 10 15:51:38 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Mar 10 16:37:40 2017 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheClearTask.java        | 57 +++++++++++++++++++-
 1 file changed, 56 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/590b82d8/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
index 0c8476f..ce74f17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorOneNodeTask;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.resources.JobContextResource;
@@ -145,4 +146,58 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
             return S.toString(VisorCacheClearJob.class, this);
         }
     }
-}
\ No newline at end of file
+
+    /**
+     * Callable to get cache size.
+     *
+     * @deprecated This class needed only for compatibility.
+     */
+    @GridInternal @Deprecated
+    private static class VisorCacheSizeCallable implements IgniteCallable<Integer> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final IgniteCache cache;
+
+        /**
+         * @param cache Cache to take size from.
+         */
+        private VisorCacheSizeCallable(IgniteCache cache) {
+            this.cache = cache;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            return cache.size(CachePeekMode.PRIMARY);
+        }
+    }
+
+    /**
+     * Callable to clear cache.
+     *
+     * @deprecated This class needed only for compatibility.
+     */
+    @GridInternal @Deprecated
+    private static class VisorCacheClearCallable implements IgniteCallable<Integer> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private final IgniteCache cache;
+
+        /**
+         * @param cache Cache to clear.
+         */
+        private VisorCacheClearCallable(IgniteCache cache) {
+            this.cache = cache;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            cache.clear();
+
+            return 0;
+        }
+    }
+}


[11/12] ignite git commit: Merge branch ignite-1.8.4 into master

Posted by nt...@apache.org.
Merge branch ignite-1.8.4 into master


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

Branch: refs/heads/master
Commit: a140a8362bc5f9485ab21c3b8829e4c605d3f7d5
Parents: 84880a8 2263283
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 21 13:23:58 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 21 13:23:58 2017 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  33 ++---
 .../affinity/fair/FairAffinityNodesRestart.java | 130 +++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   2 +
 3 files changed, 149 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


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


[06/12] ignite git commit: IGNITE-3386 - Reentrant lock is lost when owner leaves topology

Posted by nt...@apache.org.
IGNITE-3386 - Reentrant lock is lost when owner leaves topology


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

Branch: refs/heads/master
Commit: c2b201ed5bf4bea5dd5aa013f685fdfa02544527
Parents: 840ab99
Author: Evgenii Zhuravlev <ez...@gridgain.com>
Authored: Thu Mar 2 10:27:13 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Mar 2 10:32:50 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  1 +
 .../datastructures/DataStructuresProcessor.java |  3 +-
 .../datastructures/GridCacheLockImpl.java       | 17 +++++-
 .../internal/GridCacheRecreateLockTest.java     | 62 ++++++++++++++++++++
 .../testsuites/IgniteComputeGridTestSuite.java  |  2 +
 5 files changed, 82 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 0de08d5..7f8974c 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -527,6 +527,7 @@ public interface Ignite extends AutoCloseable {
      *      all threads on other nodes waiting to acquire lock are interrupted.
      * @param fair If {@code True}, fair lock will be created.
      * @param create Boolean flag indicating whether data structure should be created if does not exist.
+     *      Will re-create lock if the node that stored the lock left topology and there are no backups left.
      * @return ReentrantLock for the given name.
      * @throws IgniteException If reentrant lock could not be fetched or created.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 1cad22f..698efd9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1396,7 +1396,8 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         name,
                         key,
                         reentrantLockView,
-                        dsCacheCtx);
+                        dsCacheCtx,
+                        create);
 
                     dsMap.put(key, reentrantLock0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 3ab7289..1cf78fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -105,6 +105,9 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
     /** Flag indicating that every operation on this lock should be interrupted. */
     private volatile boolean interruptAll;
 
+    /** Re-create flag. */
+    private volatile boolean reCreate;
+
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -522,7 +525,14 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
                                 GridCacheLockState val = lockView.get(key);
 
                                 if (val == null)
-                                    throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
+                                    if (reCreate) {
+                                        val = new GridCacheLockState(0, ctx.nodeId(), 0, failoverSafe, fair);
+
+                                        lockView.put(key, val);
+                                    }
+                                    else
+                                        throw new IgniteCheckedException("Failed to find reentrant lock with " +
+                                            "the given name: " + name);
 
                                 final long newThreadID = newThread.getId();
 
@@ -1048,12 +1058,14 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
      * @param key Reentrant lock key.
      * @param lockView Reentrant lock projection.
      * @param ctx Cache context.
+     * @param reCreate If {@code true} reentrant lock will be re-created in case it is not in cache.
      */
     @SuppressWarnings("unchecked")
     public GridCacheLockImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView,
-        GridCacheContext ctx) {
+        GridCacheContext ctx,
+        boolean reCreate) {
         assert name != null;
         assert key != null;
         assert ctx != null;
@@ -1063,6 +1075,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
         this.key = key;
         this.lockView = lockView;
         this.ctx = ctx;
+        this.reCreate = reCreate;
 
         log = ctx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
new file mode 100644
index 0000000..ae4ef87
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
@@ -0,0 +1,62 @@
+package org.apache.ignite.internal;
+
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLock;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.common.GridCommonTest;
+
+/**
+ * Create lock after owner node left topology test
+ */
+@GridCommonTest(group = "Kernal Self")
+public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    public void test() throws Exception {
+        final Ignite ignite = startNodeAndLock("node1");
+
+        new Thread(new Runnable() {
+            @Override public void run() {
+                try {
+                    Thread.sleep(2000);
+                }
+                catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+
+                ignite.close();
+            }
+        }).start();
+
+        startNodeAndLock("node2");
+    }
+
+    private Ignite startNodeAndLock(String name) {
+        try {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+            cfg.setGridName(name);
+
+            Ignite ignite = Ignition.start(cfg);
+
+            IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
+
+            System.out.println("acquiring lock");
+
+            lock.lock();
+
+            System.out.println("acquired lock");
+
+            return ignite;
+        }
+        catch (Exception e) {
+            assertTrue(false);
+        }
+
+        return null;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/c2b201ed/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 8a501fd..7c8b6a9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
 import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
 import org.apache.ignite.internal.GridAffinitySelfTest;
 import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
+import org.apache.ignite.internal.GridCacheRecreateLockTest;
 import org.apache.ignite.internal.GridCancelOnGridStopSelfTest;
 import org.apache.ignite.internal.GridCancelUnusedJobSelfTest;
 import org.apache.ignite.internal.GridCancelledJobsMetricsSelfTest;
@@ -152,6 +153,7 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(TaskNodeRestartTest.class);
         suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class);
         suite.addTestSuite(PublicThreadpoolStarvationTest.class);
+        suite.addTestSuite(GridCacheRecreateLockTest.class);
 
         return suite;
     }


[02/12] ignite git commit: Implemented support for enforce join order flag. (cherry picked from commit a7f77d4)

Posted by nt...@apache.org.
Implemented support for enforce join order flag.
(cherry picked from commit a7f77d4)


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

Branch: refs/heads/master
Commit: 5f5cce4f36e6bc0a3468ba1d80fa5f3d158e927f
Parents: 9df5e94
Author: Alexey Kuznetsov <ak...@gridgain.com>
Authored: Wed Mar 1 22:09:40 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 10:49:48 2017 +0700

----------------------------------------------------------------------
 .../internal/visor/query/VisorQueryArgV3.java   | 51 ++++++++++++++++++++
 .../internal/visor/query/VisorQueryJob.java     |  6 +--
 .../resources/META-INF/classnames.properties    |  1 +
 3 files changed, 55 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
new file mode 100644
index 0000000..f32c00a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryArgV3.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+/**
+ * Arguments for {@link VisorQueryTask}.
+ */
+public class VisorQueryArgV3 extends VisorQueryArgV2 {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Enforce join order flag. */
+    private final boolean enforceJoinOrder;
+
+    /**
+     * @param cacheName Cache name for query.
+     * @param qryTxt Query text.
+     * @param distributedJoins If {@code true} then distributed joins enabled.
+     * @param enforceJoinOrder If {@code true} then enforce join order.
+     * @param loc Flag whether to execute query locally.
+     * @param pageSize Result batch size.
+     */
+    public VisorQueryArgV3(String cacheName, String qryTxt,
+        boolean distributedJoins, boolean enforceJoinOrder, boolean loc, int pageSize) {
+        super(cacheName, qryTxt, distributedJoins, loc, pageSize);
+
+        this.enforceJoinOrder = enforceJoinOrder;
+    }
+
+    /**
+     * @return Enforce join order flag.
+     */
+    public boolean enforceJoinOrder() {
+        return enforceJoinOrder;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
index c66b2dd..1ac90ad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
@@ -131,9 +131,8 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 if (scanWithFilter) {
                     boolean caseSensitive = qryTxt.startsWith(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE);
 
-                    String ptrn = caseSensitive
-                        ? qryTxt.substring(SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length())
-                        : qryTxt.substring(SCAN_CACHE_WITH_FILTER.length());
+                    String ptrn = qryTxt.substring(
+                        caseSensitive ? SCAN_CACHE_WITH_FILTER_CASE_SENSITIVE.length() : SCAN_CACHE_WITH_FILTER.length());
 
                     filter = new VisorQueryScanSubstringFilter(caseSensitive, ptrn);
                 }
@@ -162,6 +161,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 qry.setPageSize(arg.pageSize());
                 qry.setLocal(arg.local());
                 qry.setDistributedJoins(arg instanceof VisorQueryArgV2 && ((VisorQueryArgV2)arg).distributedJoins());
+                qry.setEnforceJoinOrder(arg instanceof VisorQueryArgV3 && ((VisorQueryArgV3)arg).enforceJoinOrder());
 
                 long start = U.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/5f5cce4f/modules/core/src/main/resources/META-INF/classnames.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/META-INF/classnames.properties b/modules/core/src/main/resources/META-INF/classnames.properties
index df605da..7bebe24 100644
--- a/modules/core/src/main/resources/META-INF/classnames.properties
+++ b/modules/core/src/main/resources/META-INF/classnames.properties
@@ -1819,6 +1819,7 @@ org.apache.ignite.internal.visor.node.VisorSpisConfiguration
 org.apache.ignite.internal.visor.node.VisorTransactionConfiguration
 org.apache.ignite.internal.visor.query.VisorQueryArg
 org.apache.ignite.internal.visor.query.VisorQueryArgV2
+org.apache.ignite.internal.visor.query.VisorQueryArgV3
 org.apache.ignite.internal.visor.query.VisorQueryCleanupTask
 org.apache.ignite.internal.visor.query.VisorQueryCleanupTask$VisorQueryCleanupJob
 org.apache.ignite.internal.visor.query.VisorQueryField


[10/12] ignite git commit: Merge ignite-1.7.8 into ignite-1.8.4

Posted by nt...@apache.org.
Merge ignite-1.7.8 into ignite-1.8.4

Conflicts:
	modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
	modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java


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

Branch: refs/heads/master
Commit: 22632839bd59123167ed96255d6fbcd9c04f45a9
Parents: 6633da6 c0e2df2
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Mar 15 13:47:55 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Mar 15 13:47:55 2017 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   4 +
 .../distributed/dht/GridDhtCacheAdapter.java    |  19 +-
 .../cache/query/GridCacheQueryManager.java      |  97 +++++-----
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   5 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   6 +
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  32 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  19 ++
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   5 +
 .../tcp/internal/TcpDiscoveryNode.java          |   6 +-
 ...veryNodeAttributesUpdateOnReconnectTest.java | 110 +++++++++++
 .../TcpDiscoverySslSecuredUnsecuredTest.java    | 185 +++++++++++++++++++
 .../tcp/TestReconnectPluginProvider.java        | 111 +++++++++++
 .../discovery/tcp/TestReconnectProcessor.java   |  93 ++++++++++
 .../testframework/GridSpiTestContext.java       |   5 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   5 +
 .../org.apache.ignite.plugin.PluginProvider     |   1 +
 parent/pom.xml                                  |   1 +
 17 files changed, 648 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/22632839/parent/pom.xml
----------------------------------------------------------------------


[09/12] ignite git commit: IGNITE-3386 Reverted: "Reentrant lock is lost when lock owner leaves topology"

Posted by nt...@apache.org.
IGNITE-3386 Reverted: "Reentrant lock is lost when lock owner leaves topology"


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

Branch: refs/heads/master
Commit: 6633da64d57eadacf30bf437a9c3a3d205903dcd
Parents: 590b82d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Mar 14 16:40:58 2017 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Mar 14 16:40:58 2017 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/Ignite.java |  1 -
 .../datastructures/DataStructuresProcessor.java |  3 +-
 .../datastructures/GridCacheLockImpl.java       | 17 +-----
 .../internal/GridCacheRecreateLockTest.java     | 58 --------------------
 .../testsuites/IgniteComputeGridTestSuite.java  |  2 -
 5 files changed, 3 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/Ignite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignite.java b/modules/core/src/main/java/org/apache/ignite/Ignite.java
index 7f8974c..0de08d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignite.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignite.java
@@ -527,7 +527,6 @@ public interface Ignite extends AutoCloseable {
      *      all threads on other nodes waiting to acquire lock are interrupted.
      * @param fair If {@code True}, fair lock will be created.
      * @param create Boolean flag indicating whether data structure should be created if does not exist.
-     *      Will re-create lock if the node that stored the lock left topology and there are no backups left.
      * @return ReentrantLock for the given name.
      * @throws IgniteException If reentrant lock could not be fetched or created.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
index 698efd9..1cad22f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/DataStructuresProcessor.java
@@ -1396,8 +1396,7 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         name,
                         key,
                         reentrantLockView,
-                        dsCacheCtx,
-                        create);
+                        dsCacheCtx);
 
                     dsMap.put(key, reentrantLock0);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
index 1cf78fa..3ab7289 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheLockImpl.java
@@ -105,9 +105,6 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
     /** Flag indicating that every operation on this lock should be interrupted. */
     private volatile boolean interruptAll;
 
-    /** Re-create flag. */
-    private volatile boolean reCreate;
-
     /**
      * Empty constructor required by {@link Externalizable}.
      */
@@ -525,14 +522,7 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
                                 GridCacheLockState val = lockView.get(key);
 
                                 if (val == null)
-                                    if (reCreate) {
-                                        val = new GridCacheLockState(0, ctx.nodeId(), 0, failoverSafe, fair);
-
-                                        lockView.put(key, val);
-                                    }
-                                    else
-                                        throw new IgniteCheckedException("Failed to find reentrant lock with " +
-                                            "the given name: " + name);
+                                    throw new IgniteCheckedException("Failed to find reentrant lock with given name: " + name);
 
                                 final long newThreadID = newThread.getId();
 
@@ -1058,14 +1048,12 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
      * @param key Reentrant lock key.
      * @param lockView Reentrant lock projection.
      * @param ctx Cache context.
-     * @param reCreate If {@code true} reentrant lock will be re-created in case it is not in cache.
      */
     @SuppressWarnings("unchecked")
     public GridCacheLockImpl(String name,
         GridCacheInternalKey key,
         IgniteInternalCache<GridCacheInternalKey, GridCacheLockState> lockView,
-        GridCacheContext ctx,
-        boolean reCreate) {
+        GridCacheContext ctx) {
         assert name != null;
         assert key != null;
         assert ctx != null;
@@ -1075,7 +1063,6 @@ public final class GridCacheLockImpl implements GridCacheLockEx, Externalizable
         this.key = key;
         this.lockView = lockView;
         this.ctx = ctx;
-        this.reCreate = reCreate;
 
         log = ctx.logger(getClass());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
deleted file mode 100644
index ae850f7..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
+++ /dev/null
@@ -1,58 +0,0 @@
-package org.apache.ignite.internal;
-
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLock;
-import org.apache.ignite.Ignition;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-
-/**
- * Create lock after owner node left topology test
- */
-@GridCommonTest(group = "Kernal Self")
-public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
-    /**
-     * @throws IgniteCheckedException If failed.
-     */
-    public void testLockOwnerLeavesGrid() throws Exception {
-        final Ignite ignite = startNodeAndLock("node1");
-
-        new Thread(new Runnable() {
-            @Override public void run() {
-                try {
-                    Thread.sleep(2000);
-                }
-                catch (InterruptedException e) {
-                    e.printStackTrace();
-                }
-
-                ignite.close();
-            }
-        }).start();
-
-        startNodeAndLock("node2");
-    }
-
-    /**
-     * @param name Grid name.
-     * @return Started Ignite instance.
-     */
-    private Ignite startNodeAndLock(String name) throws Exception {
-        IgniteConfiguration cfg = new IgniteConfiguration();
-        cfg.setGridName(name);
-
-        Ignite ignite = Ignition.start(cfg);
-
-        IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
-
-        System.out.println("acquiring lock");
-
-        lock.lock();
-
-        System.out.println("acquired lock");
-
-        return ignite;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/6633da64/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index 7c8b6a9..8a501fd 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -22,7 +22,6 @@ import org.apache.ignite.internal.ClusterNodeMetricsSelfTest;
 import org.apache.ignite.internal.GridAffinityNoCacheSelfTest;
 import org.apache.ignite.internal.GridAffinitySelfTest;
 import org.apache.ignite.internal.GridAlwaysFailoverSpiFailSelfTest;
-import org.apache.ignite.internal.GridCacheRecreateLockTest;
 import org.apache.ignite.internal.GridCancelOnGridStopSelfTest;
 import org.apache.ignite.internal.GridCancelUnusedJobSelfTest;
 import org.apache.ignite.internal.GridCancelledJobsMetricsSelfTest;
@@ -153,7 +152,6 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(TaskNodeRestartTest.class);
         suite.addTestSuite(IgniteRoundRobinErrorAfterClientReconnectTest.class);
         suite.addTestSuite(PublicThreadpoolStarvationTest.class);
-        suite.addTestSuite(GridCacheRecreateLockTest.class);
 
         return suite;
     }


[04/12] ignite git commit: Minor cleanup.

Posted by nt...@apache.org.
Minor cleanup.


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

Branch: refs/heads/master
Commit: 620235f3ca0fa32c1f4a4dc1a5e0bd5545f6e07e
Parents: 7292a53
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Thu Mar 2 11:05:19 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 11:05:19 2017 +0700

----------------------------------------------------------------------
 .../ignite/internal/visor/cache/VisorCacheTypeMetadata.java    | 6 ++++++
 .../ignite/internal/visor/node/VisorIgfsConfiguration.java     | 1 -
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/620235f3/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
index f17e588..c87ad05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheTypeMetadata.java
@@ -32,6 +32,7 @@ import org.apache.ignite.cache.store.jdbc.JdbcType;
 import org.apache.ignite.cache.store.jdbc.JdbcTypeField;
 import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiTuple;
 
@@ -372,4 +373,9 @@ public class VisorCacheTypeMetadata implements Serializable, LessNamingBean {
     public Map<String, LinkedHashMap<String, IgniteBiTuple<String, Boolean>>> grps() {
         return grps;
     }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(VisorCacheTypeMetadata.class, this);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/620235f3/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
index 9f7652b..cb10c1c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorIgfsConfiguration.java
@@ -25,7 +25,6 @@ import java.util.Map;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.igfs.IgfsIpcEndpointConfiguration;
 import org.apache.ignite.igfs.IgfsMode;
-import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystem;
 import org.apache.ignite.internal.LessNamingBean;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.Nullable;


[07/12] ignite git commit: IGNITE-3386 - Minor code style changes

Posted by nt...@apache.org.
IGNITE-3386 - Minor code style changes


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

Branch: refs/heads/master
Commit: 3310f10b9d48d64280d0ebf2464ad892f3f31c52
Parents: c2b201e
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Mar 2 10:37:00 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Mar 2 10:37:00 2017 +0300

----------------------------------------------------------------------
 .../internal/GridCacheRecreateLockTest.java     | 32 +++++++++-----------
 1 file changed, 14 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3310f10b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
index ae4ef87..ae850f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridCacheRecreateLockTest.java
@@ -13,11 +13,10 @@ import org.apache.ignite.testframework.junits.common.GridCommonTest;
  */
 @GridCommonTest(group = "Kernal Self")
 public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
-
     /**
      * @throws IgniteCheckedException If failed.
      */
-    public void test() throws Exception {
+    public void testLockOwnerLeavesGrid() throws Exception {
         final Ignite ignite = startNodeAndLock("node1");
 
         new Thread(new Runnable() {
@@ -36,27 +35,24 @@ public class GridCacheRecreateLockTest extends GridCommonAbstractTest {
         startNodeAndLock("node2");
     }
 
-    private Ignite startNodeAndLock(String name) {
-        try {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-            cfg.setGridName(name);
-
-            Ignite ignite = Ignition.start(cfg);
+    /**
+     * @param name Grid name.
+     * @return Started Ignite instance.
+     */
+    private Ignite startNodeAndLock(String name) throws Exception {
+        IgniteConfiguration cfg = new IgniteConfiguration();
+        cfg.setGridName(name);
 
-            IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
+        Ignite ignite = Ignition.start(cfg);
 
-            System.out.println("acquiring lock");
+        IgniteLock lock = ignite.reentrantLock("lock", true, true, true);
 
-            lock.lock();
+        System.out.println("acquiring lock");
 
-            System.out.println("acquired lock");
+        lock.lock();
 
-            return ignite;
-        }
-        catch (Exception e) {
-            assertTrue(false);
-        }
+        System.out.println("acquired lock");
 
-        return null;
+        return ignite;
     }
 }
\ No newline at end of file


[05/12] ignite git commit: IGNITE-4436 API for collecting list of running queries and cancel them. (cherry picked from commit 4923734)

Posted by nt...@apache.org.
IGNITE-4436 API for collecting list of running queries and cancel them.
(cherry picked from commit 4923734)


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

Branch: refs/heads/master
Commit: 840ab997436dfad8cdbb4ad182de24bceec37962
Parents: 620235f
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Feb 14 20:54:31 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 11:17:39 2017 +0700

----------------------------------------------------------------------
 .../processors/query/GridQueryIndexing.java     |  17 +-
 .../processors/query/GridQueryProcessor.java    |  32 ++-
 .../processors/query/GridRunningQueryInfo.java  | 132 ++++++++++++
 .../internal/visor/VisorMultiNodeTask.java      |   2 +-
 .../visor/query/VisorCancelQueriesTask.java     |  72 +++++++
 .../query/VisorCollectRunningQueriesTask.java   |  96 +++++++++
 .../internal/visor/query/VisorRunningQuery.java | 132 ++++++++++++
 .../cache/query/GridCacheTwoStepQuery.java      |  18 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  86 +++++++-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  60 +++++-
 .../cache/CacheSqlQueryValueCopySelfTest.java   | 208 +++++++++++++++++--
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   2 +-
 .../h2/GridIndexingSpiAbstractSelfTest.java     |   7 +
 14 files changed, 826 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index ef39d96..ca04724 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -241,7 +241,22 @@ public interface GridQueryIndexing {
     public PreparedStatement prepareNativeStatement(String schema, String sql) throws SQLException;
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> runningQueries(long duration);
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries ID's to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries);
+
+    /**
      * Cancels all executing queries.
      */
     public void cancelAllQueries();
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index f8919a1..10bf75a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -44,7 +44,6 @@ import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.binary.BinaryField;
 import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryObjectBuilder;
@@ -119,7 +118,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     private static final int QRY_DETAIL_METRICS_EVICTION_FREQ = 3_000;
 
     /** */
-    private static Set<Class<?>> SQL_TYPES = new HashSet<>(F.<Class<?>>asList(
+    private static final Set<Class<?>> SQL_TYPES = new HashSet<>(F.<Class<?>>asList(
         Integer.class,
         Boolean.class,
         Byte.class,
@@ -920,6 +919,29 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> runningQueries(long duration) {
+        if (moduleEnabled())
+            return idx.runningQueries(duration);
+
+        return Collections.emptyList();
+    }
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries ID's to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries) {
+        if (moduleEnabled())
+            idx.cancelQueries(queries);
+    }
+
+    /**
      * @param sqlQry Sql query.
      * @param params Params.
      */
@@ -2722,7 +2744,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Accessor that deals with fields. */
-    private final static class FieldAccessor implements PropertyAccessor {
+    private static final class FieldAccessor implements PropertyAccessor {
         /** Field to access. */
         private final Field fld;
 
@@ -2765,7 +2787,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Getter and setter methods based accessor. */
-    private final static class MethodsAccessor implements PropertyAccessor {
+    private static final class MethodsAccessor implements PropertyAccessor {
         /** */
         private final Method getter;
 
@@ -2823,7 +2845,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** Accessor with getter only. */
-    private final static class ReadOnlyMethodsAccessor implements PropertyAccessor {
+    private static final class ReadOnlyMethodsAccessor implements PropertyAccessor {
         /** */
         private final Method getter;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
new file mode 100644
index 0000000..d77c8c0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridRunningQueryInfo.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.query;
+
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+
+/**
+ * Query descriptor.
+ */
+public class GridRunningQueryInfo {
+    /** */
+    private final long id;
+
+    /** */
+    private final String qry;
+
+    /** Query type. */
+    private final GridCacheQueryType qryType;
+
+    /** */
+    private final String cache;
+
+    /** */
+    private final long startTime;
+
+    /** */
+    private final GridQueryCancel cancel;
+
+    /** */
+    private final boolean loc;
+
+    /**
+     * @param id Query ID.
+     * @param qry Query text.
+     * @param qryType Query type.
+     * @param cache Cache where query was executed.
+     * @param startTime Query start time.
+     * @param cancel Query cancel.
+     * @param loc Local query flag.
+     */
+    public GridRunningQueryInfo(Long id, String qry, GridCacheQueryType qryType, String cache, long startTime,
+        GridQueryCancel cancel, boolean loc) {
+        this.id = id;
+        this.qry = qry;
+        this.qryType = qryType;
+        this.cache = cache;
+        this.startTime = startTime;
+        this.cancel = cancel;
+        this.loc = loc;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public Long id() {
+        return id;
+    }
+
+    /**
+     * @return Query text.
+     */
+    public String query() {
+        return qry;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public GridCacheQueryType queryType() {
+        return qryType;
+    }
+
+    /**
+     * @return Cache where query was executed.
+     */
+    public String cache() {
+        return cache;
+    }
+
+    /**
+     * @return Query start time.
+     */
+    public long startTime() {
+        return startTime;
+    }
+
+    /**
+     * @param curTime Current time.
+     * @param duration Duration of long query.
+     * @return {@code true} if this query should be considered as long running query.
+     */
+    public boolean longQuery(long curTime, long duration) {
+        return curTime - startTime > duration;
+    }
+
+    /**
+     * Cancel query.
+     */
+    public void cancel() {
+        if (cancel != null)
+            cancel.cancel();
+    }
+
+    /**
+     * @return {@code true} if query can be cancelled.
+     */
+    public boolean cancelable() {
+        return cancel != null;
+    }
+
+    /**
+     * @return {@code true} if query is local.
+     */
+    public boolean local() {
+        return loc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
index 57f1346..ece1a17 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
@@ -130,4 +130,4 @@ public abstract class VisorMultiNodeTask<A, R, J> implements ComputeTask<VisorTa
                 logFinish(ignite.log(), getClass(), start);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
new file mode 100644
index 0000000..a6f2d82
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCancelQueriesTask.java
@@ -0,0 +1,72 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorOneNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Task to cancel queries.
+ */
+@GridInternal
+public class VisorCancelQueriesTask extends VisorOneNodeTask<Collection<Long>, Void> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCancelQueriesJob job(Collection<Long> arg) {
+        return new VisorCancelQueriesJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override protected Void reduce0(List<ComputeJobResult> results) throws IgniteException {
+        return null;
+    }
+
+    /**
+     * Job to cancel queries on node.
+     */
+    private static class VisorCancelQueriesJob extends VisorJob<Collection<Long>, Void> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorCancelQueriesJob(@Nullable Collection<Long> arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Void run(@Nullable Collection<Long> queries) throws IgniteException {
+            ignite.context().query().cancelQueries(queries);
+
+            return null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
new file mode 100644
index 0000000..2b40e61
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorCollectRunningQueriesTask.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.compute.ComputeJobResult;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
+import org.apache.ignite.internal.processors.task.GridInternal;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.internal.visor.VisorJob;
+import org.apache.ignite.internal.visor.VisorMultiNodeTask;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Task to collect currently running queries.
+ */
+@GridInternal
+public class VisorCollectRunningQueriesTask extends VisorMultiNodeTask<Long, Map<UUID, Collection<VisorRunningQuery>>, Collection<VisorRunningQuery>> {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** {@inheritDoc} */
+    @Override protected VisorCollectRunningQueriesJob job(Long arg) {
+        return new VisorCollectRunningQueriesJob(arg, debug);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override protected Map<UUID, Collection<VisorRunningQuery>> reduce0(List<ComputeJobResult> results) throws IgniteException {
+        Map<UUID, Collection<VisorRunningQuery>> map = new HashMap<>();
+
+        for (ComputeJobResult res : results)
+            if (res.getException() == null) {
+                Collection<VisorRunningQuery> queries = res.getData();
+
+                map.put(res.getNode().id(), queries);
+            }
+
+        return map;
+    }
+
+    /**
+     * Job to collect currently running queries from node.
+     */
+    private static class VisorCollectRunningQueriesJob extends VisorJob<Long, Collection<VisorRunningQuery>> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Create job with specified argument.
+         *
+         * @param arg Job argument.
+         * @param debug Flag indicating whether debug information should be printed into node log.
+         */
+        protected VisorCollectRunningQueriesJob(@Nullable Long arg, boolean debug) {
+            super(arg, debug);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected Collection<VisorRunningQuery> run(@Nullable Long duration) throws IgniteException {
+            Collection<GridRunningQueryInfo> queries = ignite.context().query()
+                .runningQueries(duration != null ? duration : 0);
+
+            Collection<VisorRunningQuery> res = new ArrayList<>(queries.size());
+
+            long curTime = U.currentTimeMillis();
+
+            for (GridRunningQueryInfo qry : queries)
+                res.add(new VisorRunningQuery(qry.id(), qry.query(), qry.queryType(), qry.cache(),
+                    qry.startTime(), curTime - qry.startTime(),
+                    qry.cancelable(), qry.local()));
+
+            return res;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
new file mode 100644
index 0000000..fc6bc7a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorRunningQuery.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.query;
+
+import java.io.Serializable;
+import org.apache.ignite.internal.processors.cache.query.GridCacheQueryType;
+
+/**
+ * Descriptor of running query.
+ */
+public class VisorRunningQuery implements Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long id;
+
+    /** Query text. */
+    private String qry;
+
+    /** Query type. */
+    private GridCacheQueryType qryType;
+
+    /** Cache name for query. */
+    private String cache;
+
+    /** */
+    private long startTime;
+
+    /** */
+    private long duration;
+
+    /** */
+    private boolean cancellable;
+
+    /** */
+    private boolean loc;
+
+    /**
+     * @param id Query ID.
+     * @param qry Query text.
+     * @param qryType Query type.
+     * @param cache Cache where query was executed.
+     * @param startTime Query start time.
+     * @param duration Query current duration.
+     * @param cancellable {@code true} if query can be canceled.
+     * @param loc {@code true} if query is local.
+     */
+    public VisorRunningQuery(long id, String qry, GridCacheQueryType qryType, String cache,
+        long startTime, long duration,
+        boolean cancellable, boolean loc) {
+        this.id = id;
+        this.qry = qry;
+        this.qryType = qryType;
+        this.cache = cache;
+        this.startTime = startTime;
+        this.duration = duration;
+        this.cancellable = cancellable;
+        this.loc = loc;
+    }
+
+    /**
+     * @return Query ID.
+     */
+    public long getId() {
+        return id;
+    }
+
+    /**
+     * @return Query txt.
+     */
+    public String getQuery() {
+        return qry;
+    }
+
+    /**
+     * @return Query type.
+     */
+    public GridCacheQueryType getQueryType() {
+        return qryType;
+    }
+
+    /**
+     * @return Cache name.
+     */
+    public String getCache() {
+        return cache;
+    }
+
+    /**
+     * @return Query start time.
+     */
+    public long getStartTime() {
+        return startTime;
+    }
+
+    /**
+     * @return Query duration.
+     */
+    public long getDuration() {
+        return duration;
+    }
+
+    /**
+     * @return {@code true} if query can be cancelled.
+     */
+    public boolean isCancelable() {
+        return cancellable;
+    }
+
+    /**
+     * @return {@code true} if query is local.
+     */
+    public boolean isLocal() {
+        return loc;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
index 8dcba2f..f53936f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheTwoStepQuery.java
@@ -46,6 +46,9 @@ public class GridCacheTwoStepQuery {
     private boolean explain;
 
     /** */
+    private String originalSql;
+
+    /** */
     private Collection<String> spaces;
 
     /** */
@@ -67,10 +70,12 @@ public class GridCacheTwoStepQuery {
     private List<Integer> extraCaches;
 
     /**
+     * @param originalSql Original query SQL.
      * @param schemas Schema names in query.
      * @param tbls Tables in query.
      */
-    public GridCacheTwoStepQuery(Set<String> schemas, Set<String> tbls) {
+    public GridCacheTwoStepQuery(String originalSql, Set<String> schemas, Set<String> tbls) {
+        this.originalSql = originalSql;
         this.schemas = schemas;
         this.tbls = tbls;
     }
@@ -196,6 +201,13 @@ public class GridCacheTwoStepQuery {
     }
 
     /**
+     * @return Original query SQL.
+     */
+    public String originalSql() {
+        return originalSql;
+    }
+
+    /**
      * @return Spaces.
      */
     public Collection<String> spaces() {
@@ -223,7 +235,7 @@ public class GridCacheTwoStepQuery {
     public GridCacheTwoStepQuery copy(Object[] args) {
         assert !explain;
 
-        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(schemas, tbls);
+        GridCacheTwoStepQuery cp = new GridCacheTwoStepQuery(originalSql, schemas, tbls);
 
         cp.caches = caches;
         cp.extraCaches = extraCaches;
@@ -250,4 +262,4 @@ public class GridCacheTwoStepQuery {
     @Override public String toString() {
         return S.toString(GridCacheTwoStepQuery.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index e375867..8c5c2a3 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -52,6 +52,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
 import javax.cache.Cache;
 import javax.cache.CacheException;
 import org.apache.ignite.IgniteCheckedException;
@@ -81,6 +82,7 @@ import org.apache.ignite.internal.processors.cache.QueryCursorImpl;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
 import org.apache.ignite.internal.processors.cache.query.IgniteQueryErrorCode;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.GridQueryFieldMetadata;
 import org.apache.ignite.internal.processors.query.GridQueryFieldsResult;
@@ -176,7 +178,11 @@ import org.jsr166.ConcurrentHashMap8;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_CLEANUP_PERIOD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_H2_INDEXING_CACHE_THREAD_USAGE_TIMEOUT;
+import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.IgniteSystemProperties.getString;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.TEXT;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.FULLTEXT;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.GEO_SPATIAL;
 import static org.apache.ignite.internal.processors.query.GridQueryIndexType.SORTED;
@@ -284,9 +290,15 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     private final Map<String, String> space2schema = new ConcurrentHashMap8<>();
 
     /** */
+    private AtomicLong qryIdGen;
+
+    /** */
     private GridSpinBusyLock busyLock;
 
     /** */
+    private final ConcurrentMap<Long, GridRunningQueryInfo> runs = new ConcurrentHashMap8<>();
+
+    /** */
     private final ThreadLocal<ConnectionWrapper> connCache = new ThreadLocal<ConnectionWrapper>() {
         @Nullable @Override public ConnectionWrapper get() {
             ConnectionWrapper c = super.get();
@@ -771,8 +783,19 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         IndexingQueryFilter filters) throws IgniteCheckedException {
         TableDescriptor tbl = tableDescriptor(spaceName, type);
 
-        if (tbl != null && tbl.luceneIdx != null)
-            return tbl.luceneIdx.query(qry, filters);
+        if (tbl != null && tbl.luceneIdx != null) {
+            GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, TEXT, spaceName,
+                U.currentTimeMillis(), null, true);
+
+            try {
+                runs.put(run.id(), run);
+
+                return tbl.luceneIdx.query(qry, filters);
+            }
+            finally {
+                runs.remove(run.id());
+            }
+        }
 
         return new GridEmptyCloseableIterator<>();
     }
@@ -830,6 +853,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
                 GridH2QueryContext.set(ctx);
 
+                GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL_FIELDS,
+                    spaceName, U.currentTimeMillis(), cancel, true);
+
+                runs.putIfAbsent(run.id(), run);
+
                 try {
                     ResultSet rs = executeSqlQueryWithTimer(spaceName, stmt, conn, qry, params, timeout, cancel);
 
@@ -837,6 +865,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 }
                 finally {
                     GridH2QueryContext.clearThreadLocal();
+
+                    runs.remove(run.id());
                 }
             }
         };
@@ -1086,6 +1116,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         GridH2QueryContext.set(new GridH2QueryContext(nodeId, nodeId, 0, LOCAL).filter(filter).distributedJoins(false));
 
+        GridRunningQueryInfo run = new GridRunningQueryInfo(qryIdGen.incrementAndGet(), qry, SQL, spaceName,
+            U.currentTimeMillis(), null, true);
+
+        runs.put(run.id(), run);
+
         try {
             ResultSet rs = executeSqlQueryWithTimer(spaceName, conn, sql, params, true, 0, null);
 
@@ -1093,6 +1128,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         }
         finally {
             GridH2QueryContext.clearThreadLocal();
+
+            runs.remove(run.id());
         }
     }
 
@@ -1232,7 +1269,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                             try {
                                 ctx.cache().createMissingCaches();
                             }
-                            catch (IgniteCheckedException e1) {
+                            catch (IgniteCheckedException ignored) {
                                 throw new CacheException("Failed to create missing caches.", e);
                             }
 
@@ -1737,6 +1774,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
 
         this.busyLock = busyLock;
 
+        qryIdGen = new AtomicLong();
+
         if (SysProperties.serializeJavaObject) {
             U.warn(log, "Serialization of Java objects in H2 was enabled.");
 
@@ -1787,7 +1826,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             marshaller = ctx.config().getMarshaller();
 
             mapQryExec = new GridMapQueryExecutor(busyLock);
-            rdcQryExec = new GridReduceQueryExecutor(busyLock);
+            rdcQryExec = new GridReduceQueryExecutor(qryIdGen, busyLock);
 
             mapQryExec.start(ctx, this);
             rdcQryExec.start(ctx, this);
@@ -2241,6 +2280,37 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         return cols;
     }
 
+
+    /** {@inheritDoc} */
+    @Override public Collection<GridRunningQueryInfo> runningQueries(long duration) {
+        Collection<GridRunningQueryInfo> res = new ArrayList<>();
+
+        res.addAll(runs.values());
+        res.addAll(rdcQryExec.longRunningQueries(duration));
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelQueries(Collection<Long> queries) {
+        if (!F.isEmpty(queries)) {
+            for (Long qryId : queries) {
+                GridRunningQueryInfo run = runs.get(qryId);
+
+                if (run != null)
+                    run.cancel();
+            }
+
+            rdcQryExec.cancelQueries(queries);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void cancelAllQueries() {
+        for (Connection conn : conns)
+            U.close(conn, log);
+    }
+
     /**
      * Wrapper to store connection and flag is schema set or not.
      */
@@ -3151,10 +3221,4 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             lastUsage = U.currentTimeMillis();
         }
     }
-
-    /** {@inheritDoc} */
-    @Override public void cancelAllQueries() {
-        for (Connection conn : conns)
-            U.close(conn, log);
-    }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
index 09952cf..e164315 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/sql/GridSqlQuerySplitter.java
@@ -174,7 +174,7 @@ public class GridSqlQuerySplitter {
         qry = collectAllTables(qry, schemas, tbls);
 
         // Build resulting two step query.
-        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(schemas, tbls);
+        GridCacheTwoStepQuery res = new GridCacheTwoStepQuery(qry.getSQL(), schemas, tbls);
 
         // Map query will be direct reference to the original query AST.
         // Thus all the modifications will be performed on the original AST, so we should be careful when
@@ -958,4 +958,4 @@ public class GridSqlQuerySplitter {
     private static GridSqlFunction function(GridSqlFunctionType type) {
         return new GridSqlFunction(type);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 1f00ed2..ee9976c 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -62,6 +62,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 import org.apache.ignite.internal.processors.query.GridQueryCacheObjectsIterator;
 import org.apache.ignite.internal.processors.query.GridQueryCancel;
 import org.apache.ignite.internal.processors.query.h2.IgniteH2Indexing;
@@ -98,6 +99,7 @@ import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
 import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion.NONE;
+import static org.apache.ignite.internal.processors.cache.query.GridCacheQueryType.SQL_FIELDS;
 import static org.apache.ignite.internal.processors.query.h2.opt.GridH2QueryType.REDUCE;
 
 /**
@@ -120,7 +122,7 @@ public class GridReduceQueryExecutor {
     private IgniteLogger log;
 
     /** */
-    private final AtomicLong reqIdGen = new AtomicLong();
+    private final AtomicLong qryIdGen;
 
     /** */
     private final ConcurrentMap<Long, QueryRun> runs = new ConcurrentHashMap8<>();
@@ -167,9 +169,11 @@ public class GridReduceQueryExecutor {
     };
 
     /**
+     * @param qryIdGen Query ID generator.
      * @param busyLock Busy lock.
      */
-    public GridReduceQueryExecutor(GridSpinBusyLock busyLock) {
+    public GridReduceQueryExecutor(AtomicLong qryIdGen, GridSpinBusyLock busyLock) {
+        this.qryIdGen = qryIdGen;
         this.busyLock = busyLock;
     }
 
@@ -493,11 +497,13 @@ public class GridReduceQueryExecutor {
                 }
             }
 
-            final long qryReqId = reqIdGen.incrementAndGet();
+            final long qryReqId = qryIdGen.incrementAndGet();
 
             final String space = cctx.name();
 
-            final QueryRun r = new QueryRun(h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize());
+            final QueryRun r = new QueryRun(qryReqId, qry.originalSql(), space,
+                h2.connectionForSpace(space), qry.mapQueries().size(), qry.pageSize(),
+                U.currentTimeMillis(), cancel);
 
             AffinityTopologyVersion topVer = h2.readyTopologyVersion();
 
@@ -1303,10 +1309,46 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * Collect queries that already running more than specified duration.
+     *
+     * @param duration Duration to check.
+     * @return Collection of IDs and statements of long running queries.
+     */
+    public Collection<GridRunningQueryInfo> longRunningQueries(long duration) {
+        Collection<GridRunningQueryInfo> res = new ArrayList<>();
+
+        long curTime = U.currentTimeMillis();
+
+        for (QueryRun run : runs.values()) {
+            if (run.qry.longQuery(curTime, duration))
+                res.add(run.qry);
+        }
+
+        return res;
+    }
+
+    /**
+     * Cancel specified queries.
+     *
+     * @param queries Queries IDs to cancel.
+     */
+    public void cancelQueries(Collection<Long> queries) {
+        for (Long qryId : queries) {
+            QueryRun run = runs.get(qryId);
+
+            if (run != null)
+                run.qry.cancel();
+        }
+    }
+
+    /**
      * Query run.
      */
     private static class QueryRun {
         /** */
+        private final GridRunningQueryInfo qry;
+
+        /** */
         private final List<GridMergeIndex> idxs;
 
         /** */
@@ -1322,11 +1364,17 @@ public class GridReduceQueryExecutor {
         private final AtomicReference<Object> state = new AtomicReference<>();
 
         /**
+         * @param id Query ID.
+         * @param qry Query text.
+         * @param cache Cache where query was executed.
          * @param conn Connection.
          * @param idxsCnt Number of indexes.
          * @param pageSize Page size.
+         * @param startTime Start time.
+         * @param cancel Query cancel handler.
          */
-        private QueryRun(Connection conn, int idxsCnt, int pageSize) {
+        private QueryRun(Long id, String qry, String cache, Connection conn, int idxsCnt, int pageSize, long startTime, GridQueryCancel cancel) {
+            this.qry = new GridRunningQueryInfo(id, qry, SQL_FIELDS, cache, startTime, cancel, false);
             this.conn = (JdbcConnection)conn;
             this.idxs = new ArrayList<>(idxsCnt);
             this.pageSize = pageSize > 0 ? pageSize : GridCacheTwoStepQuery.DFLT_PAGE_SIZE;
@@ -1384,4 +1432,4 @@ public class GridReduceQueryExecutor {
             return copy(msg, n, partsMap);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
index e47e893..66e7e4a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/CacheSqlQueryValueCopySelfTest.java
@@ -17,15 +17,23 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.Query;
 import org.apache.ignite.cache.query.QueryCursor;
 import org.apache.ignite.cache.query.SqlFieldsQuery;
 import org.apache.ignite.cache.query.SqlQuery;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.cache.query.annotations.QuerySqlFunction;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.query.GridQueryProcessor;
+import org.apache.ignite.internal.processors.query.GridRunningQueryInfo;
 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;
@@ -54,6 +62,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
 
         cc.setCopyOnRead(true);
         cc.setIndexedTypes(Integer.class, Value.class);
+        cc.setSqlFunctionClasses(TestSQLFunctions.class);
 
         cfg.setCacheConfiguration(cc);
 
@@ -72,7 +81,7 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         IgniteCache<Integer, Value> cache = grid(0).cache(null);
 
         for (int i = 0; i < KEYS; i++)
-            cache.put(i, new Value("before"));
+            cache.put(i, new Value(i, "before-" + i));
     }
 
     /** {@inheritDoc} */
@@ -195,17 +204,148 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         check(cache);
     }
 
-    /** */
-    private static class Value {
-        /** */
-        private String str;
+    /**
+     * Run specified query in separate thread.
+     *
+     * @param qry Query to execute.
+     */
+    private IgniteInternalFuture<?> runQueryAsync(final Query<?> qry) throws Exception {
+        return multithreadedAsync(new Runnable() {
+            @Override public void run() {
+                try {
+                    log.info(">>> Query started");
+
+                    grid(0).cache(null).query(qry).getAll();
+
+                    log.info(">>> Query finished");
+                }
+                catch (Throwable e) {
+                    e.printStackTrace();
+                }
+            }
+        }, 1, "run-query");
+    }
 
-        /**
-         * @param str String.
-         */
-        public Value(String str) {
-            this.str = str;
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRunningSqlFieldsQuery() throws Exception {
+        IgniteInternalFuture<?> fut = runQueryAsync(new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3"));
+
+        Thread.sleep(500);
+
+        GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+
+        SqlFieldsQuery qry = new SqlFieldsQuery("select _val, sleep(1000) from Value limit 3");
+        qry.setLocal(true);
+
+        fut = runQueryAsync(qry);
+
+        Thread.sleep(500);
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+    }
+
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRunningSqlQuery() throws Exception {
+        IgniteInternalFuture<?> fut = runQueryAsync(new SqlQuery<Integer, Value>(Value.class, "id > sleep(100)"));
+
+        Thread.sleep(500);
+
+        GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+
+        SqlQuery<Integer, Value> qry = new SqlQuery<>(Value.class, "id > sleep(100)");
+        qry.setLocal(true);
+
+        fut = runQueryAsync(qry);
+
+        Thread.sleep(500);
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        fut.get();
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
+    }
+
+    /**
+     * Test collecting info about running.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCancelingSqlFieldsQuery() throws Exception {
+        runQueryAsync(new SqlFieldsQuery("select * from (select _val, sleep(100) from Value limit 50)"));
+
+        Thread.sleep(500);
+
+        final GridQueryProcessor qryProc = grid(0).context().query();
+
+        Collection<GridRunningQueryInfo> queries = qryProc.runningQueries(0);
+
+        assertEquals(1, queries.size());
+
+        final Collection<GridRunningQueryInfo> finalQueries = queries;
+
+        for (GridRunningQueryInfo query : finalQueries)
+            qryProc.cancelQueries(Collections.singleton(query.id()));
+
+        int n = 100;
+
+        // Give cluster some time to cancel query and cleanup resources.
+        while (n > 0) {
+            Thread.sleep(100);
+
+            queries = qryProc.runningQueries(0);
+
+            if (queries.isEmpty())
+                break;
+
+            log.info(">>>> Wait for cancel: " + n);
+
+            n--;
         }
+
+        queries = qryProc.runningQueries(0);
+
+        assertEquals(0, queries.size());
     }
 
     /**
@@ -218,9 +358,53 @@ public class CacheSqlQueryValueCopySelfTest extends GridCommonAbstractTest {
         for (Cache.Entry<Integer, Value> entry : cache) {
             cnt++;
 
-            assertEquals("before", entry.getValue().str);
+            assertEquals("before-" + entry.getKey(), entry.getValue().str);
         }
 
         assertEquals(KEYS, cnt);
     }
-}
\ No newline at end of file
+
+    /** */
+    private static class Value {
+        /** */
+        @QuerySqlField
+        private int id;
+
+        /** */
+        @QuerySqlField
+        private String str;
+
+        /**
+         * @param id ID.
+         * @param str String.
+         */
+        public Value(int id, String str) {
+            this.id = id;
+            this.str = str;
+        }
+    }
+
+    /**
+     * Utility class with custom SQL functions.
+     */
+    public static class TestSQLFunctions {
+        /**
+         * Sleep function to simulate long running queries.
+         *
+         * @param x Time to sleep.
+         * @return Return specified argument.
+         */
+        @QuerySqlFunction
+        public static long sleep(long x) {
+            if (x >= 0)
+                try {
+                    Thread.sleep(x);
+                }
+                catch (InterruptedException ignored) {
+                    // No-op.
+                }
+
+            return x;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
index 1f10593..01fefa3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheCrossCacheQuerySelfTest.java
@@ -477,4 +477,4 @@ public class GridCacheCrossCacheQuerySelfTest extends GridCommonAbstractTest {
             return storeId;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/840ab997/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
index 17151d8..09b27d2 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridIndexingSpiAbstractSelfTest.java
@@ -106,6 +106,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         spi.registerCache(null, cacheCfg("B"));
     }
 
+    /**
+     * @param name Name.
+     */
     private CacheConfiguration cacheCfg(String name) {
         CacheConfiguration<?,?> cfg = new CacheConfiguration<>();
 
@@ -114,6 +117,7 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return cfg;
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         idx.stop();
 
@@ -182,6 +186,9 @@ public abstract class GridIndexingSpiAbstractSelfTest extends GridCommonAbstract
         return idx;
     }
 
+    /**
+     * @return {@code true} if OFF-HEAP mode should be tested.
+     */
     protected boolean offheap() {
         return false;
     }


[03/12] ignite git commit: IGNITE-4717 Fixed hangs in VisorCacheClearTask. (cherry picked from commit 76f3060)

Posted by nt...@apache.org.
IGNITE-4717 Fixed hangs in VisorCacheClearTask.
(cherry picked from commit 76f3060)


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

Branch: refs/heads/master
Commit: 7292a5306c90d5c48c520bf9ffad388769250488
Parents: 5f5cce4
Author: Andrey Novikov <an...@gridgain.com>
Authored: Mon Feb 20 18:23:33 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Thu Mar 2 10:56:21 2017 +0700

----------------------------------------------------------------------
 .../visor/cache/VisorCacheClearTask.java        | 88 +++++---------------
 .../visor/compute/VisorGatewayTask.java         | 30 ++++++-
 2 files changed, 49 insertions(+), 69 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7292a530/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
index 1f1a6fb..0c8476f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCacheClearTask.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.visor.cache;
 
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.internal.processors.task.GridInternal;
@@ -26,7 +25,6 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.visor.VisorJob;
 import org.apache.ignite.internal.visor.VisorOneNodeTask;
 import org.apache.ignite.lang.IgniteBiTuple;
-import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.resources.JobContextResource;
@@ -90,17 +88,11 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
         }
 
         /**
-         * @param subJob Sub job to execute asynchronously.
+         * @param fut Future for asynchronous cache operation.
          * @param idx Index.
          * @return {@code true} If subJob was not completed and this job should be suspended.
          */
-        private boolean callAsync(IgniteCallable<Integer> subJob, int idx) {
-            IgniteCompute compute = ignite.compute(ignite.cluster().forCacheNodes(cacheName)).withAsync();
-
-            compute.call(subJob);
-
-            IgniteFuture<Integer> fut = compute.future();
-
+        private boolean callAsync(IgniteFuture<Integer> fut, int idx) {
             futs[idx] = fut;
 
             if (fut.isDone())
@@ -119,16 +111,28 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
                 futs = new IgniteFuture[3];
 
             if (futs[0] == null || futs[1] == null || futs[2] == null) {
-                IgniteCache cache = ignite.cache(cacheName);
+                IgniteCache cache = ignite.cache(cacheName).withAsync();
+
+                if (futs[0] == null) {
+                    cache.size(CachePeekMode.PRIMARY);
+
+                    if (callAsync(cache.<Integer>future(), 0))
+                        return null;
+                }
 
-                if (futs[0] == null && callAsync(new VisorCacheSizeCallable(cache), 0))
-                    return null;
+                if (futs[1] == null) {
+                    cache.clear();
 
-                if (futs[1] == null && callAsync(new VisorCacheClearCallable(cache), 1))
-                    return null;
+                    if (callAsync(cache.<Integer>future(), 1))
+                        return null;
+                }
+                
+                if (futs[2] == null) {
+                    cache.size(CachePeekMode.PRIMARY);
 
-                if (futs[2] == null && callAsync(new VisorCacheSizeCallable(cache), 2))
-                    return null;
+                    if (callAsync(cache.<Integer>future(), 2))
+                        return null;
+                }
             }
 
             assert futs[0].isDone() && futs[1].isDone() && futs[2].isDone();
@@ -141,54 +145,4 @@ public class VisorCacheClearTask extends VisorOneNodeTask<String, IgniteBiTuple<
             return S.toString(VisorCacheClearJob.class, this);
         }
     }
-
-    /**
-     * Callable to get cache size.
-     */
-    @GridInternal
-    private static class VisorCacheSizeCallable implements IgniteCallable<Integer> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final IgniteCache cache;
-
-        /**
-         * @param cache Cache to take size from.
-         */
-        private VisorCacheSizeCallable(IgniteCache cache) {
-            this.cache = cache;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
-            return cache.size(CachePeekMode.PRIMARY);
-        }
-    }
-
-    /**
-     * Callable to clear cache.
-     */
-    @GridInternal
-    private static class VisorCacheClearCallable implements IgniteCallable<Integer> {
-        /** */
-        private static final long serialVersionUID = 0L;
-
-        /** */
-        private final IgniteCache cache;
-
-        /**
-         * @param cache Cache to clear.
-         */
-        private VisorCacheClearCallable(IgniteCache cache) {
-            this.cache = cache;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer call() throws Exception {
-            cache.clear();
-
-            return 0;
-        }
-    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/7292a530/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
index 2539a26..a64ec6d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorGatewayTask.java
@@ -29,21 +29,26 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
+import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.compute.ComputeJob;
 import org.apache.ignite.compute.ComputeJobAdapter;
+import org.apache.ignite.compute.ComputeJobContext;
 import org.apache.ignite.compute.ComputeJobResult;
 import org.apache.ignite.compute.ComputeJobResultPolicy;
 import org.apache.ignite.compute.ComputeTask;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.task.GridInternal;
 import org.apache.ignite.internal.util.lang.GridTuple3;
+import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.visor.VisorTaskArgument;
 import org.apache.ignite.lang.IgniteBiTuple;
+import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.resources.JobContextResource;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -101,9 +106,16 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
         @IgniteInstanceResource
         protected transient IgniteEx ignite;
 
+        /** Auto-inject job context. */
+        @JobContextResource
+        protected transient ComputeJobContext jobCtx;
+
         /** Arguments count. */
         private final int argsCnt;
 
+        /** Future for spawned task. */
+        private transient IgniteFuture fut;
+
         /**
          * Create job with specified argument.
          *
@@ -284,6 +296,9 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
         /** {@inheritDoc} */
         @SuppressWarnings("unchecked")
         @Override public Object execute() throws IgniteException {
+            if (fut != null)
+                return fut.get();
+
             String nidsArg = argument(0);
             String taskName = argument(1);
 
@@ -355,8 +370,19 @@ public class VisorGatewayTask implements ComputeTask<Object[], Object> {
                 }
             }
 
-            return ignite.compute(ignite.cluster().forNodeIds(nids))
-                .execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false));
+            IgniteCompute comp = ignite.compute(ignite.cluster().forNodeIds(nids)).withAsync();
+            
+            comp.execute(taskName, new VisorTaskArgument<>(nids, jobArgs, false));
+
+            fut = comp.future();
+
+            fut.listen(new CI1<IgniteFuture<Object>>() {
+                @Override public void apply(IgniteFuture<Object> f) {
+                    jobCtx.callcc();
+                }
+            });
+
+            return jobCtx.holdcc();
         }
     }
 }