You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2015/06/26 16:09:45 UTC

[1/4] incubator-ignite git commit: # GG-10404 Added check for empty topology for query cleanup task.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-7 6e23608fd -> 50a462643


# GG-10404 Added check for empty topology for query cleanup task.


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

Branch: refs/heads/ignite-sprint-7
Commit: 32697ab08071c708c9f8b4c1b70b2ee4ab62d858
Parents: f0e74c0
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 12 11:19:37 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 12 11:19:37 2015 +0700

----------------------------------------------------------------------
 .../connection/GridClientNioTcpConnection.java  |  7 ++++-
 .../visor/query/VisorQueryCleanupTask.java      | 14 +++++++++
 .../visor/util/VisorEmptyTopologyException.java | 33 ++++++++++++++++++++
 3 files changed, 53 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/32697ab0/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index d247e05..67709b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -750,7 +750,12 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             new GridClientFutureCallback<GridClientTaskResultBean, R>() {
                 @Override public R onComplete(GridClientFuture<GridClientTaskResultBean> fut)
                     throws GridClientException {
-                    return fut.get().getResult();
+                    GridClientTaskResultBean resBean = fut.get();
+
+                    if (resBean != null)
+                        return resBean.getResult();
+                    else
+                        return null;
                 }
             });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/32697ab0/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
index 722ad91..b9a55e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
@@ -22,6 +22,7 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.visor.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -47,6 +48,9 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
         @Nullable VisorTaskArgument<Map<UUID, Collection<String>>> arg) {
         Set<UUID> nodeIds = taskArg.keySet();
 
+        if (nodeIds.isEmpty())
+            throw new VisorEmptyTopologyException("Nothing to clear. List with node IDs is empty!");
+
         Map<ComputeJob, ClusterNode> map = U.newHashMap(nodeIds.size());
 
         try {
@@ -54,6 +58,16 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
                 if (nodeIds.contains(node.id()))
                     map.put(new VisorQueryCleanupJob(taskArg.get(node.id()), debug), node);
 
+            if (map.isEmpty()) {
+                String notFoundNodes = "";
+
+                for (UUID nid : nodeIds)
+                    notFoundNodes = notFoundNodes + (notFoundNodes.isEmpty() ? "" : ",")  + U.id8(nid);
+
+                throw new VisorEmptyTopologyException("Failed to clear query results. Nodes are not available: [" +
+                    notFoundNodes + "]");
+            }
+
             return map;
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/32697ab0/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
new file mode 100644
index 0000000..fda1bd7
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.util;
+
+import org.apache.ignite.*;
+
+/**
+ * Marker exception for indication of empty topology in Visor tasks.
+ */
+public class VisorEmptyTopologyException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** @inheritDoc */
+    public VisorEmptyTopologyException(String msg) {
+        super(msg);
+    }
+}


[2/4] incubator-ignite git commit: Merge branches 'ignite-gg-10404' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404

Posted by ak...@apache.org.
Merge branches 'ignite-gg-10404' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404


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

Branch: refs/heads/ignite-sprint-7
Commit: 67790714098c9c413a06b855c5b51e665a39e755
Parents: 32697ab b087aca
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 12 14:15:58 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 12 14:15:58 2015 +0700

----------------------------------------------------------------------
 .../processors/cache/GridCacheProcessor.java    |  17 ++
 .../continuous/GridContinuousProcessor.java     |  15 +-
 ...teStartCacheInTransactionAtomicSelfTest.java |  32 +++
 .../IgniteStartCacheInTransactionSelfTest.java  | 254 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite4.java       |   3 +
 5 files changed, 315 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[3/4] incubator-ignite git commit: # GG-10404 Suppress Visor exceptions.

Posted by ak...@apache.org.
# GG-10404 Suppress Visor exceptions.


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

Branch: refs/heads/ignite-sprint-7
Commit: c62c410bcf1a1f353fa586e515f846746cb4a482
Parents: 6779071
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 12 17:39:09 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 12 17:39:09 2015 +0700

----------------------------------------------------------------------
 .../connection/GridClientNioTcpConnection.java  |  7 +----
 .../processors/rest/GridRestProcessor.java      |  4 ++-
 .../handlers/task/GridTaskCommandHandler.java   | 12 ++++---
 .../processors/task/GridTaskWorker.java         |  4 ++-
 .../visor/query/VisorQueryCleanupTask.java      |  4 +--
 .../util/VisorClusterGroupEmptyException.java   | 33 ++++++++++++++++++++
 .../visor/util/VisorEmptyTopologyException.java | 33 --------------------
 7 files changed, 50 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
index 67709b8..d247e05 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientNioTcpConnection.java
@@ -750,12 +750,7 @@ public class GridClientNioTcpConnection extends GridClientConnection {
             new GridClientFutureCallback<GridClientTaskResultBean, R>() {
                 @Override public R onComplete(GridClientFuture<GridClientTaskResultBean> fut)
                     throws GridClientException {
-                    GridClientTaskResultBean resBean = fut.get();
-
-                    if (resBean != null)
-                        return resBean.getResult();
-                    else
-                        return null;
+                    return fut.get().getResult();
                 }
             });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 52ca610..2d1d802 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.plugin.security.SecurityException;
@@ -214,7 +215,8 @@ public class GridRestProcessor extends GridProcessorAdapter {
                     res = f.get();
                 }
                 catch (Exception e) {
-                    LT.error(log, e, "Failed to handle request: " + req.command());
+                    if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                        LT.error(log, e, "Failed to handle request: " + req.command());
 
                     if (log.isDebugEnabled())
                         log.debug("Failed to handle request [req=" + req + ", e=" + e + "]");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
index a647cd1..d832b21 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.jetbrains.annotations.*;
@@ -134,7 +135,8 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
             return handleAsyncUnsafe(req);
         }
         catch (IgniteCheckedException e) {
-            U.error(log, "Failed to execute task command: " + req, e);
+            if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                U.error(log, "Failed to execute task command: " + req, e);
 
             return new GridFinishedFuture<>(e);
         }
@@ -237,9 +239,11 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                                     U.warn(log, "Failed to execute task due to topology issues (are all mapped " +
                                         "nodes alive?) [name=" + name + ", clientId=" + req.clientId() +
                                         ", err=" + e + ']');
-                                else
-                                    U.error(log, "Failed to execute task [name=" + name + ", clientId=" +
-                                        req.clientId() + ']', e);
+                                else {
+                                    if (!X.hasCause(e, VisorClusterGroupEmptyException.class))
+                                        U.error(log, "Failed to execute task [name=" + name + ", clientId=" +
+                                            req.clientId() + ']', e);
+                                }
 
                                 desc = new TaskDescriptor(true, null, e);
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index f6d686c..eb5fa77 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
+import org.apache.ignite.internal.visor.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.resources.*;
@@ -443,7 +444,8 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         }
         catch (IgniteException | IgniteCheckedException e) {
             if (!fut.isCancelled()) {
-                U.error(log, "Failed to map task jobs to nodes: " + ses, e);
+                if (!(e instanceof VisorClusterGroupEmptyException))
+                    U.error(log, "Failed to map task jobs to nodes: " + ses, e);
 
                 finishTask(null, e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
index b9a55e1..5ceb300 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryCleanupTask.java
@@ -49,7 +49,7 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
         Set<UUID> nodeIds = taskArg.keySet();
 
         if (nodeIds.isEmpty())
-            throw new VisorEmptyTopologyException("Nothing to clear. List with node IDs is empty!");
+            throw new VisorClusterGroupEmptyException("Nothing to clear. List with node IDs is empty!");
 
         Map<ComputeJob, ClusterNode> map = U.newHashMap(nodeIds.size());
 
@@ -64,7 +64,7 @@ public class VisorQueryCleanupTask extends VisorMultiNodeTask<Map<UUID, Collecti
                 for (UUID nid : nodeIds)
                     notFoundNodes = notFoundNodes + (notFoundNodes.isEmpty() ? "" : ",")  + U.id8(nid);
 
-                throw new VisorEmptyTopologyException("Failed to clear query results. Nodes are not available: [" +
+                throw new VisorClusterGroupEmptyException("Failed to clear query results. Nodes are not available: [" +
                     notFoundNodes + "]");
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
new file mode 100644
index 0000000..b969178
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorClusterGroupEmptyException.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.visor.util;
+
+import org.apache.ignite.cluster.*;
+
+/**
+ * Exception to throw from Visor tasks in case of empty topology.
+ */
+public class VisorClusterGroupEmptyException extends ClusterGroupEmptyException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** @inheritDoc */
+    public VisorClusterGroupEmptyException(String msg) {
+        super(msg);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c62c410b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
deleted file mode 100644
index fda1bd7..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/util/VisorEmptyTopologyException.java
+++ /dev/null
@@ -1,33 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.visor.util;
-
-import org.apache.ignite.*;
-
-/**
- * Marker exception for indication of empty topology in Visor tasks.
- */
-public class VisorEmptyTopologyException extends IgniteException {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** @inheritDoc */
-    public VisorEmptyTopologyException(String msg) {
-        super(msg);
-    }
-}


[4/4] incubator-ignite git commit: Merge branches 'ignite-gg-10404' and 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404

Posted by ak...@apache.org.
Merge branches 'ignite-gg-10404' and 'ignite-sprint-7' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-gg-10404


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

Branch: refs/heads/ignite-sprint-7
Commit: 50a4626439afcf9c8113476da0a425d35409f153
Parents: c62c410 6e23608
Author: AKuznetsov <ak...@gridgain.com>
Authored: Fri Jun 26 20:53:15 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Fri Jun 26 20:53:15 2015 +0700

----------------------------------------------------------------------
 RELEASE_NOTES.txt                               |  12 +
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 .../s3/S3CheckpointManagerSelfTest.java         |   2 +-
 .../checkpoint/s3/S3CheckpointSpiSelfTest.java  |   4 +-
 .../s3/S3CheckpointSpiStartStopSelfTest.java    |   2 +-
 .../s3/S3SessionCheckpointSelfTest.java         |   2 +-
 .../s3/TcpDiscoveryS3IpFinderSelfTest.java      |   2 +-
 modules/clients/pom.xml                         |   2 +-
 .../ClientAbstractConnectivitySelfTest.java     |   4 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../apache/ignite/cache/query/ScanQuery.java    |  23 +-
 .../cache/store/jdbc/CacheJdbcBlobStore.java    |  22 +-
 .../store/jdbc/CacheJdbcBlobStoreFactory.java   | 290 +++++++
 .../cache/store/jdbc/CacheJdbcPojoStore.java    |   6 +-
 .../store/jdbc/CacheJdbcPojoStoreFactory.java   | 148 ++++
 .../org/apache/ignite/cluster/ClusterGroup.java |   9 +
 .../org/apache/ignite/cluster/ClusterNode.java  |   2 +
 .../ignite/compute/ComputeTaskSplitAdapter.java |   2 +-
 .../configuration/CacheConfiguration.java       |   3 +-
 .../configuration/IgniteReflectionFactory.java  |  81 +-
 .../ignite/internal/GridKernalContextImpl.java  |   8 +-
 .../apache/ignite/internal/IgniteKernal.java    |  11 +-
 .../internal/MarshallerContextAdapter.java      |  18 +-
 .../ignite/internal/MarshallerContextImpl.java  |  26 +-
 .../client/GridClientConfiguration.java         |   2 +-
 .../GridClientOptimizedMarshaller.java          |  26 +
 .../impl/GridTcpRouterNioListenerAdapter.java   |   2 +-
 .../internal/cluster/ClusterGroupAdapter.java   |  38 +
 .../cluster/IgniteClusterAsyncImpl.java         |   5 +
 .../internal/interop/InteropBootstrap.java      |   3 +-
 .../internal/interop/InteropIgnition.java       |   5 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  39 +-
 .../affinity/AffinityTopologyVersion.java       |   7 -
 .../processors/cache/GridCacheAdapter.java      |   4 +
 .../processors/cache/GridCacheContext.java      |   2 +-
 .../processors/cache/GridCacheIoManager.java    |  64 +-
 .../GridCachePartitionExchangeManager.java      |  77 +-
 .../processors/cache/GridCacheProcessor.java    |  34 +-
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../processors/cache/GridCacheUtils.java        |   9 +
 .../processors/cache/IgniteCacheProxy.java      |  12 +
 .../distributed/GridCacheTxRecoveryRequest.java |  26 +-
 .../GridCacheTxRecoveryResponse.java            |  14 +-
 .../distributed/GridDistributedBaseMessage.java |  77 +-
 .../distributed/GridDistributedLockRequest.java |  54 +-
 .../GridDistributedLockResponse.java            |  14 +-
 .../GridDistributedTxFinishRequest.java         |  46 +-
 .../GridDistributedTxPrepareRequest.java        |  62 +-
 .../GridDistributedTxPrepareResponse.java       |  64 +-
 .../GridDistributedUnlockRequest.java           |   6 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  59 +-
 .../distributed/dht/GridDhtLockFuture.java      |   2 +-
 .../distributed/dht/GridDhtLockRequest.java     |  72 +-
 .../distributed/dht/GridDhtLockResponse.java    |  18 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   4 +-
 .../dht/GridDhtPartitionsReservation.java       | 292 +++++++
 .../dht/GridDhtTransactionalCacheAdapter.java   |   2 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |  38 +-
 .../dht/GridDhtTxPrepareRequest.java            |  54 +-
 .../dht/GridDhtTxPrepareResponse.java           |  22 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |   6 +-
 .../cache/distributed/dht/GridReservable.java   |  35 +
 .../dht/atomic/GridDhtAtomicCache.java          |   9 +-
 .../dht/preloader/GridDhtPartitionMap.java      |  26 +-
 .../GridDhtPartitionsExchangeFuture.java        |  95 ++-
 .../dht/preloader/GridDhtPreloader.java         |   2 +-
 .../distributed/near/GridNearLockRequest.java   |  58 +-
 .../distributed/near/GridNearLockResponse.java  |  26 +-
 .../near/GridNearTxFinishRequest.java           |  26 +-
 .../near/GridNearTxPrepareRequest.java          |  50 +-
 .../near/GridNearTxPrepareResponse.java         |  46 +-
 .../distributed/near/GridNearUnlockRequest.java |   2 +-
 .../cache/query/GridCacheQueryManager.java      |  33 -
 .../cache/query/GridCacheTwoStepQuery.java      |  22 +-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../cache/transactions/IgniteTxHandler.java     |   5 +-
 .../transactions/IgniteTxLocalAdapter.java      |  12 +-
 .../datastructures/DataStructuresProcessor.java | 129 +--
 .../dr/IgniteDrDataStreamerCacheUpdater.java    |   7 +-
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../processors/plugin/CachePluginManager.java   |  10 +-
 .../plugin/IgnitePluginProcessor.java           |  16 +-
 .../processors/query/GridQueryIndexing.java     |  14 +-
 .../processors/query/GridQueryProcessor.java    | 123 ++-
 .../messages/GridQueryNextPageResponse.java     |  35 +-
 .../h2/twostep/messages/GridQueryRequest.java   | 111 ++-
 .../rest/client/message/GridRouterRequest.java  |  18 +
 .../rest/client/message/GridRouterResponse.java |  18 +
 .../rest/protocols/tcp/GridTcpRestProtocol.java |   3 +-
 .../processors/task/GridTaskProcessor.java      |  23 +-
 .../internal/util/GridConfigurationFinder.java  |  55 +-
 .../apache/ignite/internal/util/GridDebug.java  |  48 +-
 .../ignite/internal/util/IgniteUtils.java       |  27 +-
 .../ignite/internal/util/nio/GridNioServer.java |  64 +-
 .../util/spring/IgniteSpringHelper.java         |  10 +
 .../SpringApplicationContextResource.java       |   4 +-
 .../apache/ignite/resources/SpringResource.java |   6 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 148 +++-
 .../tcp/TcpCommunicationSpiMBean.java           |  19 +
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 657 +++++++++++-----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 376 ++++++---
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  69 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  51 +-
 .../tcp/internal/TcpDiscoveryNode.java          |  18 +
 .../ipfinder/TcpDiscoveryIpFinderAdapter.java   |  34 +-
 .../TcpDiscoveryMulticastIpFinder.java          |  19 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |  10 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 +
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../internal/ClusterForHostsSelfTest.java       | 113 +++
 .../internal/ClusterGroupAbstractTest.java      | 777 ++++++++++++++++++
 .../ignite/internal/ClusterGroupSelfTest.java   | 251 ++++++
 .../internal/GridDiscoveryEventSelfTest.java    |  12 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../internal/GridProjectionAbstractTest.java    | 784 -------------------
 .../ignite/internal/GridProjectionSelfTest.java | 251 ------
 .../apache/ignite/internal/GridSelfTest.java    |  30 +-
 .../GridTaskFailoverAffinityRunTest.java        | 170 ++++
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++
 .../GridDiscoveryManagerAliveCacheSelfTest.java |  17 +-
 .../cache/CacheClientStoreSelfTest.java         | 228 ++++++
 ...acheReadOnlyTransactionalClientSelfTest.java | 327 --------
 .../CacheReadThroughAtomicRestartSelfTest.java  |  32 +
 ...heReadThroughLocalAtomicRestartSelfTest.java |  32 +
 .../CacheReadThroughLocalRestartSelfTest.java   |  32 +
 ...dThroughReplicatedAtomicRestartSelfTest.java |  32 +
 ...cheReadThroughReplicatedRestartSelfTest.java |  32 +
 .../cache/CacheReadThroughRestartSelfTest.java  | 133 ++++
 .../CacheStoreUsageMultinodeAbstractTest.java   | 305 ++++++++
 ...eUsageMultinodeDynamicStartAbstractTest.java | 169 ++++
 ...oreUsageMultinodeDynamicStartAtomicTest.java |  32 +
 ...heStoreUsageMultinodeDynamicStartTxTest.java |  32 +
 ...reUsageMultinodeStaticStartAbstractTest.java | 158 ++++
 ...toreUsageMultinodeStaticStartAtomicTest.java |  32 +
 ...cheStoreUsageMultinodeStaticStartTxTest.java |  32 +
 .../GridCacheAbstractFailoverSelfTest.java      |   8 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  24 +-
 .../cache/GridCacheAbstractSelfTest.java        |   2 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++
 ...ridCacheMultinodeUpdateAbstractSelfTest.java |   9 +
 .../cache/GridCachePutAllFailoverSelfTest.java  |   5 -
 .../cache/GridCacheVersionMultinodeTest.java    |   8 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |  30 +-
 .../IgniteCacheAtomicStopBusySelfTest.java      |   8 +-
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  15 +-
 .../IgniteCacheP2pUnmarshallingTxErrorTest.java |  25 +-
 ...gniteCacheTransactionalStopBusySelfTest.java |   8 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  37 +
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |   4 +-
 .../GridCacheSetAbstractSelfTest.java           |  22 +-
 .../IgniteDataStructureWithJobTest.java         | 111 +++
 ...ridCachePartitionNotLoadedEventSelfTest.java |  82 ++
 .../distributed/IgniteCache150ClientsTest.java  | 189 +++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  14 +-
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../distributed/IgniteCacheManyClientsTest.java | 191 ++++-
 .../IgniteCacheTxMessageRecoveryTest.java       |   5 +
 .../dht/GridCacheColocatedFailoverSelfTest.java |   5 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |   5 -
 .../GridCachePartitionedFailoverSelfTest.java   |   5 -
 ...achePartitionedMultiNodeFullApiSelfTest.java |  53 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |  37 +-
 .../GridCacheReplicatedFailoverSelfTest.java    |   2 +-
 .../IgniteCacheTxStoreSessionTest.java          |   4 +
 ...CacheClientWriteBehindStoreAbstractTest.java | 104 +++
 ...teCacheClientWriteBehindStoreAtomicTest.java |  38 +
 .../IgnteCacheClientWriteBehindStoreTxTest.java |  32 +
 .../DataStreamProcessorSelfTest.java            |   3 +-
 .../DataStreamerMultiThreadedSelfTest.java      |   3 +
 .../internal/util/IgniteUtilsSelfTest.java      |  22 +
 .../marshaller/MarshallerContextTestImpl.java   |  18 +
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   1 -
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 338 +++++++-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 +-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../junits/GridTestKernalContext.java           |   3 +-
 .../junits/common/GridCommonAbstractTest.java   |  15 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   7 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |   1 +
 .../IgniteCacheFailoverTestSuite.java           |   8 -
 .../IgniteCacheFailoverTestSuite2.java          |  47 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   4 +-
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |  18 +-
 .../IgniteCacheWriteBehindTestSuite.java        |   2 +
 .../testsuites/IgniteClientTestSuite.java       |  38 +
 .../testsuites/IgniteComputeGridTestSuite.java  |   1 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |   2 +-
 .../ignite/util/TestTcpCommunicationSpi.java    |  21 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |  80 +-
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +-
 .../internal/processors/hadoop/HadoopUtils.java | 237 ------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemCacheUtils.java   | 241 ++++++
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 +-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 +-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 modules/hibernate/pom.xml                       |  16 +-
 .../hibernate/CacheHibernateBlobStore.java      |  87 +-
 .../CacheHibernateBlobStoreFactory.java         | 235 ++++++
 .../hibernate/src/test/config/factory-cache.xml |  59 ++
 .../src/test/config/factory-cache1.xml          |  61 ++
 .../config/factory-incorrect-store-cache.xml    |  56 ++
 .../CacheHibernateStoreFactorySelfTest.java     | 273 +++++++
 .../testsuites/IgniteHibernateTestSuite.java    |   2 +
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  81 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |  49 +-
 .../query/h2/twostep/GridMapQueryExecutor.java  | 321 ++++++--
 .../query/h2/twostep/GridMergeIndex.java        |  17 +-
 .../h2/twostep/GridMergeIndexUnsorted.java      |   7 +-
 .../h2/twostep/GridReduceQueryExecutor.java     | 650 ++++++++++++---
 .../query/h2/twostep/GridResultPage.java        |  21 +-
 .../CacheAbstractQueryMetricsSelfTest.java      | 205 +++++
 .../CachePartitionedQueryMetricsSelfTest.java   |  32 +
 .../CacheReplicatedQueryMetricsSelfTest.java    |  32 +
 .../cache/GridCacheCrossCacheQuerySelfTest.java |   3 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    | 206 -----
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   1 -
 ...QueryOffheapEvictsMultiThreadedSelfTest.java |   5 +
 ...lientQueryReplicatedNodeRestartSelfTest.java | 419 ++++++++++
 .../IgniteCacheQueryNodeRestartSelfTest.java    |  36 +-
 .../IgniteCacheQueryNodeRestartSelfTest2.java   | 383 +++++++++
 .../query/h2/sql/BaseH2CompareQueryTest.java    |   2 +-
 .../IgniteCacheQuerySelfTestSuite.java          |   9 +-
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   9 +-
 .../GridResourceSpringBeanInjector.java         |   2 +-
 .../util/spring/IgniteSpringHelperImpl.java     |  17 +
 .../src/test/config/incorrect-store-cache.xml   |  57 ++
 modules/spring/src/test/config/node.xml         |  43 +
 modules/spring/src/test/config/node1.xml        |  45 ++
 .../test/config/pojo-incorrect-store-cache.xml  |  56 ++
 modules/spring/src/test/config/store-cache.xml  |  59 ++
 modules/spring/src/test/config/store-cache1.xml |  62 ++
 .../jdbc/CacheJdbcBlobStoreFactorySelfTest.java | 172 ++++
 .../jdbc/CacheJdbcPojoStoreFactorySelfTest.java | 193 +++++
 .../testsuites/IgniteSpringTestSuite.java       |   5 +
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |  30 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console-2.10/pom.xml              |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../commands/cache/VisorCacheCommand.scala      |   7 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 .../IgniteWebSessionSelfTestSuite.java          |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 scripts/git-patch-prop.sh                       |   2 +-
 297 files changed, 12522 insertions(+), 3763 deletions(-)
----------------------------------------------------------------------