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 2015/09/07 22:46:28 UTC

[01/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-890

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.4 b1f119fd8 -> 35bc5ecaf
  refs/heads/ignite-695 a2b305d63 -> 2b2356d48
  refs/heads/ignite-788-dev b88a7563b -> 70275005f
  refs/heads/ignite-788-dev-review bd46365d0 -> 0c9bae449


Merge remote-tracking branch 'remotes/origin/master' into ignite-890


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

Branch: refs/heads/ignite-788-dev
Commit: e402e98e384f9d1de0d6e64591ecbc1a8464f22c
Parents: 86b8545 de7e06f
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 08:44:53 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 08:44:53 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/cache/CacheTypeMetadata.java  | 53 ++++++++++----------
 .../dht/atomic/GridDhtAtomicCache.java          |  4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 20 ++++++--
 ...eAtomicInvalidPartitionHandlingSelfTest.java |  2 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java | 15 ------
 5 files changed, 45 insertions(+), 49 deletions(-)
----------------------------------------------------------------------



[32/50] [abbrv] ignite git commit: GG-10559 - Improvements.

Posted by nt...@apache.org.
GG-10559 - Improvements.


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

Branch: refs/heads/ignite-788-dev
Commit: 692dd041af8c45347f193ce6c70bfeff59e1e178
Parents: c559692
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jul 16 18:18:26 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jul 16 18:18:26 2015 -0700

----------------------------------------------------------------------
 .../org/apache/ignite/plugin/security/SecurityPermission.java | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/692dd041/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
index 3cab511..0e660d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
+++ b/modules/core/src/main/java/org/apache/ignite/plugin/security/SecurityPermission.java
@@ -45,14 +45,17 @@ public enum SecurityPermission {
     /** Events {@code disable} permission. */
     EVENTS_DISABLE,
 
-    /** Common visor tasks permission. */
+    /** Common visor view tasks permission. */
     ADMIN_VIEW,
 
     /** Visor cache read (query) permission. */
     ADMIN_QUERY,
 
     /** Visor cache load permission. */
-    ADMIN_CACHE;
+    ADMIN_CACHE,
+
+    /** Visor admin operations permissions. */
+    ADMIN_OPS;
 
     /** Enumerated values. */
     private static final SecurityPermission[] VALS = values();


[39/50] [abbrv] ignite git commit: #ignite-1087: AffinityRun runs job on not primary nodes.

Posted by nt...@apache.org.
#ignite-1087: AffinityRun runs job on not primary nodes.


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

Branch: refs/heads/ignite-788-dev
Commit: f9d2a2ef4424bf166912594791b6414b9a9e8457
Parents: ede9612
Author: ivasilinets <iv...@gridgain.com>
Authored: Mon Jul 20 11:25:00 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Mon Jul 20 11:25:00 2015 +0300

----------------------------------------------------------------------
 .../ignite/compute/ComputeJobResultPolicy.java  |   3 +-
 .../failover/GridFailoverContextImpl.java       |  28 ++-
 .../managers/failover/GridFailoverManager.java  |  13 +-
 .../processors/closure/AffinityTask.java        |  35 ++++
 .../closure/GridClosureProcessor.java           |  63 ++++++-
 .../processors/task/GridTaskWorker.java         |  24 ++-
 .../ignite/spi/failover/FailoverContext.java    |  18 ++
 .../spi/failover/always/AlwaysFailoverSpi.java  |  25 +++
 .../cache/CacheAffinityCallSelfTest.java        | 172 +++++++++++++++++++
 .../cache/GridCacheAffinityRoutingSelfTest.java | 157 ++++++++++++++++-
 .../spi/failover/GridFailoverTestContext.java   |  10 ++
 .../ignite/testsuites/IgniteCacheTestSuite.java |   1 +
 12 files changed, 533 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResultPolicy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResultPolicy.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResultPolicy.java
index 37aba91..26eb542 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResultPolicy.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeJobResultPolicy.java
@@ -50,8 +50,7 @@ public enum ComputeJobResultPolicy {
      * @param ord Ordinal value.
      * @return Enumerated value.
      */
-    @Nullable
-    public static ComputeJobResultPolicy fromOrdinal(byte ord) {
+    @Nullable public static ComputeJobResultPolicy fromOrdinal(byte ord) {
         return ord >= 0 && ord < VALS.length ? VALS[ord] : null;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverContextImpl.java
index a3f8e44..c2b104e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverContextImpl.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.managers.loadbalancer.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.spi.failover.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -41,15 +42,26 @@ public class GridFailoverContextImpl implements FailoverContext {
     @GridToStringExclude
     private final GridLoadBalancerManager loadMgr;
 
+    /** Affinity key for affinityCall. */
+    private final Object affKey;
+
+    /** Affinity cache name for affinityCall. */
+    private final String affCacheName;
+
     /**
      * Initializes failover context.
      *
      * @param taskSes Grid task session.
      * @param jobRes Failed job result.
      * @param loadMgr Load manager.
+     * @param affKey Affinity key.
+     * @param affCacheName Affinity cache name.
      */
-    public GridFailoverContextImpl(GridTaskSessionImpl taskSes, ComputeJobResult jobRes,
-        GridLoadBalancerManager loadMgr) {
+    public GridFailoverContextImpl(GridTaskSessionImpl taskSes,
+        ComputeJobResult jobRes,
+        GridLoadBalancerManager loadMgr,
+        @Nullable Object affKey,
+        @Nullable String affCacheName) {
         assert taskSes != null;
         assert jobRes != null;
         assert loadMgr != null;
@@ -57,6 +69,8 @@ public class GridFailoverContextImpl implements FailoverContext {
         this.taskSes = taskSes;
         this.jobRes = jobRes;
         this.loadMgr = loadMgr;
+        this.affKey = affKey;
+        this.affCacheName = affCacheName;
     }
 
     /** {@inheritDoc} */
@@ -75,6 +89,16 @@ public class GridFailoverContextImpl implements FailoverContext {
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public Object affinityKey() {
+        return affKey;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public String affinityCacheName() {
+        return affCacheName;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridFailoverContextImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverManager.java
index 714cccb..dffc965 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/failover/GridFailoverManager.java
@@ -23,6 +23,7 @@ import org.apache.ignite.compute.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.*;
 import org.apache.ignite.spi.failover.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -56,11 +57,17 @@ public class GridFailoverManager extends GridManagerAdapter<FailoverSpi> {
     /**
      * @param taskSes Task session.
      * @param jobRes Job result.
-     * @param top Collection of all top nodes that does not include the failed node.
+     * @param top Collection of all topology nodes.
+     * @param affKey Affinity key.
+     * @param affCacheName Affinity cache name.
      * @return New node to route this job to.
      */
-    public ClusterNode failover(GridTaskSessionImpl taskSes, ComputeJobResult jobRes, List<ClusterNode> top) {
+    public ClusterNode failover(GridTaskSessionImpl taskSes,
+        ComputeJobResult jobRes,
+        List<ClusterNode> top,
+        @Nullable Object affKey,
+        @Nullable String affCacheName) {
         return getSpi(taskSes.getFailoverSpi()).failover(new GridFailoverContextImpl(taskSes, jobRes,
-            ctx.loadBalancing()), top);
+            ctx.loadBalancing(), affKey, affCacheName), top);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/AffinityTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/AffinityTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/AffinityTask.java
new file mode 100644
index 0000000..1b32444
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/AffinityTask.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.closure;
+
+import org.jetbrains.annotations.*;
+
+/**
+ * Affinity mapped task.
+ */
+public interface AffinityTask {
+    /**
+     * @return Affinity key.
+     */
+    public Object affinityKey();
+
+    /**
+     * @return Affinity cache name.
+     */
+    @Nullable public String affinityCacheName();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index 658557e..21bfc11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -413,9 +413,12 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
             final ClusterNode node = ctx.affinity().mapKeyToNode(cacheName, affKey0);
 
+            if (node == null)
+                return ComputeTaskInternalFuture.finishedFuture(ctx, T5.class, U.emptyTopologyException());
+
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
 
-            return ctx.task().execute(new T5(node, job), null, false);
+            return ctx.task().execute(new T5(node, job, affKey0, cacheName), null, false);
         }
         catch (IgniteCheckedException e) {
             return ComputeTaskInternalFuture.finishedFuture(ctx, T5.class, e);
@@ -445,9 +448,12 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
             final ClusterNode node = ctx.affinity().mapKeyToNode(cacheName, affKey0);
 
+            if (node == null)
+                return ComputeTaskInternalFuture.finishedFuture(ctx, T4.class, U.emptyTopologyException());
+
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
 
-            return ctx.task().execute(new T4(node, job), null, false);
+            return ctx.task().execute(new T4(node, job, affKey0, cacheName), null, false);
         }
         catch (IgniteCheckedException e) {
             return ComputeTaskInternalFuture.finishedFuture(ctx, T4.class, e);
@@ -1223,7 +1229,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
     /**
      */
-    private static class T4 extends TaskNoReduceAdapter<Void> implements GridNoImplicitInjection {
+    private static class T4 extends TaskNoReduceAdapter<Void> implements GridNoImplicitInjection, AffinityTask {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1233,15 +1239,27 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /** */
         private Runnable job;
 
+        /** */
+        private Object affKey;
+
+        /** */
+        private String affCacheName;
+
         /**
          * @param node Cluster node.
          * @param job Job.
+         * @param affKey Affinity key.
+         * @param affCacheName Affinity cache name.
          */
-        private T4(ClusterNode node, Runnable job) {
+        private T4(ClusterNode node, Runnable job, Object affKey, String affCacheName) {
             super(U.peerDeployAware0(job));
 
+            assert affKey != null;
+
             this.node = node;
             this.job = job;
+            this.affKey = affKey;
+            this.affCacheName = affCacheName;
         }
 
         /** {@inheritDoc} */
@@ -1250,11 +1268,22 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
             return Collections.singletonMap(job, node);
         }
+
+        /** {@inheritDoc} */
+        @Override public Object affinityKey() {
+            return affKey;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public String affinityCacheName() {
+            return affCacheName;
+        }
     }
 
     /**
      */
-    private static class T5<R> extends GridPeerDeployAwareTaskAdapter<Void, R> implements GridNoImplicitInjection {
+    private static class T5<R> extends GridPeerDeployAwareTaskAdapter<Void, R> implements
+        GridNoImplicitInjection, AffinityTask {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -1264,15 +1293,27 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /** */
         private Callable<R> job;
 
+        /** */
+        private Object affKey;
+
+        /** */
+        private String affCacheName;
+
         /**
          * @param node Cluster node.
          * @param job Job.
+         * @param affKey Affinity key.
+         * @param affCacheName Affinity cache name.
          */
-        private T5(ClusterNode node, Callable<R> job) {
+        private T5(ClusterNode node, Callable<R> job, Object affKey, String affCacheName) {
             super(U.peerDeployAware0(job));
 
+            assert affKey != null;
+
             this.node = node;
             this.job = job;
+            this.affKey = affKey;
+            this.affCacheName = affCacheName;
         }
 
         /** {@inheritDoc} */
@@ -1291,6 +1332,16 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
             throw new IgniteException("Failed to find successful job result: " + res);
         }
+
+        /** {@inheritDoc} */
+        @Override public Object affinityKey() {
+            return affKey;
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public String affinityCacheName() {
+            return affCacheName;
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/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 133a31f..f241bcc 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
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.compute.*;
 import org.apache.ignite.internal.managers.deployment.*;
+import org.apache.ignite.internal.processors.closure.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -136,6 +137,12 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
     private final boolean noFailover;
 
     /** */
+    private final Object affKey;
+
+    /** */
+    private final String affCache;
+
+    /** */
     private final UUID subjId;
 
     /** Continuous mapper. */
@@ -245,6 +252,17 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
         Boolean noFailover = getThreadContext(TC_NO_FAILOVER);
 
         this.noFailover = noFailover != null ? noFailover : false;
+
+        if (task instanceof AffinityTask) {
+            AffinityTask affTask = (AffinityTask)task;
+
+            affKey = affTask.affinityKey();
+            affCache = affTask.affinityCacheName();
+        }
+        else {
+            affKey = null;
+            affCache = null;
+        }
     }
 
     /**
@@ -397,7 +415,9 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
 
             ses.setClassLoader(dep.classLoader());
 
-            final List<ClusterNode> shuffledNodes = getTaskTopology();
+            // Nodes are ignored by affinity tasks.
+            final List<ClusterNode> shuffledNodes =
+                affKey == null ? getTaskTopology() : Collections.<ClusterNode>emptyList();
 
             // Load balancer.
             ComputeLoadBalancer balancer = ctx.loadBalancing().getLoadBalancer(ses, shuffledNodes);
@@ -968,7 +988,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
             ctx.resource().invokeAnnotated(dep, jobRes.getJob(), ComputeJobBeforeFailover.class);
 
             // Map to a new node.
-            ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top));
+            ClusterNode node = ctx.failover().failover(ses, jobRes, new ArrayList<>(top), affKey, affCache);
 
             if (node == null) {
                 String msg = "Failed to failover a job to another node (failover SPI returned null) [job=" +

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
index b0cae92..865f1a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/FailoverContext.java
@@ -20,6 +20,8 @@ package org.apache.ignite.spi.failover;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -52,4 +54,20 @@ public interface FailoverContext {
      * @throws IgniteException If anything failed.
      */
     public ClusterNode getBalancedNode(List<ClusterNode> top) throws IgniteException;
+
+    /**
+     * Gets affinity key for {@link IgniteCompute#affinityRun(String, Object, IgniteRunnable)}
+     * and {@link IgniteCompute#affinityCall(String, Object, IgniteCallable)}.
+     *
+     * @return Affinity key.
+     */
+    @Nullable public Object affinityKey();
+
+    /**
+     * Returns affinity cache name {@link IgniteCompute#affinityRun(String, Object, IgniteRunnable)}
+     * and {@link IgniteCompute#affinityCall(String, Object, IgniteCallable)}.
+     *
+     * @return Cache name.
+     */
+    @Nullable public String affinityCacheName();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
index e075d3e..e925995 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
@@ -92,6 +92,11 @@ public class AlwaysFailoverSpi extends IgniteSpiAdapter implements FailoverSpi,
      */
     public static final String FAILED_NODE_LIST_ATTR = "gg:failover:failednodelist";
 
+    /**
+     * Name of job context attribute containing number of affinity call attempts.
+     */
+    public static final String AFFINITY_CALL_ATTEMPT = "ignite:failover:affinitycallattempt";
+
     /** Maximum attempts attribute key should be the same on all nodes. */
     public static final String MAX_FAILOVER_ATTEMPT_ATTR = "gg:failover:maxattempts";
 
@@ -173,6 +178,26 @@ public class AlwaysFailoverSpi extends IgniteSpiAdapter implements FailoverSpi,
             return null;
         }
 
+        if (ctx.affinityKey() != null) {
+            Integer affCallAttempt = ctx.getJobResult().getJobContext().getAttribute(AFFINITY_CALL_ATTEMPT);
+
+            if (affCallAttempt == null)
+                affCallAttempt = 1;
+
+            if (maxFailoverAttempts <= affCallAttempt) {
+                U.warn(log, "Job failover failed because number of maximum failover attempts for affinity call" +
+                    " is exceeded [failedJob=" + ctx.getJobResult().getJob() + ", maxFailoverAttempts=" +
+                    maxFailoverAttempts + ']');
+
+                return null;
+            }
+            else {
+                ctx.getJobResult().getJobContext().setAttribute(AFFINITY_CALL_ATTEMPT, affCallAttempt + 1);
+
+                return ignite.affinity(ctx.affinityCacheName()).mapKeyToNode(ctx.affinityKey());
+            }
+        }
+
         Collection<UUID> failedNodes = ctx.getJobResult().getJobContext().getAttribute(FAILED_NODE_LIST_ATTR);
 
         if (failedNodes == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
new file mode 100644
index 0000000..c4436ca
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheAffinityCallSelfTest.java
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.failover.always.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for {@link IgniteCompute#affinityCall(String, Object, IgniteCallable)} and
+ * {@link IgniteCompute#affinityRun(String, Object, IgniteRunnable)}.
+ */
+public class CacheAffinityCallSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE_NAME = "myCache";
+
+    /** */
+    private static final int MAX_FAILOVER_ATTEMPTS = 5;
+
+    /** */
+    private static final int SERVERS_COUNT = 4;
+
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi spi = new TcpDiscoverySpi();
+
+        spi.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(spi);
+
+        AlwaysFailoverSpi failSpi = new AlwaysFailoverSpi();
+        failSpi.setMaximumFailoverAttempts(MAX_FAILOVER_ATTEMPTS);
+        cfg.setFailoverSpi(failSpi);
+
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+        ccfg.setName(CACHE_NAME);
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        if (gridName.equals(getTestGridName(SERVERS_COUNT)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCallRestartNode() throws Exception {
+        startGrids(4);
+
+        Integer key = primaryKey(grid(0).cache(CACHE_NAME));
+
+        IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                U.sleep(500);
+                stopGrid(0);
+
+                return null;
+            }
+        });
+
+        while (!fut.isDone())
+            grid(1).compute().affinityCall(CACHE_NAME, key, new CheckCallable(key));
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCallNoServerNode() throws Exception {
+        startGrids(SERVERS_COUNT + 1);
+
+        final Integer key = 1;
+
+        final Ignite client = grid(SERVERS_COUNT);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < SERVERS_COUNT; ++i)
+                    stopGrid(i);
+
+                return null;
+            }
+        });
+
+        try {
+            while (!fut.isDone())
+                client.compute().affinityCall(CACHE_NAME, key, new CheckCallable(key));
+        }
+        catch (ComputeTaskCancelledException e) {
+            assertTrue(e.getMessage().contains("stopping"));
+        }
+        catch(ClusterGroupEmptyException e) {
+            assertTrue(e.getMessage().contains("Topology projection is empty"));
+        }
+        catch(IgniteException e) {
+            assertTrue(e.getMessage().contains("cache (or node) is stopping"));
+        }
+
+        stopGrid(SERVERS_COUNT);
+    }
+
+    /**
+     * Test callable.
+     */
+    public static class CheckCallable implements IgniteCallable<Object> {
+        /** Key. */
+        private final Object key;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param key Key.
+         */
+        public CheckCallable(Object key) {
+            this.key = key;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws IgniteCheckedException {
+            assert ignite.cluster().localNode().id().equals(ignite.cluster().mapKeyToNode(CACHE_NAME, key).id());
+
+            return null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java
index 78ecf08..a56ab9f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityRoutingSelfTest.java
@@ -19,17 +19,21 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.failover.always.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
+import java.util.concurrent.*;
+
 import static org.apache.ignite.cache.CacheMode.*;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
 
@@ -47,6 +51,9 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest {
     private static final int KEY_CNT = 50;
 
     /** */
+    private static final int MAX_FAILOVER_ATTEMPTS = 5;
+
+    /** */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
 
     /**
@@ -66,6 +73,10 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest {
 
         cfg.setDiscoverySpi(spi);
 
+        AlwaysFailoverSpi failSpi = new AlwaysFailoverSpi();
+        failSpi.setMaximumFailoverAttempts(MAX_FAILOVER_ATTEMPTS);
+        cfg.setFailoverSpi(failSpi);
+
         if (!gridName.equals(getTestGridName(GRID_CNT))) {
             // Default cache configuration.
             CacheConfiguration dfltCacheCfg = defaultCacheConfiguration();
@@ -129,6 +140,48 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCallRestartFails() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                grid(0).compute().affinityCall(NON_DFLT_CACHE_NAME, "key",
+                    new FailedCallable("key", MAX_FAILOVER_ATTEMPTS + 1));
+                return null;
+            }
+        }, ClusterTopologyException.class, "Failed to failover a job to another node");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityCallRestart() throws Exception {
+        assertEquals(MAX_FAILOVER_ATTEMPTS,
+            grid(0).compute().affinityCall(NON_DFLT_CACHE_NAME, "key",
+                new FailedCallable("key", MAX_FAILOVER_ATTEMPTS)));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityRunRestartFails() throws Exception {
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                grid(0).compute().affinityRun(NON_DFLT_CACHE_NAME, "key",
+                    new FailedRunnable("key", MAX_FAILOVER_ATTEMPTS + 1));
+                return null;
+            }
+        }, ClusterTopologyException.class, "Failed to failover a job to another node");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAffinityRunRestart() throws Exception {
+        grid(0).compute().affinityRun(NON_DFLT_CACHE_NAME, "key", new FailedRunnable("key", MAX_FAILOVER_ATTEMPTS));
+    }
+
+    /**
      * JUnit.
      *
      * @throws Exception If failed.
@@ -224,6 +277,108 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * Test runnable.
+     */
+    private static class FailedCallable implements IgniteCallable<Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private static final String ATTR_ATTEMPT = "Attempt";
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** */
+        @JobContextResource
+        private ComputeJobContext jobCtx;
+
+        /** Key. */
+        private final Object key;
+
+        /** Call attempts. */
+        private final Integer callAttempt;
+
+        /**
+         * @param key Key.
+         * @param callAttempt Call attempts.
+         */
+        public FailedCallable(Object key, Integer callAttempt) {
+            this.key = key;
+            this.callAttempt = callAttempt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws IgniteCheckedException {
+            Integer attempt = jobCtx.getAttribute(ATTR_ATTEMPT);
+
+            if (attempt == null)
+                attempt = 1;
+
+            assertEquals(ignite.affinity(NON_DFLT_CACHE_NAME).mapKeyToNode(key), ignite.cluster().localNode());
+
+            jobCtx.setAttribute(ATTR_ATTEMPT, attempt + 1);
+
+            if (attempt < callAttempt)
+                throw new ComputeJobFailoverException("Failover exception.");
+            else
+                return attempt;
+        }
+    }
+
+    /**
+     * Test runnable.
+     */
+    private static class FailedRunnable implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private static final String ATTR_ATTEMPT = "Attempt";
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** */
+        @JobContextResource
+        private ComputeJobContext jobCtx;
+
+        /** Key. */
+        private final Object key;
+
+        /** Call attempts. */
+        private final Integer callAttempt;
+
+        /**
+         * @param key Key.
+         * @param callAttempt Call attempts.
+         */
+        public FailedRunnable(Object key, Integer callAttempt) {
+            this.key = key;
+            this.callAttempt = callAttempt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            Integer attempt = jobCtx.getAttribute(ATTR_ATTEMPT);
+
+            if (attempt == null)
+                attempt = 1;
+
+            assertEquals(ignite.affinity(NON_DFLT_CACHE_NAME).mapKeyToNode(key), ignite.cluster().localNode());
+
+            jobCtx.setAttribute(ATTR_ATTEMPT, attempt + 1);
+
+            if (attempt < callAttempt)
+                throw new ComputeJobFailoverException("Failover exception.");
+            else
+                assertEquals(callAttempt, attempt);
+        }
+    }
+
+    /**
      * Test callable.
      */
     private static class CheckCallable implements IgniteCallable<Object> {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/modules/core/src/test/java/org/apache/ignite/spi/failover/GridFailoverTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/failover/GridFailoverTestContext.java b/modules/core/src/test/java/org/apache/ignite/spi/failover/GridFailoverTestContext.java
index db64475..bfca83d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/failover/GridFailoverTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/failover/GridFailoverTestContext.java
@@ -66,4 +66,14 @@ public class GridFailoverTestContext implements FailoverContext {
     @Override public ClusterNode getBalancedNode(List<ClusterNode> grid) {
         return grid.get(RAND.nextInt(grid.size()));
     }
+
+    /** {@inheritDoc} */
+    @Override public Object affinityKey() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String affinityCacheName() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f9d2a2ef/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 39702a3..bafdfef 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -109,6 +109,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         // Common tests.
         suite.addTestSuite(GridCacheConcurrentMapSelfTest.class);
         suite.addTestSuite(GridCacheAffinityMapperSelfTest.class);
+        suite.addTestSuite(CacheAffinityCallSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, GridCacheAffinityRoutingSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheMvccSelfTest.class, ignoredTests);
         suite.addTestSuite(GridCacheMvccPartitionedSelfTest.class);


[49/50] [abbrv] ignite git commit: Merge branch 'master' into ignite-788-dev

Posted by nt...@apache.org.
Merge branch 'master' into ignite-788-dev


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

Branch: refs/heads/ignite-788-dev
Commit: 70275005f5b084d063de62a98de49a996e9731c5
Parents: b88a756 3194415
Author: ashutak <as...@gridgain.com>
Authored: Mon Jul 20 17:57:24 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jul 20 17:57:24 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |    8 +
 RELEASE_NOTES.txt                               |    2 +
 assembly/LICENSE_FABRIC                         |  306 +++++
 assembly/LICENSE_HADOOP                         |  259 ++++
 assembly/NOTICE_FABRIC                          |   13 +
 assembly/NOTICE_HADOOP                          |   12 +
 assembly/dependencies-fabric.xml                |   13 +
 assembly/dependencies-hadoop.xml                |   12 +
 assembly/dependencies-visor-console.xml         |   20 +-
 assembly/release-base.xml                       |   10 -
 assembly/release-fabric.xml                     |   12 +
 assembly/release-hadoop.xml                     |   12 +
 examples/pom.xml                                |    2 +-
 modules/aop/licenses/aspectj-epl-license.txt    |   69 -
 modules/aop/pom.xml                             |    2 +-
 modules/apache-license-gen/pom.xml              |   48 +
 .../src/main/resources/META-INF/licenses.txt.vm |   42 +
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/licenses/jsr166-license.txt        |    3 +
 modules/core/licenses/snaptree-bsd-license.txt  |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../IgniteClientDisconnectedException.java      |   61 +
 .../java/org/apache/ignite/IgniteCluster.java   |    8 +
 .../apache/ignite/cache/CacheTypeMetadata.java  |   53 +-
 .../ignite/compute/ComputeJobResultPolicy.java  |    3 +-
 .../apache/ignite/internal/GridComponent.java   |   18 +
 .../ignite/internal/GridJobSiblingImpl.java     |    2 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |   31 +-
 .../ignite/internal/GridKernalGateway.java      |   46 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   85 +-
 .../apache/ignite/internal/GridKernalState.java |    3 +
 .../ignite/internal/GridPluginComponent.java    |   11 +
 ...gniteClientDisconnectedCheckedException.java |   49 +
 .../apache/ignite/internal/IgniteKernal.java    |  234 +++-
 .../cluster/IgniteClusterAsyncImpl.java         |    5 +
 .../internal/cluster/IgniteClusterImpl.java     |   18 +
 .../internal/managers/GridManagerAdapter.java   |   19 +-
 .../deployment/GridDeploymentCommunication.java |    2 +-
 .../deployment/GridDeploymentManager.java       |   95 +-
 .../discovery/GridDiscoveryManager.java         |  188 ++-
 .../failover/GridFailoverContextImpl.java       |   28 +-
 .../managers/failover/GridFailoverManager.java  |   13 +-
 .../processors/GridProcessorAdapter.java        |   11 +
 .../affinity/GridAffinityAssignmentCache.java   |   26 +-
 .../cache/CacheOsConflictResolutionManager.java |    6 +
 .../cache/DynamicCacheChangeBatch.java          |   17 +
 .../processors/cache/GridCacheAdapter.java      |   27 +-
 .../cache/GridCacheAffinityManager.java         |   21 +-
 .../cache/GridCacheConcurrentMap.java           |   15 +-
 .../processors/cache/GridCacheContext.java      |   55 +-
 .../processors/cache/GridCacheGateway.java      |  116 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +
 .../processors/cache/GridCacheManager.java      |    6 +
 .../cache/GridCacheManagerAdapter.java          |    6 +
 .../processors/cache/GridCacheMvccManager.java  |   43 +-
 .../GridCachePartitionExchangeManager.java      |  134 +-
 .../processors/cache/GridCachePreloader.java    |    5 +
 .../cache/GridCachePreloaderAdapter.java        |    5 +
 .../processors/cache/GridCacheProcessor.java    |  311 ++++-
 .../cache/GridCacheSharedContext.java           |  113 +-
 .../cache/GridCacheSharedManager.java           |   11 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   20 +-
 .../processors/cache/GridCacheUtils.java        |    9 +
 .../processors/cache/IgniteCacheFutureImpl.java |   11 +
 .../processors/cache/IgniteCacheProxy.java      |   59 +-
 .../processors/cache/IgniteInternalCache.java   |    1 +
 .../CacheDataStructuresManager.java             |   35 +
 .../distributed/GridCacheTxFinishSync.java      |   46 +
 .../distributed/dht/GridDhtCacheAdapter.java    |   14 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   24 +
 .../distributed/dht/GridDhtTopologyFuture.java  |   14 -
 .../distributed/dht/GridDhtTxPrepareFuture.java |    9 +-
 .../dht/GridPartitionedGetFuture.java           |   13 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    6 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   31 +-
 .../GridDhtPartitionsExchangeFuture.java        |   31 +-
 .../dht/preloader/GridDhtPreloader.java         |   16 +-
 .../distributed/near/GridNearCacheAdapter.java  |    8 +
 .../distributed/near/GridNearGetFuture.java     |   13 +-
 .../cache/dr/GridOsCacheDrManager.java          |    7 +-
 .../query/GridCacheDistributedQueryManager.java |   22 +
 .../cache/query/GridCacheQueryAdapter.java      |   11 +-
 .../query/GridCacheQueryFutureAdapter.java      |    2 +-
 .../continuous/CacheContinuousQueryHandler.java |    5 +
 .../transactions/IgniteTransactionsImpl.java    |   59 +-
 .../cache/transactions/IgniteTxManager.java     |   23 +-
 .../transactions/TransactionProxyImpl.java      |    2 +-
 .../cache/version/GridCacheVersionManager.java  |    9 +-
 .../clock/GridClockSyncProcessor.java           |    6 +-
 .../processors/closure/AffinityTask.java        |   35 +
 .../closure/GridClosureProcessor.java           |   63 +-
 .../processors/cluster/ClusterProcessor.java    |   11 +
 .../continuous/GridContinuousHandler.java       |    9 +-
 .../continuous/GridContinuousProcessor.java     |  127 +-
 .../datastreamer/DataStreamProcessor.java       |   24 +-
 .../datastreamer/DataStreamerImpl.java          |   90 +-
 .../datastructures/DataStructuresProcessor.java |   33 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   33 +-
 .../GridCacheAtomicReferenceImpl.java           |   34 +-
 .../GridCacheAtomicSequenceImpl.java            |   33 +-
 .../GridCacheAtomicStampedImpl.java             |   33 +-
 .../GridCacheCountDownLatchImpl.java            |   51 +-
 .../datastructures/GridCacheRemovable.java      |    6 +-
 .../datastructures/GridCacheSetImpl.java        |   15 +-
 .../datastructures/GridCacheSetProxy.java       |   47 +-
 .../processors/job/GridJobProcessor.java        |    2 +-
 .../internal/processors/job/GridJobWorker.java  |    2 +-
 .../processors/query/GridQueryIndexing.java     |    7 +
 .../processors/query/GridQueryProcessor.java    |   10 +-
 .../service/GridServiceProcessor.java           |   45 +-
 .../processors/service/GridServiceProxy.java    |   13 +-
 .../processors/task/GridTaskProcessor.java      |   55 +-
 .../processors/task/GridTaskWorker.java         |   83 +-
 .../ignite/internal/util/IgniteUtils.java       |   30 +-
 .../util/future/GridFutureChainListener.java    |    4 -
 .../internal/util/future/IgniteFutureImpl.java  |   12 +-
 .../shmem/IpcSharedMemoryClientEndpoint.java    |    5 +-
 .../ignite/internal/util/lang/GridFunc.java     |    2 +
 .../plugin/security/SecurityPermission.java     |    7 +-
 .../plugin/security/SecuritySubjectType.java    |    3 +-
 .../java/org/apache/ignite/spi/IgniteSpi.java   |   15 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   51 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  356 ++++--
 .../spi/discovery/DiscoverySpiDataExchange.java |    3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  408 ++++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  177 ++-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    9 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   33 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   19 +
 .../messages/TcpDiscoveryAbstractMessage.java   |    3 +
 .../messages/TcpDiscoveryClientAckResponse.java |   64 +
 .../messages/TcpDiscoveryHandshakeResponse.java |   14 +
 .../ignite/spi/failover/FailoverContext.java    |   18 +
 .../spi/failover/always/AlwaysFailoverSpi.java  |   25 +
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    2 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   15 +-
 .../IgniteClientReconnectAbstractTest.java      |  363 ++++++
 .../IgniteClientReconnectApiExceptionTest.java  |  846 ++++++++++++
 .../IgniteClientReconnectAtomicsTest.java       |  672 ++++++++++
 .../IgniteClientReconnectCacheTest.java         | 1202 ++++++++++++++++++
 .../IgniteClientReconnectCollectionsTest.java   |  443 +++++++
 .../IgniteClientReconnectComputeTest.java       |  192 +++
 ...eClientReconnectContinuousProcessorTest.java |  372 ++++++
 ...IgniteClientReconnectDiscoveryStateTest.java |  123 ++
 ...niteClientReconnectFailoverAbstractTest.java |  231 ++++
 .../IgniteClientReconnectFailoverTest.java      |  227 ++++
 .../IgniteClientReconnectServicesTest.java      |  260 ++++
 .../internal/IgniteClientReconnectStopTest.java |  106 ++
 .../IgniteClientReconnectStreamerTest.java      |  233 ++++
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 .../GridDeploymentManagerStopSelfTest.java      |    7 +
 .../cache/CacheAffinityCallSelfTest.java        |  172 +++
 .../cache/GridCacheAbstractFullApiSelfTest.java |   32 +
 .../cache/GridCacheAffinityRoutingSelfTest.java |  157 ++-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    2 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |    6 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    2 +
 .../IgniteTxExceptionAbstractSelfTest.java      |    1 +
 .../GridCacheAbstractNodeRestartSelfTest.java   |   12 +-
 .../distributed/IgniteCache150ClientsTest.java  |    1 +
 .../IgniteCacheClientReconnectTest.java         |  175 +++
 .../IgniteCacheServerNodeConcurrentStart.java   |   96 ++
 .../IgniteCacheSystemTransactionsSelfTest.java  |    2 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    |   52 +-
 ...gniteCachePutRetryTransactionalSelfTest.java |   17 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    2 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |    4 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |    3 +-
 ...acheAtomicReplicatedNodeRestartSelfTest.java |   14 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  562 +++++++-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |  140 +-
 .../spi/failover/GridFailoverTestContext.java   |   10 +
 .../testframework/junits/GridAbstractTest.java  |    5 +
 .../multijvm/IgniteClusterProcessProxy.java     |    5 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    1 +
 .../testsuites/IgniteCacheTestSuite2.java       |    2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 -
 .../testsuites/IgniteClientNodesTestSuite.java  |   42 +
 .../IgniteClientReconnectTestSuite.java         |   48 +
 modules/extdata/p2p/pom.xml                     |    2 +-
 .../extdata/uri/modules/uri-dependency/pom.xml  |    6 +-
 modules/extdata/uri/pom.xml                     |   26 +-
 modules/gce/pom.xml                             |    2 +-
 .../geospatial/licenses/jts-lgpl-license.txt    |  165 ---
 modules/geospatial/pom.xml                      |    2 +-
 modules/hadoop/pom.xml                          |    2 +-
 .../licenses/hibernate-lgpl-2.1-license.txt     |  174 ---
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/licenses/h2-epl-license.txt    |   69 -
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |    5 +
 .../query/h2/twostep/GridMergeIndex.java        |   45 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   70 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |  225 ++++
 .../cache/IgniteClientReconnectQueriesTest.java |  427 +++++++
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |    4 +
 .../IgniteCacheWithIndexingTestSuite.java       |    1 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/licenses/jta-license.txt            |    2 +
 modules/jta/pom.xml                             |    2 +-
 modules/kafka/pom.xml                           |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/mesos/licenses/jetty-epl-license.txt    |   69 -
 modules/mesos/pom.xml                           |    3 +-
 .../apache/ignite/mesos/ClusterProperties.java  |   14 +
 .../apache/ignite/mesos/IgniteScheduler.java    |   26 +-
 modules/rest-http/pom.xml                       |    8 +-
 .../scalar-2.10/licenses/scala-bsd-license.txt  |   18 -
 modules/scalar-2.10/pom.xml                     |    2 +-
 modules/scalar/licenses/scala-bsd-license.txt   |   18 -
 modules/scalar/pom.xml                          |    2 +-
 .../licenses/cron4j-lgpl-2.1-license.txt        |  174 ---
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 .../ignite/schema/ui/SchemaImportApp.java       |   36 +-
 modules/slf4j/licenses/sl4j-mit-license.txt     |   21 -
 modules/slf4j/pom.xml                           |    2 +-
 .../spark-2.10/licenses/scala-bsd-license.txt   |   18 -
 modules/spark-2.10/pom.xml                      |    2 +-
 modules/spark/licenses/scala-bsd-license.txt    |   18 -
 modules/spark/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 .../apache/ignite/cache/spring/SpringCache.java |    2 +-
 .../ignite/cache/spring/SpringCacheManager.java |   22 +-
 modules/ssh/licenses/jcraft-revised-bsd.txt     |   28 -
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/licenses/jodd-revised-bsd.txt     |   21 -
 modules/tools/pom.xml                           |    2 +-
 .../urideploy/licenses/jtidy-mit-license.txt    |   50 -
 modules/urideploy/pom.xml                       |    8 +-
 modules/visor-console-2.10/pom.xml              |    2 +-
 .../licenses/jline-bsd-license.txt              |   18 -
 .../licenses/scala-bsd-license.txt              |   18 -
 modules/visor-console/pom.xml                   |    2 +-
 .../licenses/slf4j-mit-license.txt              |   21 -
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    8 +-
 modules/yardstick/pom.xml                       |    2 +-
 modules/yarn/README.txt                         |   28 +
 modules/yarn/licenses/apache-2.0.txt            |  202 +++
 modules/yarn/pom.xml                            |  101 ++
 .../apache/ignite/yarn/ApplicationMaster.java   |  400 ++++++
 .../apache/ignite/yarn/ClusterProperties.java   |  432 +++++++
 .../org/apache/ignite/yarn/IgniteContainer.java |   84 ++
 .../org/apache/ignite/yarn/IgniteProvider.java  |  339 +++++
 .../apache/ignite/yarn/IgniteYarnClient.java    |  178 +++
 .../org/apache/ignite/yarn/package-info.java    |   22 +
 .../ignite/yarn/utils/IgniteYarnUtils.java      |   81 ++
 .../main/resources/ignite-default-config.xml    |   35 +
 .../org/apache/ignite/IgniteYarnTestSuite.java  |   38 +
 .../yarn/IgniteApplicationMasterSelfTest.java   |  532 ++++++++
 parent/pom.xml                                  |   55 +
 pom.xml                                         |    6 +-
 259 files changed, 14502 insertions(+), 2180 deletions(-)
----------------------------------------------------------------------



[13/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1085'

Posted by nt...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1085'


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

Branch: refs/heads/ignite-788-dev
Commit: 77da7288f61868cf344fde68ced65cfe747d765b
Parents: cd3a9b6 d11fc9b
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 16 09:05:06 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 16 09:05:06 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         | 35 ++++++++++++--------
 1 file changed, 22 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[03/50] [abbrv] ignite git commit: ignite-1085: reimplemented

Posted by nt...@apache.org.
ignite-1085: reimplemented


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

Branch: refs/heads/ignite-788-dev
Commit: 24aae82fabfebcbeb13516c3f3361ac5d726d20f
Parents: 7147f3d
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 13:53:53 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 13:53:53 2015 +0300

----------------------------------------------------------------------
 .../managers/discovery/GridDiscoveryManager.java | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/24aae82f/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index eae07ed..1bac4d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -376,6 +376,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             });
         }
 
+        final CountDownLatch startLatch = new CountDownLatch(1);
+
         spi.setListener(new DiscoverySpiListener() {
             @Override public void onDiscovery(
                 int type,
@@ -464,6 +466,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                 // If this is a local join event, just save it and do not notify listeners.
                 if (type == EVT_NODE_JOINED && node.id().equals(locNode.id())) {
+                    updateTopologyVersionIfGreater(new AffinityTopologyVersion(locNode.order()),
+                        new DiscoCache(localNode(), getSpi().getRemoteNodes()));
+
+                    assert startLatch.getCount() == 1;
+
+                    startLatch.countDown();
+
                     DiscoveryEvent discoEvt = new DiscoveryEvent();
 
                     discoEvt.node(ctx.discovery().localNode());
@@ -527,6 +536,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         startSpi();
 
+        try {
+            U.await(startLatch);
+        }
+        catch (IgniteInterruptedException e) {
+            throw new IgniteCheckedException("Failed to start discovery manager (thread has been interrupted).", e);
+        }
+
         // Start segment check worker only if frequency is greater than 0.
         if (hasRslvrs && segChkFreq > 0) {
             segChkWrk = new SegmentCheckWorker();
@@ -538,9 +554,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         locNode = spi.getLocalNode();
 
-        updateTopologyVersionIfGreater(new AffinityTopologyVersion(locNode.order()), new DiscoCache(localNode(),
-            getSpi().getRemoteNodes()));
-
         checkAttributes(discoCache().remoteNodes());
 
         // Start discovery worker.


[33/50] [abbrv] ignite git commit: 1.3.2-SNAPSHOT

Posted by nt...@apache.org.
1.3.2-SNAPSHOT


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

Branch: refs/heads/ignite-788-dev
Commit: 7d27103c39be11673f52b6b68ec087742237e258
Parents: 692dd04
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Jul 17 05:04:24 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Jul 17 05:04:24 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                   | 2 +-
 modules/aop/pom.xml                                | 2 +-
 modules/apache-license-gen/pom.xml                 | 2 +-
 modules/aws/pom.xml                                | 2 +-
 modules/clients/pom.xml                            | 2 +-
 modules/cloud/pom.xml                              | 2 +-
 modules/codegen/pom.xml                            | 2 +-
 modules/core/pom.xml                               | 2 +-
 modules/core/src/main/resources/ignite.properties  | 2 +-
 modules/extdata/p2p/pom.xml                        | 2 +-
 modules/extdata/uri/modules/uri-dependency/pom.xml | 2 +-
 modules/extdata/uri/pom.xml                        | 2 +-
 modules/gce/pom.xml                                | 2 +-
 modules/geospatial/pom.xml                         | 2 +-
 modules/hadoop/pom.xml                             | 2 +-
 modules/hibernate/pom.xml                          | 2 +-
 modules/indexing/pom.xml                           | 2 +-
 modules/jcl/pom.xml                                | 2 +-
 modules/jta/pom.xml                                | 2 +-
 modules/kafka/pom.xml                              | 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                             | 2 +-
 modules/ssh/pom.xml                                | 2 +-
 modules/tools/pom.xml                              | 2 +-
 modules/urideploy/pom.xml                          | 2 +-
 modules/visor-console-2.10/pom.xml                 | 2 +-
 modules/visor-console/pom.xml                      | 2 +-
 modules/visor-plugins/pom.xml                      | 2 +-
 modules/web/pom.xml                                | 2 +-
 modules/yardstick/pom.xml                          | 2 +-
 modules/yarn/pom.xml                               | 2 +-
 pom.xml                                            | 2 +-
 41 files changed, 41 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index a94b888..6d3fb07 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index fd3e41b..3c55e7d 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 6e22668..2edb962 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 954d19b..4a568b0 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index fbbf4be..68204f3 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 60c87df..a183522 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 2471585..ea9615c 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index 2c0dfda..c42be3d 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/core/src/main/resources/ignite.properties
----------------------------------------------------------------------
diff --git a/modules/core/src/main/resources/ignite.properties b/modules/core/src/main/resources/ignite.properties
index 3a6aa6f..2313ac3 100644
--- a/modules/core/src/main/resources/ignite.properties
+++ b/modules/core/src/main/resources/ignite.properties
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 
-ignite.version=1.3.1-SNAPSHOT
+ignite.version=1.3.2-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 19c9a28..5a80380 100644
--- a/modules/extdata/p2p/pom.xml
+++ b/modules/extdata/p2p/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-p2p</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/extdata/uri/modules/uri-dependency/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/modules/uri-dependency/pom.xml b/modules/extdata/uri/modules/uri-dependency/pom.xml
index 2ea01d9..b8e4742 100644
--- a/modules/extdata/uri/modules/uri-dependency/pom.xml
+++ b/modules/extdata/uri/modules/uri-dependency/pom.xml
@@ -27,7 +27,7 @@
     <artifactId>ignite-extdata-uri-dep</artifactId>
     <packaging>jar</packaging>
 
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
     <modelVersion>4.0.0</modelVersion>
 
     <dependencies>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 984c744..e4e5bb6 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -32,7 +32,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 3ee4360..b1be8ea 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index a9fbbe8..ecee8d4 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index a07c5b9..555778c 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index 54fea68..5976baa 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index dac173f..067ec2d 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 3b5a0be..ed864ce 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index abe3497..ca5546c 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/kafka/pom.xml
----------------------------------------------------------------------
diff --git a/modules/kafka/pom.xml b/modules/kafka/pom.xml
index 9541bd4..5a8cb72 100644
--- a/modules/kafka/pom.xml
+++ b/modules/kafka/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-kafka</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index d1f4823..0489518 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index a8061ad..8a63a57 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <mesos.version>0.22.0</mesos.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index cc6cbc7..5cb522d 100644
--- a/modules/rest-http/pom.xml
+++ b/modules/rest-http/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-rest-http</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 2dcaba8..0770410 100644
--- a/modules/scalar-2.10/pom.xml
+++ b/modules/scalar-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar_2.10</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 863aaa4..4515772 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 70b38dd..3b734b0 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index e542bdb..959e231 100644
--- a/modules/schema-import/pom.xml
+++ b/modules/schema-import/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schema-import</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 9eb7958..8fa09cb 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 2dedf6d..82b08bc 100644
--- a/modules/spark-2.10/pom.xml
+++ b/modules/spark-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark_2.10</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 4ea1a82..107424a 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index f772084..fdd2b0a 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 1d3e4e1..9fddc21 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 5179489..f58fbdc 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 8c93df7..8c6224f 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/visor-console-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console-2.10/pom.xml b/modules/visor-console-2.10/pom.xml
index f37d4ed..fe56751 100644
--- a/modules/visor-console-2.10/pom.xml
+++ b/modules/visor-console-2.10/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console_2.10</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index a2d25bc..b310d4c 100644
--- a/modules/visor-console/pom.xml
+++ b/modules/visor-console/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-console</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 4aca66b..4417b49 100644
--- a/modules/visor-plugins/pom.xml
+++ b/modules/visor-plugins/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-visor-plugins</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 45ae1f0..74dd525 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 4d9adb7..1044df1 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 2679603..35728fa 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d27103c/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 85bfe14..29f5fa5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.3.2-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[05/50] [abbrv] ignite git commit: disable hanging test

Posted by nt...@apache.org.
disable hanging test


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

Branch: refs/heads/ignite-788-dev
Commit: a6d33b143a46f1d0884ee2d88d5d8edf098b1879
Parents: 7147f3d
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 14:13:00 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 14:13:00 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheAtomicReplicatedNodeRestartSelfTest.java         | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a6d33b14/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
index f556023..68c7fbb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
@@ -26,6 +26,11 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  */
 public class IgniteCacheAtomicReplicatedNodeRestartSelfTest extends GridCacheReplicatedNodeRestartSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1124");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }


[20/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
new file mode 100644
index 0000000..37773cd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectApiExceptionTest.java
@@ -0,0 +1,846 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.testframework.*;
+
+import javax.cache.*;
+import javax.cache.processor.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectApiExceptionTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testErrorOnDisconnect() throws Exception {
+        // Check cache operations.
+        cacheOperationsTest();
+
+        // Check cache operations.
+        beforeTestsStarted();
+        dataStructureOperationsTest();
+
+        // Check ignite operations.
+        beforeTestsStarted();
+        igniteOperationsTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void dataStructureOperationsTest() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(serverCount());
+
+        doTestIgniteOperationOnDisconnect(client, Arrays.asList(
+            // Check atomic long.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.atomicLong("testAtomic", 41, true);
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.atomicLong("testAtomic", 41, true);
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        IgniteAtomicLong atomicLong = (IgniteAtomicLong)o;
+
+                        assertEquals(42, atomicLong.incrementAndGet());
+
+                        return true;
+                    }
+                }
+            ),
+            // Check set.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.set("testSet", new CollectionConfiguration());
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.set("testSet", new CollectionConfiguration());
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        IgniteSet set = (IgniteSet)o;
+
+                        String val = "testVal";
+
+                        set.add(val);
+
+                        assertEquals(1, set.size());
+                        assertTrue(set.contains(val));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check ignite queue.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.queue("TestQueue", 10, new CollectionConfiguration());
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.queue("TestQueue", 10, new CollectionConfiguration());
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        IgniteQueue queue = (IgniteQueue)o;
+
+                        String val = "Test";
+
+                        queue.add(val);
+
+                        assertEquals(val, queue.poll());
+
+                        return true;
+                    }
+                }
+            )
+        ));
+
+        clientMode = false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void cacheOperationsTest() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(serverCount());
+
+        final IgniteCache<Object, Object> dfltCache = client.cache(null);
+
+        assertNotNull(dfltCache);
+
+        doTestIgniteOperationOnDisconnect(client, Arrays.asList(
+            // Check put and get operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            dfltCache.getAndPut(9999, 9999);
+                        }
+                        catch (CacheException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return dfltCache.getAndPut(9999, 9999);
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNull(o);
+
+                        assertEquals(9999, dfltCache.get(9999));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check put operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            dfltCache.put(10000, 10000);
+                        }
+                        catch (CacheException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        dfltCache.put(10000, 10000);
+
+                        return true;
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertTrue((Boolean)o);
+
+                        assertEquals(10000, dfltCache.get(10000));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check get operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            dfltCache.get(10001);
+                        }
+                        catch (CacheException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return dfltCache.get(10001);
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNull(o);
+
+                        return true;
+                    }
+                }
+            ),
+            // Check invoke operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            dfltCache.invoke(10000, new CacheEntryProcessor<Object, Object, Object>() {
+                                @Override public Object process(MutableEntry<Object, Object> entry,
+                                    Object... arguments) throws EntryProcessorException {
+                                    assertTrue(entry.exists());
+
+                                    return (int)entry.getValue() * 2;
+                                }
+                            });
+                        }
+                        catch (CacheException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return dfltCache.invoke(10000, new CacheEntryProcessor<Object, Object, Object>() {
+                            @Override public Object process(MutableEntry<Object, Object> entry,
+                                Object... arguments) throws EntryProcessorException {
+                                assertTrue(entry.exists());
+
+                                return (int)entry.getValue() * 2;
+                            }
+                        });
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        assertEquals(20000, (int)o);
+
+                        return true;
+                    }
+                }
+            ),
+            // Check put async operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        IgniteCache<Object, Object> async = dfltCache.withAsync();
+
+                        boolean failed = false;
+
+                        try {
+                            async.put(10002, 10002);
+
+                            async.future().get();
+                        }
+                        catch (CacheException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        async.put(10002, 10002);
+
+                        return async.future().get();
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNull(o);
+
+                        assertEquals(10002, dfltCache.get(10002));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check transaction.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.transactions();
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.transactions();
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        IgniteTransactions txs = (IgniteTransactions)o;
+
+                        assertNotNull(txs);
+
+                        return true;
+                    }
+                }
+            ),
+            // Check get cache.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.cache(null);
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.cache(null);
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        IgniteCache<Object, Object> cache0 = (IgniteCache<Object, Object>)o;
+
+                        assertNotNull(cache0);
+
+                        cache0.put(1, 1);
+
+                        assertEquals(1, cache0.get(1));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check streamer.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.dataStreamer(null);
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.dataStreamer(null);
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        IgniteDataStreamer<Object, Object> streamer = (IgniteDataStreamer<Object, Object>)o;
+
+                        streamer.addData(2, 2);
+
+                        streamer.close();
+
+                        assertEquals(2, client.cache(null).get(2));
+
+                        return true;
+                    }
+                }
+            ),
+            // Check create cache.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.createCache("test_cache");
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.createCache("test_cache");
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        IgniteCache<Object, Object> cache = (IgniteCache<Object, Object>)o;
+
+                        assertNotNull(cache);
+
+                        cache.put(1, 1);
+
+                        assertEquals(1, cache.get(1));
+
+                        return true;
+                    }
+                }
+            )
+
+        ));
+
+        clientMode = false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public void igniteOperationsTest() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(serverCount());
+
+        final IgniteCache<Object, Object> dfltCache = client.cache(null);
+
+        final CountDownLatch recvLatch = new CountDownLatch(1);
+
+        assertNotNull(dfltCache);
+
+        doTestIgniteOperationOnDisconnect(client, Arrays.asList(
+            // Check compute.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.compute();
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.compute();
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        IgniteCompute comp = (IgniteCompute)o;
+
+                        Collection<UUID> uuids = comp.broadcast(new IgniteCallable<UUID>() {
+                            @IgniteInstanceResource
+                            private Ignite ignite;
+
+                            @Override public UUID call() throws Exception {
+                                return ignite.cluster().localNode().id();
+                            }
+                        });
+
+                        assertFalse(uuids.isEmpty());
+
+                        for (UUID uuid : uuids)
+                            assertNotNull(uuid);
+
+                        return true;
+                    }
+                }
+            ),
+
+            // Check ping node.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.cluster().pingNode(new UUID(0, 0));
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.cluster().pingNode(new UUID(0, 0));
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        Boolean pingNode = (Boolean)o;
+
+                        assertFalse(pingNode);
+
+                        return true;
+                    }
+                }
+            ),
+            // Check register remote listener.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.events().remoteListen(null, new IgnitePredicate<Event>() {
+                                @Override public boolean apply(Event event) {
+                                    return true;
+                                }
+                            });
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.events().remoteListen(null, new IgnitePredicate<Event>() {
+                            @Override public boolean apply(Event event) {
+                                return true;
+                            }
+                        });
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        UUID remoteId = (UUID)o;
+
+                        assertNotNull(remoteId);
+
+                        client.events().stopRemoteListen(remoteId);
+
+                        return true;
+                    }
+                }
+            ),
+            // Check message operation.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.message().remoteListen(null, new IgniteBiPredicate<UUID, Object>() {
+                                @Override public boolean apply(UUID uuid, Object o) {
+                                    if (o.equals("Test message."))
+                                        recvLatch.countDown();
+
+                                    return true;
+                                }
+                            });
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.message().remoteListen(null, new IgniteBiPredicate<UUID, Object>() {
+                            @Override public boolean apply(UUID uuid, Object o) {
+                                if (o.equals("Test message."))
+                                    recvLatch.countDown();
+
+                                return true;
+                            }
+                        });
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        IgniteMessaging msg = client.message();
+
+                        msg.send(null, "Test message.");
+
+                        try {
+                            assertTrue(recvLatch.await(2, SECONDS));
+                        }
+                        catch (InterruptedException e) {
+                            fail("Message wasn't received.");
+                        }
+
+                        return true;
+                    }
+                }
+            ),
+            // Check executor.
+            new T2<Callable, C1<Object, Boolean>>(
+                new Callable() {
+                    @Override public Object call() throws Exception {
+                        boolean failed = false;
+
+                        try {
+                            client.executorService().submit(new Callable<Integer>() {
+                                @Override public Integer call() throws Exception {
+                                    return 42;
+                                }
+                            });
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            failed = true;
+
+                            checkAndWait(e);
+                        }
+
+                        assertTrue(failed);
+
+                        return client.executorService().submit(new Callable<Integer>() {
+                            @Override public Integer call() throws Exception {
+                                return 42;
+                            }
+                        });
+                    }
+                },
+                new C1<Object, Boolean>() {
+                    @Override public Boolean apply(Object o) {
+                        assertNotNull(o);
+
+                        Future<Integer> fut = (Future<Integer>)o;
+
+                        try {
+                            assertEquals(42, (int)fut.get());
+                        }
+                        catch (Exception e) {
+                            fail("Failed submit task.");
+                        }
+
+                        return true;
+                    }
+                }
+            )
+        ));
+
+        clientMode = false;
+    }
+
+    /**
+     * @param client Client.
+     * @param ops Operations closures.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void doTestIgniteOperationOnDisconnect(Ignite client, final List<T2<Callable, C1<Object, Boolean>>> ops)
+        throws Exception {
+        assertNotNull(client.cache(null));
+
+        final TestTcpDiscoverySpi clientSpi = spi(client);
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        log.info("Block reconnect.");
+
+        clientSpi.writeLatch = new CountDownLatch(1);
+
+        final List<IgniteInternalFuture> futs = new ArrayList<>();
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    assertEquals(1, reconnectLatch.getCount());
+
+                    for (T2<Callable, C1<Object, Boolean>> op : ops)
+                        futs.add(GridTestUtils.runAsync(op.get1()));
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        try {
+            log.info("Fail client.");
+
+            srvSpi.failNode(client.cluster().localNode().id(), null);
+
+            waitReconnectEvent(disconnectLatch);
+
+            assertEquals(ops.size(), futs.size());
+
+            for (IgniteInternalFuture<?> fut : futs)
+                assertNotDone(fut);
+
+            U.sleep(2000);
+
+            for (IgniteInternalFuture<?> fut : futs)
+                assertNotDone(fut);
+
+            log.info("Allow reconnect.");
+
+            clientSpi.writeLatch.countDown();
+
+            waitReconnectEvent(reconnectLatch);
+
+            // Check operation after reconnect working.
+            for (int i = 0; i < futs.size(); i++) {
+                final int i0 = i;
+
+                try {
+                    final Object futRes = futs.get(i0).get(2, SECONDS);
+
+                    assertTrue(GridTestUtils.runAsync(new Callable<Boolean>() {
+                        @Override public Boolean call() throws Exception {
+                            return ops.get(i0).get2().apply(futRes);
+                        }
+                    }).get(2, SECONDS));
+                }
+                catch (IgniteFutureTimeoutCheckedException e) {
+                    e.printStackTrace();
+
+                    fail("Operation timeout. Iteration: " + i + ".");
+                }
+            }
+        }
+        finally {
+            clientSpi.writeLatch.countDown();
+
+            for (IgniteInternalFuture fut : futs)
+                fut.cancel();
+
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
new file mode 100644
index 0000000..bb568ab
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAtomicsTest.java
@@ -0,0 +1,672 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectAtomicsTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicSeqReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeq", 0, true);
+
+        assertEquals(1L, clientAtomicSeq.incrementAndGet());
+
+        final IgniteAtomicSequence srvAtomicSeq = srv.atomicSequence("atomicSeq", 0, false);
+
+        assertEquals(1001L, srvAtomicSeq.incrementAndGet());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertEquals(1002L, srvAtomicSeq.incrementAndGet());
+            }
+        });
+
+        assertEquals(2L, clientAtomicSeq.incrementAndGet());
+
+        assertEquals(1003L, srvAtomicSeq.incrementAndGet());
+
+        assertEquals(3L, clientAtomicSeq.incrementAndGet());
+
+        clientAtomicSeq.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicSeqReconnectRemoved() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeqRmv", 0, true);
+
+        clientAtomicSeq.batchSize(1);
+
+        assertEquals(1L, clientAtomicSeq.incrementAndGet());
+
+        final IgniteAtomicSequence srvAtomicSeq = srv.atomicSequence("atomicSeqRmv", 0, false);
+
+        srvAtomicSeq.batchSize(1);
+
+        assertEquals(1001L, srvAtomicSeq.incrementAndGet());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvAtomicSeq.close();
+
+                assert srvAtomicSeq.removed();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < 2000; i++)
+                    clientAtomicSeq.incrementAndGet();
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteAtomicSequence newClientAtomicSeq = client.atomicSequence("atomicSeqRmv", 0, true);
+
+        assertEquals(0, newClientAtomicSeq.get());
+
+        assertEquals(1, newClientAtomicSeq.incrementAndGet());
+
+        newClientAtomicSeq.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicSeqReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        final IgniteAtomicSequence clientAtomicSeq = client.atomicSequence("atomicSeqInProg", 0, true);
+
+        clientAtomicSeq.batchSize(1);
+
+        final IgniteAtomicSequence srvAtomicSeq = srv.atomicSequence("atomicSeqInProg", 0, false);
+
+        srvAtomicSeq.batchSize(1);
+
+        commSpi.blockMessage(GridNearLockResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < 3000; i++) {
+                    try {
+                        clientAtomicSeq.incrementAndGet();
+                    }
+                    catch (IgniteClientDisconnectedException e) {
+                        checkAndWait(e);
+
+                        return true;
+                    }
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        // Check that after reconnect working.
+        assert clientAtomicSeq.incrementAndGet() >= 0;
+        assert srvAtomicSeq.incrementAndGet() >= 0;
+
+        clientAtomicSeq.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReferenceReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteAtomicReference<String> clientAtomicRef = client.atomicReference("atomicRef", "1st value", true);
+
+        assertEquals("1st value", clientAtomicRef.get());
+        assertTrue(clientAtomicRef.compareAndSet("1st value", "2st value"));
+        assertEquals("2st value", clientAtomicRef.get());
+
+        final IgniteAtomicReference<String> srvAtomicRef = srv.atomicReference("atomicRef", "1st value", false);
+
+        assertEquals("2st value", srvAtomicRef.get());
+        assertTrue(srvAtomicRef.compareAndSet("2st value", "3st value"));
+        assertEquals("3st value", srvAtomicRef.get());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertEquals("3st value", srvAtomicRef.get());
+                assertTrue(srvAtomicRef.compareAndSet("3st value", "4st value"));
+                assertEquals("4st value", srvAtomicRef.get());
+            }
+        });
+
+        assertEquals("4st value", clientAtomicRef.get());
+        assertTrue(clientAtomicRef.compareAndSet("4st value", "5st value"));
+        assertEquals("5st value", clientAtomicRef.get());
+
+        assertEquals("5st value", srvAtomicRef.get());
+        assertTrue(srvAtomicRef.compareAndSet("5st value", "6st value"));
+        assertEquals("6st value", srvAtomicRef.get());
+
+        srvAtomicRef.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReferenceReconnectRemoved() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicReference<String> clientAtomicRef =
+            client.atomicReference("atomicRefRemoved", "1st value", true);
+
+        assertEquals("1st value", clientAtomicRef.get());
+        assertTrue(clientAtomicRef.compareAndSet("1st value", "2st value"));
+        assertEquals("2st value", clientAtomicRef.get());
+
+        final IgniteAtomicReference<String> srvAtomicRef = srv.atomicReference("atomicRefRemoved", "1st value", false);
+
+        assertEquals("2st value", srvAtomicRef.get());
+        assertTrue(srvAtomicRef.compareAndSet("2st value", "3st value"));
+        assertEquals("3st value", srvAtomicRef.get());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvAtomicRef.close();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                clientAtomicRef.compareAndSet("3st value", "4st value");
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteAtomicReference<String> newClientAtomicRef =
+            client.atomicReference("atomicRefRemoved", "1st value", true);
+
+        IgniteAtomicReference<String> newSrvAtomicRef = srv.atomicReference("atomicRefRemoved", "1st value", false);
+
+        assertEquals("1st value", newClientAtomicRef.get());
+        assertTrue(newClientAtomicRef.compareAndSet("1st value", "2st value"));
+        assertEquals("2st value", newClientAtomicRef.get());
+
+        assertEquals("2st value", newSrvAtomicRef.get());
+        assertTrue(newSrvAtomicRef.compareAndSet("2st value", "3st value"));
+        assertEquals("3st value", newSrvAtomicRef.get());
+
+        newClientAtomicRef.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicReferenceReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicReference<String> clientAtomicRef =
+            client.atomicReference("atomicRefInProg", "1st value", true);
+
+        assertEquals("1st value", clientAtomicRef.get());
+        assertTrue(clientAtomicRef.compareAndSet("1st value", "2st value"));
+        assertEquals("2st value", clientAtomicRef.get());
+
+        IgniteAtomicReference<String> srvAtomicRef = srv.atomicReference("atomicRefInProg", "1st value", false);
+
+        assertEquals("2st value", srvAtomicRef.get());
+        assertTrue(srvAtomicRef.compareAndSet("2st value", "3st value"));
+        assertEquals("3st value", srvAtomicRef.get());
+
+        BlockTpcCommunicationSpi servCommSpi = commSpi(srv);
+
+        servCommSpi.blockMessage(GridNearLockResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    clientAtomicRef.compareAndSet("3st value", "4st value");
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        servCommSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        // Check that after reconnect working.
+        assertEquals("3st value", clientAtomicRef.get());
+        assertTrue(clientAtomicRef.compareAndSet("3st value", "4st value"));
+        assertEquals("4st value", clientAtomicRef.get());
+
+        assertEquals("4st value", srvAtomicRef.get());
+        assertTrue(srvAtomicRef.compareAndSet("4st value", "5st value"));
+        assertEquals("5st value", srvAtomicRef.get());
+
+        srvAtomicRef.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicStampedReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStamped", 0, 0, true);
+
+        assertEquals(true, clientAtomicStamped.compareAndSet(0, 1, 0, 1));
+        assertEquals(1, clientAtomicStamped.value());
+        assertEquals(1, clientAtomicStamped.stamp());
+
+        final IgniteAtomicStamped srvAtomicStamped = srv.atomicStamped("atomicStamped", 0, 0, false);
+
+        assertEquals(true, srvAtomicStamped.compareAndSet(1, 2, 1, 2));
+        assertEquals(2, srvAtomicStamped.value());
+        assertEquals(2, srvAtomicStamped.stamp());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertEquals(true, srvAtomicStamped.compareAndSet(2, 3, 2, 3));
+                assertEquals(3, srvAtomicStamped.value());
+                assertEquals(3, srvAtomicStamped.stamp());
+            }
+        });
+
+        assertEquals(true, clientAtomicStamped.compareAndSet(3, 4, 3, 4));
+        assertEquals(4, clientAtomicStamped.value());
+        assertEquals(4, clientAtomicStamped.stamp());
+
+        assertEquals(true, srvAtomicStamped.compareAndSet(4, 5, 4, 5));
+        assertEquals(5, srvAtomicStamped.value());
+        assertEquals(5, srvAtomicStamped.stamp());
+
+        srvAtomicStamped.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicStampedReconnectRemoved() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedRemoved", 0, 0, true);
+
+        assertEquals(true, clientAtomicStamped.compareAndSet(0, 1, 0, 1));
+        assertEquals(1, clientAtomicStamped.value());
+        assertEquals(1, clientAtomicStamped.stamp());
+
+        final IgniteAtomicStamped srvAtomicStamped = srv.atomicStamped("atomicStampedRemoved", 0, 0, false);
+
+        assertEquals(true, srvAtomicStamped.compareAndSet(1, 2, 1, 2));
+        assertEquals(2, srvAtomicStamped.value());
+        assertEquals(2, srvAtomicStamped.stamp());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvAtomicStamped.close();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                clientAtomicStamped.compareAndSet(2, 3, 2, 3);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteAtomicStamped newClientAtomicStamped = client.atomicStamped("atomicStampedRemoved", 0, 0, true);
+
+        assertEquals(true, newClientAtomicStamped.compareAndSet(0, 1, 0, 1));
+        assertEquals(1, newClientAtomicStamped.value());
+        assertEquals(1, newClientAtomicStamped.stamp());
+
+        IgniteAtomicStamped newSrvAtomicStamped = srv.atomicStamped("atomicStampedRemoved", 0, 0, false);
+
+        assertEquals(true, newSrvAtomicStamped.compareAndSet(1, 2, 1, 2));
+        assertEquals(2, newSrvAtomicStamped.value());
+        assertEquals(2, newSrvAtomicStamped.stamp());
+
+        newClientAtomicStamped.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicStampedReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicStamped clientAtomicStamped = client.atomicStamped("atomicStampedInProgress", 0, 0, true);
+
+        assertEquals(true, clientAtomicStamped.compareAndSet(0, 1, 0, 1));
+        assertEquals(1, clientAtomicStamped.value());
+        assertEquals(1, clientAtomicStamped.stamp());
+
+        IgniteAtomicStamped srvAtomicStamped = srv.atomicStamped("atomicStampedInProgress", 0, 0, false);
+
+        assertEquals(true, srvAtomicStamped.compareAndSet(1, 2, 1, 2));
+        assertEquals(2, srvAtomicStamped.value());
+        assertEquals(2, srvAtomicStamped.stamp());
+
+        BlockTpcCommunicationSpi servCommSpi = commSpi(srv);
+
+        servCommSpi.blockMessage(GridNearLockResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    clientAtomicStamped.compareAndSet(2, 3, 2, 3);
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        servCommSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        // Check that after reconnect working.
+        assertEquals(true, clientAtomicStamped.compareAndSet(2, 3, 2, 3));
+        assertEquals(3, clientAtomicStamped.value());
+        assertEquals(3, clientAtomicStamped.stamp());
+
+        assertEquals(true, srvAtomicStamped.compareAndSet(3, 4, 3, 4));
+        assertEquals(4, srvAtomicStamped.value());
+        assertEquals(4, srvAtomicStamped.stamp());
+
+        srvAtomicStamped.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicLongReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLong", 0, true);
+
+        assertEquals(0L, clientAtomicLong.getAndAdd(1));
+
+        final IgniteAtomicLong srvAtomicLong = srv.atomicLong("atomicLong", 0, false);
+
+        assertEquals(1L, srvAtomicLong.getAndAdd(1));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertEquals(2L, srvAtomicLong.getAndAdd(1));
+            }
+        });
+
+        assertEquals(3L, clientAtomicLong.getAndAdd(1));
+
+        assertEquals(4L, srvAtomicLong.getAndAdd(1));
+
+        assertEquals(5L, clientAtomicLong.getAndAdd(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicLongReconnectRemoved() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLongRmv", 0, true);
+
+        assertEquals(0L, clientAtomicLong.getAndAdd(1));
+
+        final IgniteAtomicLong srvAtomicLong = srv.atomicLong("atomicLongRmv", 0, false);
+
+        assertEquals(1L, srvAtomicLong.getAndAdd(1));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvAtomicLong.close();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                clientAtomicLong.getAndAdd(1);
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteAtomicLong newClientAtomicLong = client.atomicLong("atomicLongRmv", 0, true);
+
+        assertEquals(0L, newClientAtomicLong.getAndAdd(1));
+
+        IgniteAtomicLong newSrvAtomicLong = srv.atomicLong("atomicLongRmv", 0, false);
+
+        assertEquals(1L, newSrvAtomicLong.getAndAdd(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAtomicLongReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        final IgniteAtomicLong clientAtomicLong = client.atomicLong("atomicLongInProggress", 0, true);
+
+        final IgniteAtomicLong srvAtomicLong = srv.atomicLong("atomicLongInProggress", 0, false);
+
+        commSpi.blockMessage(GridNearLockResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    clientAtomicLong.getAndAdd(1);
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        // Check that after reconnect working.
+        assertEquals(1, clientAtomicLong.addAndGet(1));
+        assertEquals(2, srvAtomicLong.addAndGet(1));
+
+        clientAtomicLong.close();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLatchReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteCountDownLatch clientLatch = client.countDownLatch("latch1", 3, false, true);
+
+        assertEquals(3, clientLatch.count());
+
+        final IgniteCountDownLatch srvLatch = srv.countDownLatch("latch1", 3, false, false);
+
+        assertEquals(3, srvLatch.count());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvLatch.countDown();
+            }
+        });
+
+        assertEquals(2, srvLatch.count());
+        assertEquals(2, clientLatch.count());
+
+        srvLatch.countDown();
+
+        assertEquals(1, srvLatch.count());
+        assertEquals(1, clientLatch.count());
+
+        clientLatch.countDown();
+
+        assertEquals(0, srvLatch.count());
+        assertEquals(0, clientLatch.count());
+
+        assertTrue(srvLatch.await(1000));
+        assertTrue(clientLatch.await(1000));
+    }
+}


[38/50] [abbrv] ignite git commit: Fixed typo and minor fixes to javadocs.

Posted by nt...@apache.org.
Fixed typo and minor fixes to javadocs.


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

Branch: refs/heads/ignite-788-dev
Commit: ede961215ce3a596f7dcb90e82f48aa723d892bf
Parents: 192fefc
Author: AKuznetsov <ak...@gridgain.com>
Authored: Mon Jul 20 15:06:04 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Mon Jul 20 15:06:04 2015 +0700

----------------------------------------------------------------------
 .../ignite/schema/ui/SchemaImportApp.java       | 36 ++++++++++++++++----
 1 file changed, 29 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ede96121/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
----------------------------------------------------------------------
diff --git a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
index 00d1bc8..7b9c220 100644
--- a/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
+++ b/modules/schema-import/src/main/java/org/apache/ignite/schema/ui/SchemaImportApp.java
@@ -109,7 +109,7 @@ public class SchemaImportApp extends Application {
             "jdbc:mysql://[host]:[port]/[database]", "root"),
         new Preset("mssql", "Microsoft SQL Server", "sqljdbc41.jar", "com.microsoft.sqlserver.jdbc.SQLServerDriver",
             "jdbc:sqlserver://[host]:[port][;databaseName=database]", "sa"),
-        new Preset("posgresql", "PostgreSQL", "postgresql-9.3.jdbc4.jar", "org.postgresql.Driver",
+        new Preset("postgresql", "PostgreSQL", "postgresql-9.3.jdbc4.jar", "org.postgresql.Driver",
             "jdbc:postgresql://[host]:[port]/[database]", "sa"),
         new Preset("custom", "Custom server...", "custom-jdbc.jar", "org.custom.Driver", "jdbc:custom", "sa")
     };
@@ -444,6 +444,11 @@ public class SchemaImportApp extends Application {
         final boolean singleXml = xmlSingleFileCh.isSelected();
 
         Runnable task = new Task<Void>() {
+            /**
+             * @param pojo POJO descriptor to check.
+             * @param selected Selected flag.
+             * @param msg Message to show in case of check failed.
+             */
             private void checkEmpty(final PojoDescriptor pojo, boolean selected, String msg) {
                 if (!selected) {
                     Platform.runLater(new Runnable() {
@@ -694,6 +699,8 @@ public class SchemaImportApp extends Application {
 
     /**
      * Create connection pane with controls.
+     *
+     * @return Pane with connection controls.
      */
     private Pane createConnectionPane() {
         connPnl = paneEx(10, 10, 0, 10);
@@ -1110,10 +1117,10 @@ public class SchemaImportApp extends Application {
                 if (curPojo != null) {
                     TableView.TableViewSelectionModel<PojoDescriptor> selMdl = pojosTbl.getSelectionModel();
 
-                    List<Integer> idxs = new ArrayList<>(selMdl.getSelectedIndices());
+                    List<Integer> selIndices = new ArrayList<>(selMdl.getSelectedIndices());
 
-                    if (idxs.size() > 1) {
-                        for (Integer idx : idxs) {
+                    if (selIndices.size() > 1) {
+                        for (Integer idx : selIndices) {
                             if (pojos.get(idx) != curPojo)
                                 selMdl.clearSelection(idx);
                         }
@@ -1242,6 +1249,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as string.
      */
     private String getStringProp(String key, String dflt) {
         String val = prefs.getProperty(key);
@@ -1267,6 +1275,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as int.
      */
     private int getIntProp(String key, int dflt) {
         String val = prefs.getProperty(key);
@@ -1297,6 +1306,7 @@ public class SchemaImportApp extends Application {
      *
      * @param key Property key.
      * @param dflt Default value.
+     * @return Property value as boolean.
      */
     private boolean getBoolProp(String key, boolean dflt) {
         String val = prefs.getProperty(key);
@@ -1563,7 +1573,11 @@ public class SchemaImportApp extends Application {
         /** Combo box. */
         private final ComboBox<String> comboBox;
 
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for cell with java types combobox.
+         */
         public static Callback<TableColumn<PojoField, String>, TableCell<PojoField, String>> cellFactory() {
             return new Callback<TableColumn<PojoField, String>, TableCell<PojoField, String>>() {
                 @Override public TableCell<PojoField, String> call(TableColumn<PojoField, String> col) {
@@ -1637,7 +1651,11 @@ public class SchemaImportApp extends Application {
      * Special table cell to select schema or table.
      */
     private static class PojoDescriptorCell extends TableCell<PojoDescriptor, Boolean> {
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for schema / table selection.
+         */
         public static Callback<TableColumn<PojoDescriptor, Boolean>, TableCell<PojoDescriptor, Boolean>> cellFactory() {
             return new Callback<TableColumn<PojoDescriptor, Boolean>, TableCell<PojoDescriptor, Boolean>>() {
                 @Override public TableCell<PojoDescriptor, Boolean> call(TableColumn<PojoDescriptor, Boolean> col) {
@@ -1694,7 +1712,11 @@ public class SchemaImportApp extends Application {
      * Special table cell to select &quot;used&quot; fields for code generation.
      */
     private static class PojoFieldUseCell extends TableCell<PojoField, Boolean> {
-        /** Creates a ComboBox cell factory for use in TableColumn controls. */
+        /**
+         * Creates a ComboBox cell factory for use in TableColumn controls.
+         *
+         * @return Cell factory for used fields selection.
+         */
         public static Callback<TableColumn<PojoField, Boolean>, TableCell<PojoField, Boolean>> cellFactory() {
             return new Callback<TableColumn<PojoField, Boolean>, TableCell<PojoField, Boolean>>() {
                 @Override public TableCell<PojoField, Boolean> call(TableColumn<PojoField, Boolean> col) {


[21/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 4ca2995..2bce637 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -228,6 +228,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** */
     public static final byte HANDSHAKE_MSG_TYPE = -3;
 
+    /** */
+    private ConnectGateway connectGate;
+
     /** Server listener. */
     private final GridNioServerListener<Message> srvLsnr =
         new GridNioServerListenerAdapter<Message>() {
@@ -248,7 +251,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     if (log.isDebugEnabled())
                         log.debug("Sending local node ID to newly accepted session: " + ses);
 
-                    ses.send(nodeIdMsg);
+                    ses.send(nodeIdMessage());
                 }
             }
 
@@ -289,136 +292,163 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 }
             }
 
-            @Override public void onMessage(GridNioSession ses, Message msg) {
-                UUID sndId = ses.meta(NODE_ID_META);
+            /**
+             * @param ses Session.
+             * @param msg Message.
+             */
+            private void onFirstMessage(GridNioSession ses, Message msg) {
+                UUID sndId;
 
-                if (sndId == null) {
-                    assert ses.accepted();
+                if (msg instanceof NodeIdMessage)
+                    sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+                else {
+                    assert msg instanceof HandshakeMessage : msg;
 
-                    if (msg instanceof NodeIdMessage)
-                        sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
-                    else {
-                        assert msg instanceof HandshakeMessage : msg;
+                    sndId = ((HandshakeMessage)msg).nodeId();
+                }
 
-                        sndId = ((HandshakeMessage)msg).nodeId();
-                    }
+                if (log.isDebugEnabled())
+                    log.debug("Remote node ID received: " + sndId);
 
-                    if (log.isDebugEnabled())
-                        log.debug("Remote node ID received: " + sndId);
+                final UUID old = ses.addMeta(NODE_ID_META, sndId);
 
-                    final UUID old = ses.addMeta(NODE_ID_META, sndId);
+                assert old == null;
 
-                    assert old == null;
+                final ClusterNode rmtNode = getSpiContext().node(sndId);
 
-                    final ClusterNode rmtNode = getSpiContext().node(sndId);
+                if (rmtNode == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Close incoming connection, unknown node: " + sndId);
 
-                    if (rmtNode == null) {
-                        ses.close();
+                    ses.close();
 
-                        return;
-                    }
+                    return;
+                }
 
-                    ClusterNode locNode = getSpiContext().localNode();
+                ClusterNode locNode = getSpiContext().localNode();
 
-                    if (ses.remoteAddress() == null)
-                        return;
+                if (ses.remoteAddress() == null)
+                    return;
 
-                    GridCommunicationClient oldClient = clients.get(sndId);
+                GridCommunicationClient oldClient = clients.get(sndId);
 
-                    boolean hasShmemClient = false;
+                boolean hasShmemClient = false;
 
-                    if (oldClient != null) {
-                        if (oldClient instanceof GridTcpNioCommunicationClient) {
-                            if (log.isDebugEnabled())
-                                log.debug("Received incoming connection when already connected " +
+                if (oldClient != null) {
+                    if (oldClient instanceof GridTcpNioCommunicationClient) {
+                        if (log.isDebugEnabled())
+                            log.debug("Received incoming connection when already connected " +
                                     "to this node, rejecting [locNode=" + locNode.id() +
                                     ", rmtNode=" + sndId + ']');
 
-                            ses.send(new RecoveryLastReceivedMessage(-1));
+                        ses.send(new RecoveryLastReceivedMessage(-1));
 
-                            return;
-                        }
-                        else {
-                            assert oldClient instanceof GridShmemCommunicationClient;
+                        return;
+                    }
+                    else {
+                        assert oldClient instanceof GridShmemCommunicationClient;
 
-                            hasShmemClient = true;
-                        }
+                        hasShmemClient = true;
                     }
+                }
 
-                    GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
+                GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
 
-                    GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
+                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
 
-                    assert msg instanceof HandshakeMessage : msg;
+                assert msg instanceof HandshakeMessage : msg;
 
-                    HandshakeMessage msg0 = (HandshakeMessage)msg;
+                HandshakeMessage msg0 = (HandshakeMessage)msg;
 
-                    final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
+                final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
 
-                    if (oldFut == null) {
-                        oldClient = clients.get(sndId);
+                if (oldFut == null) {
+                    oldClient = clients.get(sndId);
 
-                        if (oldClient != null) {
-                            if (oldClient instanceof GridTcpNioCommunicationClient) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Received incoming connection when already connected " +
+                    if (oldClient != null) {
+                        if (oldClient instanceof GridTcpNioCommunicationClient) {
+                            if (log.isDebugEnabled())
+                                log.debug("Received incoming connection when already connected " +
                                         "to this node, rejecting [locNode=" + locNode.id() +
                                         ", rmtNode=" + sndId + ']');
 
-                                ses.send(new RecoveryLastReceivedMessage(-1));
+                            ses.send(new RecoveryLastReceivedMessage(-1));
 
-                                return;
-                            }
-                            else {
-                                assert oldClient instanceof GridShmemCommunicationClient;
+                            return;
+                        }
+                        else {
+                            assert oldClient instanceof GridShmemCommunicationClient;
 
-                                hasShmemClient = true;
-                            }
+                            hasShmemClient = true;
                         }
+                    }
 
-                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                    boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
                             new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
 
-                        if (log.isDebugEnabled())
-                            log.debug("Received incoming connection from remote node " +
+                    if (log.isDebugEnabled())
+                        log.debug("Received incoming connection from remote node " +
                                 "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ']');
 
-                        if (reserved) {
-                            try {
-                                GridTcpNioCommunicationClient client =
+                    if (reserved) {
+                        try {
+                            GridTcpNioCommunicationClient client =
                                     connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
-                                fut.onDone(client);
-                            }
-                            finally {
-                                clientFuts.remove(rmtNode.id(), fut);
-                            }
+                            fut.onDone(client);
+                        }
+                        finally {
+                            clientFuts.remove(rmtNode.id(), fut);
                         }
                     }
-                    else {
-                        if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
-                            if (log.isDebugEnabled()) {
-                                log.debug("Received incoming connection from remote node while " +
+                }
+                else {
+                    if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
+                        if (log.isDebugEnabled()) {
+                            log.debug("Received incoming connection from remote node while " +
                                     "connecting to this node, rejecting [locNode=" + locNode.id() +
                                     ", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() +
                                     ", rmtNodeOrder=" + rmtNode.order() + ']');
-                            }
-
-                            ses.send(new RecoveryLastReceivedMessage(-1));
                         }
-                        else {
-                            boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+
+                        ses.send(new RecoveryLastReceivedMessage(-1));
+                    }
+                    else {
+                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
                                 new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
 
-                            if (reserved) {
-                                GridTcpNioCommunicationClient client =
+                        if (reserved) {
+                            GridTcpNioCommunicationClient client =
                                     connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
-                                fut.onDone(client);
-                            }
+                            fut.onDone(client);
                         }
                     }
                 }
+            }
+
+            @Override public void onMessage(GridNioSession ses, Message msg) {
+                UUID sndId = ses.meta(NODE_ID_META);
+
+                if (sndId == null) {
+                    assert ses.accepted() : ses;
+
+                    if (!connectGate.tryEnter()) {
+                        if (log.isDebugEnabled())
+                            log.debug("Close incoming connection, failed to enter gateway.");
+
+                        ses.close();
+
+                        return;
+                    }
+
+                    try {
+                        onFirstMessage(ses, msg);
+                    }
+                    finally {
+                        connectGate.leave();
+                    }
+                }
                 else {
                     rcvdMsgsCnt.increment();
 
@@ -700,9 +730,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Address resolver. */
     private AddressResolver addrRslvr;
 
-    /** Local node ID message. */
-    private NodeIdMessage nodeIdMsg;
-
     /** Received messages count. */
     private final LongAdder8 rcvdMsgsCnt = new LongAdder8();
 
@@ -739,8 +766,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
         @Override public void onEvent(Event evt) {
-            assert evt instanceof DiscoveryEvent;
-            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED;
+            assert evt instanceof DiscoveryEvent : evt;
+            assert evt.type() == EVT_NODE_LEFT || evt.type() == EVT_NODE_FAILED ;
 
             onNodeLeft(((DiscoveryEvent)evt).eventNode().id());
         }
@@ -1237,8 +1264,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /** {@inheritDoc} */
     @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
-        nodeIdMsg = new NodeIdMessage(getLocalNodeId());
-
         assertParameter(locPort > 1023, "locPort > 1023");
         assertParameter(locPort <= 0xffff, "locPort < 0xffff");
         assertParameter(locPortRange >= 0, "locPortRange >= 0");
@@ -1346,6 +1371,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
+        connectGate = new ConnectGateway();
+
         if (shmemSrv != null) {
             shmemAcceptWorker = new ShmemAcceptWorker(shmemSrv);
 
@@ -1608,6 +1635,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             // Safety.
             ctxInitLatch.countDown();
 
+        if (connectGate != null)
+            connectGate.stopped();
+
         // Force closing.
         for (GridCommunicationClient client : clients.values())
             client.forceClose();
@@ -1617,6 +1647,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         getSpiContext().removeLocalEventListener(discoLsnr);
     }
 
+    /** {@inheritDoc} */
+    @Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) {
+        connectGate.disconnected(reconnectFut);
+
+        for (GridCommunicationClient client : clients.values())
+            client.forceClose();
+
+        IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
+            "Failed to connect client node disconnected.");
+
+        for (GridFutureAdapter<GridCommunicationClient> clientFut : clientFuts.values())
+            clientFut.onDone(err);
+
+        recoveryDescs.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClientReconnected(boolean clusterRestarted) {
+        connectGate.reconnected();
+    }
+
     /**
      * @param nodeId Left node ID.
      */
@@ -1666,10 +1717,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (log.isTraceEnabled())
             log.trace("Sending message to node [node=" + node + ", msg=" + msg + ']');
 
-        UUID locNodeId = getLocalNodeId();
-
-        if (node.id().equals(locNodeId))
-            notifyListener(locNodeId, msg, NOOP);
+        if (node.id().equals(getLocalNode().id()))
+            notifyListener(node.id(), msg, NOOP);
         else {
             GridCommunicationClient client = null;
 
@@ -1834,7 +1883,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
         }
 
-        return createTcpClient(node);
+        connectGate.enter();
+
+        try {
+            return createTcpClient(node);
+        }
+        finally {
+            connectGate.leave();
+        }
     }
 
     /**
@@ -2208,7 +2264,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     ch.write(ByteBuffer.wrap(U.IGNITE_HEADER));
 
                     if (recovery != null) {
-                        HandshakeMessage msg = new HandshakeMessage(getLocalNodeId(),
+                        HandshakeMessage msg = new HandshakeMessage(getLocalNode().id(),
                             recovery.incrementConnectCount(),
                             recovery.receivedCount());
 
@@ -2228,7 +2284,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         ch.write(buf);
                     }
                     else
-                        ch.write(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType));
+                        ch.write(ByteBuffer.wrap(nodeIdMessage().nodeIdBytesWithType));
 
                     if (recovery != null) {
                         if (log.isDebugEnabled())
@@ -2355,6 +2411,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         getExceptionRegistry().onException(msg, e);
     }
 
+    /**
+     * @return Node ID message.
+     */
+    private NodeIdMessage nodeIdMessage() {
+        return new NodeIdMessage(getLocalNode().id());
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TcpCommunicationSpi.class, this);
@@ -2692,10 +2755,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         private void processRecovery(GridNioRecoveryDescriptor recoveryDesc) {
             ClusterNode node = recoveryDesc.node();
 
-            if (clients.containsKey(node.id()) ||
-                !recoveryDesc.nodeAlive(getSpiContext().node(node.id())) ||
-                !getSpiContext().pingNode(node.id()))
+            try {
+                if (clients.containsKey(node.id()) ||
+                    !recoveryDesc.nodeAlive(getSpiContext().node(node.id())) ||
+                    !getSpiContext().pingNode(node.id()))
+                    return;
+            }
+            catch (IgniteClientDisconnectedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to ping node, client disconnected.");
+
                 return;
+            }
 
             try {
                 if (log.isDebugEnabled())
@@ -2860,15 +2931,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
 
             try {
+                UUID id = getLocalNode().id();
+
+                NodeIdMessage msg = new NodeIdMessage(id);
+
                 out.write(U.IGNITE_HEADER);
                 out.write(NODE_ID_MSG_TYPE);
-                out.write(nodeIdMsg.nodeIdBytes);
+                out.write(msg.nodeIdBytes);
 
                 out.flush();
 
                 if (log.isDebugEnabled())
-                    log.debug("Sent local node ID [locNodeId=" + getLocalNodeId() + ", rmtNodeId="
-                        + rmtNodeId + ']');
+                    log.debug("Sent local node ID [locNodeId=" + id + ", rmtNodeId=" + rmtNodeId + ']');
             }
             catch (IOException e) {
                 throw new IgniteCheckedException("Failed to perform handshake.", e);
@@ -3082,6 +3156,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
          * @param nodeId Node ID.
          */
         private NodeIdMessage(UUID nodeId) {
+            assert nodeId != null;
+
             nodeIdBytes = U.uuidToBytes(nodeId);
 
             nodeIdBytesWithType = new byte[nodeIdBytes.length + 1];
@@ -3131,4 +3207,86 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             return S.toString(NodeIdMessage.class, this);
         }
     }
+
+    /**
+     *
+     */
+    private class ConnectGateway {
+        /** */
+        private GridSpinReadWriteLock lock = new GridSpinReadWriteLock();
+
+        /** */
+        private IgniteException err;
+
+        /**
+         *
+         */
+        void enter() {
+            lock.readLock();
+
+            if (err != null) {
+                lock.readUnlock();
+
+                throw err;
+            }
+        }
+
+        /**
+         * @return {@code True} if entered gateway.
+         */
+        boolean tryEnter() {
+            lock.readLock();
+
+            boolean res = err == null;
+
+            if (!res)
+                lock.readUnlock();
+
+            return res;
+        }
+
+        /**
+         *
+         */
+        void leave() {
+            lock.readUnlock();
+        }
+
+        /**
+         * @param reconnectFut Reconnect future.
+         */
+        void disconnected(IgniteFuture<?> reconnectFut) {
+            lock.writeLock();
+
+            err = new IgniteClientDisconnectedException(reconnectFut, "Failed to connect, client node disconnected.");
+
+            lock.writeUnlock();
+        }
+
+        /**
+         *
+         */
+        void reconnected() {
+            lock.writeLock();
+
+            try {
+                if (err instanceof IgniteClientDisconnectedException)
+                    err = null;
+            }
+            finally {
+                lock.writeUnlock();
+            }
+        }
+
+        /**
+         *
+         */
+        void stopped() {
+            lock.readLock();
+
+            err = new IgniteException("Failed to connect, node stopped.");
+
+            lock.readUnlock();
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
index 46d6716..038ea59 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpiDataExchange.java
@@ -39,7 +39,8 @@ public interface DiscoverySpiDataExchange {
     /**
      * Notifies discovery manager about data received from remote node.
      *
-     * @param joiningNodeId Remote node ID.
+     * @param joiningNodeId ID of new node that joins topology.
+     * @param nodeId ID of the node provided data.
      * @param data Collection of discovery data objects from different components.
      */
     public void onExchange(UUID joiningNodeId, UUID nodeId, Map<Integer, Serializable> data);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 3f05f59..572ba2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -41,6 +42,7 @@ import java.util.concurrent.atomic.*;
 import static java.util.concurrent.TimeUnit.*;
 import static org.apache.ignite.events.EventType.*;
 import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
+import static org.apache.ignite.spi.discovery.tcp.ClientImpl.State.*;
 
 /**
  *
@@ -71,7 +73,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     private SocketReader sockReader;
 
     /** */
-    private boolean segmented;
+    private volatile State state;
 
     /** Last message ID. */
     private volatile IgniteUuid lastMsgId;
@@ -94,6 +96,10 @@ class ClientImpl extends TcpDiscoveryImpl {
     /** */
     protected MessageWorker msgWorker;
 
+    /** */
+    @GridToStringExclude
+    private int joinCnt;
+
     /**
      * @param adapter Adapter.
      */
@@ -157,6 +163,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         locNode = spi.locNode;
 
+        // Marshal credentials for backward compatibility and security.
+        marshalCredentials(locNode);
+
         sockWriter = new SocketWriter();
         sockWriter.start();
 
@@ -258,23 +267,36 @@ class ClientImpl extends TcpDiscoveryImpl {
             if (oldFut != null)
                 fut = oldFut;
             else {
-                if (spi.getSpiContext().isStopping()) {
+                State state = this.state;
+
+                if (spi.getSpiContext().isStopping() || state == STOPPED || state == SEGMENTED) {
                     if (pingFuts.remove(nodeId, fut))
                         fut.onDone(false);
 
                     return false;
                 }
+                else if (state == DISCONNECTED) {
+                    if (pingFuts.remove(nodeId, fut))
+                        fut.onDone(new IgniteClientDisconnectedCheckedException(null,
+                            "Failed to ping node, client node disconnected."));
+                }
+                else {
+                    final GridFutureAdapter<Boolean> finalFut = fut;
 
-                final GridFutureAdapter<Boolean> finalFut = fut;
-
-                timer.schedule(new TimerTask() {
-                    @Override public void run() {
-                        if (pingFuts.remove(nodeId, finalFut))
-                            finalFut.onDone(false);
-                    }
-                }, spi.netTimeout);
+                    timer.schedule(new TimerTask() {
+                        @Override public void run() {
+                            if (pingFuts.remove(nodeId, finalFut)) {
+                                if (ClientImpl.this.state == DISCONNECTED)
+                                    finalFut.onDone(new IgniteClientDisconnectedCheckedException(null,
+                                        "Failed to ping node, client node disconnected."));
+                                else
+                                    finalFut.onDone(false);
+                            }
+                        }
+                    }, spi.netTimeout);
 
-                sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
+                    sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
+                }
             }
         }
 
@@ -285,7 +307,7 @@ class ClientImpl extends TcpDiscoveryImpl {
             return false;
         }
         catch (IgniteCheckedException e) {
-            throw new IgniteSpiException(e); // Should newer occur.
+            throw new IgniteSpiException(e);
         }
     }
 
@@ -325,8 +347,13 @@ class ClientImpl extends TcpDiscoveryImpl {
 
     /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
-        if (segmented)
-            throw new IgniteException("Failed to send custom message: client is disconnected");
+        State state = this.state;
+
+        if (state == SEGMENTED)
+            throw new IgniteException("Failed to send custom message: client is segmented.");
+
+        if (state == DISCONNECTED)
+            throw new IgniteClientDisconnectedException(null, "Failed to send custom message: client is disconnected.");
 
         try {
             sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt,
@@ -361,14 +388,11 @@ class ClientImpl extends TcpDiscoveryImpl {
      * @see TcpDiscoverySpi#joinTimeout
      */
     @SuppressWarnings("BusyWait")
-    @Nullable private Socket joinTopology(boolean recon, long timeout) throws IgniteSpiException, InterruptedException {
+    @Nullable private T2<Socket, Boolean> joinTopology(boolean recon, long timeout) throws IgniteSpiException, InterruptedException {
         Collection<InetSocketAddress> addrs = null;
 
         long startTime = U.currentTimeMillis();
 
-        // Marshal credentials for backward compatibility and security.
-        marshalCredentials(locNode);
-
         while (true) {
             if (Thread.currentThread().isInterrupted())
                 throw new InterruptedException();
@@ -400,7 +424,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 InetSocketAddress addr = it.next();
 
-                T2<Socket, Integer> sockAndRes = sendJoinRequest(recon, addr);
+                T3<Socket, Integer, Boolean> sockAndRes = sendJoinRequest(recon, addr);
 
                 if (sockAndRes == null) {
                     it.remove();
@@ -414,7 +438,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 switch (sockAndRes.get2()) {
                     case RES_OK:
-                        return sock;
+                        return new T2<>(sock, sockAndRes.get3());
 
                     case RES_CONTINUE_JOIN:
                     case RES_WAIT:
@@ -445,9 +469,9 @@ class ClientImpl extends TcpDiscoveryImpl {
     /**
      * @param recon {@code True} if reconnects.
      * @param addr Address.
-     * @return Socket and connect response.
+     * @return Socket, connect response and client acknowledge support flag.
      */
-    @Nullable private T2<Socket, Integer> sendJoinRequest(boolean recon, InetSocketAddress addr) {
+    @Nullable private T3<Socket, Integer, Boolean> sendJoinRequest(boolean recon, InetSocketAddress addr) {
         assert addr != null;
 
         if (log.isDebugEnabled())
@@ -493,9 +517,18 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 tstamp = U.currentTimeMillis();
 
-                TcpDiscoveryAbstractMessage msg = recon ?
-                    new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId, lastMsgId) :
-                    new TcpDiscoveryJoinRequestMessage(locNode, spi.collectExchangeData(getLocalNodeId()));
+                TcpDiscoveryAbstractMessage msg;
+
+                if (!recon) {
+                    TcpDiscoveryNode node = locNode;
+
+                    if (locNode.order() > 0)
+                        node = locNode.clientReconnectNode();
+
+                    msg = new TcpDiscoveryJoinRequestMessage(node, spi.collectExchangeData(getLocalNodeId()));
+                }
+                else
+                    msg = new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId, lastMsgId);
 
                 msg.client(true);
 
@@ -507,7 +540,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                     log.debug("Message has been sent to address [msg=" + msg + ", addr=" + addr +
                         ", rmtNodeId=" + rmtNodeId + ']');
 
-                return new T2<>(sock, spi.readReceipt(sock, ackTimeout0));
+                return new T3<>(sock, spi.readReceipt(sock, ackTimeout0), res.clientAck());
             }
             catch (IOException | IgniteCheckedException e) {
                 U.closeQuiet(sock);
@@ -786,10 +819,16 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                         spi.stats.onMessageReceived(msg);
 
-                        if (spi.ensured(msg) && joinLatch.getCount() == 0L)
-                            lastMsgId = msg.id();
+                        boolean ack = msg instanceof TcpDiscoveryClientAckResponse;
+
+                        if (!ack) {
+                            if (spi.ensured(msg) && joinLatch.getCount() == 0L)
+                                lastMsgId = msg.id();
 
-                        msgWorker.addMessage(msg);
+                            msgWorker.addMessage(msg);
+                        }
+                        else
+                            sockWriter.ackReceived((TcpDiscoveryClientAckResponse)msg);
                     }
                 }
                 catch (IOException e) {
@@ -823,8 +862,14 @@ class ClientImpl extends TcpDiscoveryImpl {
         private Socket sock;
 
         /** */
+        private boolean clientAck;
+
+        /** */
         private final Queue<TcpDiscoveryAbstractMessage> queue = new ArrayDeque<>();
 
+        /** */
+        private TcpDiscoveryAbstractMessage unackedMsg;
+
         /**
          *
          */
@@ -845,11 +890,16 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         /**
          * @param sock Socket.
+         * @param clientAck {@code True} is server supports client message acknowlede.
          */
-        private void setSocket(Socket sock) {
+        private void setSocket(Socket sock, boolean clientAck) {
             synchronized (mux) {
                 this.sock = sock;
 
+                this.clientAck = clientAck;
+
+                unackedMsg = null;
+
                 mux.notifyAll();
             }
         }
@@ -863,6 +913,21 @@ class ClientImpl extends TcpDiscoveryImpl {
             }
         }
 
+        /**
+         * @param res Acknowledge response.
+         */
+        void ackReceived(TcpDiscoveryClientAckResponse res) {
+            synchronized (mux) {
+                if (unackedMsg != null) {
+                    assert unackedMsg.id().equals(res.messageId()) : unackedMsg;
+
+                    unackedMsg = null;
+                }
+
+                mux.notifyAll();
+            }
+        }
+
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
             TcpDiscoveryAbstractMessage msg = null;
@@ -892,10 +957,43 @@ class ClientImpl extends TcpDiscoveryImpl {
                 for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : spi.sendMsgLsnrs)
                     msgLsnr.apply(msg);
 
+                boolean ack = clientAck && !(msg instanceof TcpDiscoveryPingResponse);
+
                 try {
+                    if (ack) {
+                        synchronized (mux) {
+                            assert unackedMsg == null : unackedMsg;
+
+                            unackedMsg = msg;
+                        }
+                    }
+
                     spi.writeToSocket(sock, msg);
 
                     msg = null;
+
+                    if (ack) {
+                        long waitEnd = U.currentTimeMillis() + spi.ackTimeout;
+
+                        TcpDiscoveryAbstractMessage unacked;
+
+                        synchronized (mux) {
+                            while (unackedMsg != null && U.currentTimeMillis() < waitEnd)
+                                mux.wait(waitEnd);
+
+                            unacked = unackedMsg;
+
+                            unackedMsg = null;
+                        }
+
+                        if (unacked != null) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to get acknowledge for message, will try to reconnect " +
+                                "[msg=" + unacked + ", timeout=" + spi.ackTimeout + ']');
+
+                            throw new IOException("Failed to get acknowledge for message: " + unacked);
+                        }
+                    }
                 }
                 catch (IOException e) {
                     if (log.isDebugEnabled())
@@ -926,6 +1024,9 @@ class ClientImpl extends TcpDiscoveryImpl {
         private volatile Socket sock;
 
         /** */
+        private boolean clientAck;
+
+        /** */
         private boolean join;
 
         /**
@@ -948,8 +1049,6 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
-            assert !segmented;
-
             boolean success = false;
 
             Exception err = null;
@@ -958,11 +1057,14 @@ class ClientImpl extends TcpDiscoveryImpl {
 
             long startTime = U.currentTimeMillis();
 
+            if (log.isDebugEnabled())
+                log.debug("Started reconnect process [join=" + join + ", timeout=" + timeout + ']');
+
             try {
                 while (true) {
-                    sock = joinTopology(true, timeout);
+                    T2<Socket, Boolean> joinRes = joinTopology(true, timeout);
 
-                    if (sock == null) {
+                    if (joinRes == null) {
                         if (join) {
                             joinError(new IgniteSpiException("Join process timed out, connection failed and " +
                                 "failed to reconnect (consider increasing 'joinTimeout' configuration property) " +
@@ -970,11 +1072,14 @@ class ClientImpl extends TcpDiscoveryImpl {
                         }
                         else
                             U.error(log, "Failed to reconnect to cluster (consider increasing 'networkTimeout' " +
-                                "configuration  property) [networkTimeout=" + spi.netTimeout + ", sock=" + sock + ']');
+                                "configuration property) [networkTimeout=" + spi.netTimeout + ", sock=" + sock + ']');
 
                         return;
                     }
 
+                    sock = joinRes.get1();
+                    clientAck = joinRes.get2();
+
                     if (isInterrupted())
                         throw new InterruptedException();
 
@@ -999,6 +1104,10 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 TcpDiscoveryClientReconnectMessage res = (TcpDiscoveryClientReconnectMessage)msg;
 
                                 if (res.creatorNodeId().equals(getLocalNodeId())) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Received reconnect response [success=" + res.success() +
+                                            ", msg=" + msg + ']');
+
                                     if (res.success()) {
                                         msgWorker.addMessage(res);
 
@@ -1008,9 +1117,11 @@ class ClientImpl extends TcpDiscoveryImpl {
                                         }
 
                                         success = true;
-                                    }
 
-                                    return;
+                                        return;
+                                    }
+                                    else
+                                        return;
                                 }
                             }
                             else if (spi.ensured(msg)) {
@@ -1081,6 +1192,9 @@ class ClientImpl extends TcpDiscoveryImpl {
         /** */
         private Reconnector reconnector;
 
+        /** */
+        private boolean nodeAdded;
+
         /**
          *
          */
@@ -1091,45 +1205,37 @@ class ClientImpl extends TcpDiscoveryImpl {
         /** {@inheritDoc} */
         @SuppressWarnings("InfiniteLoopStatement")
         @Override protected void body() throws InterruptedException {
+            state = STARTING;
+
             spi.stats.onJoinStarted();
 
             try {
-                final Socket sock = joinTopology(false, spi.joinTimeout);
-
-                if (sock == null) {
-                    joinError(new IgniteSpiException("Join process timed out."));
-
-                    return;
-                }
-
-                currSock = sock;
-
-                sockWriter.setSocket(sock);
-
-                if (spi.joinTimeout > 0) {
-                    timer.schedule(new TimerTask() {
-                        @Override public void run() {
-                            if (joinLatch.getCount() > 0)
-                                queue.add(JOIN_TIMEOUT);
-                        }
-                    }, spi.joinTimeout);
-                }
-
-                sockReader.setSocket(sock, locNode.clientRouterNodeId());
+                tryJoin();
 
                 while (true) {
                     Object msg = queue.take();
 
                     if (msg == JOIN_TIMEOUT) {
-                        if (joinLatch.getCount() > 0) {
+                        if (state == STARTING) {
                             joinError(new IgniteSpiException("Join process timed out, did not receive response for " +
                                 "join request (consider increasing 'joinTimeout' configuration property) " +
-                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + sock + ']'));
+                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + currSock + ']'));
 
                             break;
                         }
+                        else if (state == DISCONNECTED) {
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to reconnect, local node segmented " +
+                                    "[joinTimeout=" + spi.joinTimeout + ']');
+
+                            state = SEGMENTED;
+
+                            notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                        }
                     }
                     else if (msg == SPI_STOP) {
+                        state = STOPPED;
+
                         assert spi.getSpiContext().isStopping();
 
                         if (currSock != null) {
@@ -1148,7 +1254,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                             boolean join = joinLatch.getCount() > 0;
 
-                            if (spi.getSpiContext().isStopping() || segmented) {
+                            if (spi.getSpiContext().isStopping() || state == SEGMENTED) {
                                 leaveLatch.countDown();
 
                                 if (join) {
@@ -1158,6 +1264,9 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 }
                             }
                             else {
+                                if (log.isDebugEnabled())
+                                    log.debug("Connection closed, will try to restore connection.");
+
                                 assert reconnector == null;
 
                                 final Reconnector reconnector = new Reconnector(join);
@@ -1167,19 +1276,64 @@ class ClientImpl extends TcpDiscoveryImpl {
                         }
                     }
                     else if (msg == SPI_RECONNECT_FAILED) {
-                        if (!segmented) {
-                            segmented = true;
+                        reconnector.cancel();
+                        reconnector.join();
 
-                            reconnector.cancel();
-                            reconnector.join();
+                        reconnector = null;
 
-                            notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                        if (spi.isClientReconnectDisabled()) {
+                            if (state != SEGMENTED && state != STOPPED) {
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Failed to restore closed connection, reconnect disabled, " +
+                                        "local node segmented [networkTimeout=" + spi.netTimeout + ']');
+                                }
+
+                                state = SEGMENTED;
+
+                                notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                            }
+                        }
+                        else {
+                            if (state == STARTING || state == CONNECTED) {
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Failed to restore closed connection, will try to reconnect " +
+                                        "[networkTimeout=" + spi.netTimeout + ", joinTimeout=" + spi.joinTimeout + ']');
+                                }
+
+                                state = DISCONNECTED;
+
+                                nodeAdded = false;
+
+                                IgniteClientDisconnectedCheckedException err =
+                                    new IgniteClientDisconnectedCheckedException(null, "Failed to ping node, " +
+                                    "client node disconnected.");
+
+                                for (Map.Entry<UUID, GridFutureAdapter<Boolean>> e : pingFuts.entrySet()) {
+                                    GridFutureAdapter<Boolean> fut = e.getValue();
+
+                                    if (pingFuts.remove(e.getKey(), fut))
+                                        fut.onDone(err);
+                                }
+
+                                notifyDiscovery(EVT_CLIENT_NODE_DISCONNECTED, topVer, locNode, allVisibleNodes());
+                            }
+
+                            UUID newId = UUID.randomUUID();
+
+                            if (log.isInfoEnabled()) {
+                                log.info("Client node disconnected from cluster, will try to reconnect with new id " +
+                                    "[newId=" + newId + ", prevId=" + locNode.id() + ", locNode=" + locNode + ']');
+                            }
+
+                            locNode.onClientDisconnected(newId);
+
+                            tryJoin();
                         }
                     }
                     else {
                         TcpDiscoveryAbstractMessage discoMsg = (TcpDiscoveryAbstractMessage)msg;
 
-                        if (joinLatch.getCount() > 0) {
+                        if (joining()) {
                             IgniteSpiException err = null;
 
                             if (discoMsg instanceof TcpDiscoveryDuplicateIdMessage)
@@ -1190,7 +1344,15 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 err = spi.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
 
                             if (err != null) {
-                                joinError(err);
+                                if (state == DISCONNECTED) {
+                                    U.error(log, "Failed to reconnect, segment local node.", err);
+
+                                    state = SEGMENTED;
+
+                                    notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                                }
+                                else
+                                    joinError(err);
 
                                 break;
                             }
@@ -1215,6 +1377,48 @@ class ClientImpl extends TcpDiscoveryImpl {
         }
 
         /**
+         * @throws InterruptedException If interrupted.
+         */
+        private void tryJoin() throws InterruptedException {
+            assert state == DISCONNECTED || state == STARTING : state;
+
+            boolean join = state == STARTING;
+
+            joinCnt++;
+
+            T2<Socket, Boolean> joinRes = joinTopology(false, spi.joinTimeout);
+
+            if (joinRes == null) {
+                if (join)
+                    joinError(new IgniteSpiException("Join process timed out."));
+                else {
+                    state = SEGMENTED;
+
+                    notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                }
+
+                return;
+            }
+
+            currSock = joinRes.get1();
+
+            sockWriter.setSocket(joinRes.get1(), joinRes.get2());
+
+            if (spi.joinTimeout > 0) {
+                final int joinCnt0 = joinCnt;
+
+                timer.schedule(new TimerTask() {
+                    @Override public void run() {
+                        if (joinCnt == joinCnt0 && joining())
+                            queue.add(JOIN_TIMEOUT);
+                    }
+                }, spi.joinTimeout);
+            }
+
+            sockReader.setSocket(joinRes.get1(), locNode.clientRouterNodeId());
+        }
+
+        /**
          * @param msg Message.
          */
         protected void processDiscoveryMessage(TcpDiscoveryAbstractMessage msg) {
@@ -1246,6 +1450,22 @@ class ClientImpl extends TcpDiscoveryImpl {
         }
 
         /**
+         * @return {@code True} if client in process of join.
+         */
+        private boolean joining() {
+            ClientImpl.State state = ClientImpl.this.state;
+
+            return state == STARTING || state == DISCONNECTED;
+        }
+
+        /**
+         * @return {@code True} if client disconnected.
+         */
+        private boolean disconnected() {
+            return state == DISCONNECTED;
+        }
+
+        /**
          * @param msg Message.
          */
         private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
@@ -1257,12 +1477,15 @@ class ClientImpl extends TcpDiscoveryImpl {
             UUID newNodeId = node.id();
 
             if (getLocalNodeId().equals(newNodeId)) {
-                if (joinLatch.getCount() > 0) {
+                if (joining()) {
                     Collection<TcpDiscoveryNode> top = msg.topology();
 
                     if (top != null) {
                         spi.gridStartTime = msg.gridStartTime();
 
+                        if (disconnected())
+                            rmtNodes.clear();
+
                         for (TcpDiscoveryNode n : top) {
                             if (n.order() > 0)
                                 n.visible(true);
@@ -1272,6 +1495,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                         topHist.clear();
 
+                        nodeAdded = true;
+
                         if (msg.topologyHistory() != null)
                             topHist.putAll(msg.topologyHistory());
                     }
@@ -1309,7 +1534,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 return;
 
             if (getLocalNodeId().equals(msg.nodeId())) {
-                if (joinLatch.getCount() > 0) {
+                if (joining()) {
                     Map<UUID, Map<Integer, byte[]>> dataMap = msg.clientDiscoData();
 
                     if (dataMap != null) {
@@ -1324,13 +1549,22 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     locNode.order(topVer);
 
-                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer, msg));
+                    Collection<ClusterNode> nodes = updateTopologyHistory(topVer, msg);
+
+                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, nodes);
+
+                    boolean disconnected = disconnected();
+
+                    state = CONNECTED;
+
+                    if (disconnected)
+                        notifyDiscovery(EVT_CLIENT_NODE_RECONNECTED, topVer, locNode, nodes);
+                    else
+                        spi.stats.onJoinFinished();
 
                     joinErr.set(null);;
 
                     joinLatch.countDown();
-
-                    spi.stats.onJoinFinished();
                 }
                 else if (log.isDebugEnabled())
                     log.debug("Discarding node add finished message (this message has already been processed) " +
@@ -1438,7 +1672,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @return {@code True} if received node added message for local node.
          */
         private boolean nodeAdded() {
-            return !topHist.isEmpty();
+            return nodeAdded;
         }
 
         /**
@@ -1539,7 +1773,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     currSock = reconnector.sock;
 
-                    sockWriter.setSocket(currSock);
+                    sockWriter.setSocket(currSock, reconnector.clientAck);
                     sockReader.setSocket(currSock, locNode.clientRouterNodeId());
 
                     reconnector = null;
@@ -1583,7 +1817,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          */
         private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
-            if (msg.verified() && joinLatch.getCount() == 0) {
+            if (msg.verified() && state == CONNECTED) {
                 DiscoverySpiListener lsnr = spi.lsnr;
 
                 if (lsnr != null) {
@@ -1719,4 +1953,24 @@ class ClientImpl extends TcpDiscoveryImpl {
             this.sock = sock;
         }
     }
+
+    /**
+     *
+     */
+    enum State {
+        /** */
+        STARTING,
+
+        /** */
+        CONNECTED,
+
+        /** */
+        DISCONNECTED,
+
+        /** */
+        SEGMENTED,
+
+        /** */
+        STOPPED
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index d51293e..1a28e86 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -1447,6 +1447,8 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (log.isDebugEnabled())
                 log.debug("Heartbeats sender has been started.");
 
+            UUID nodeId = getConfiguredNodeId();
+
             while (!isInterrupted()) {
                 if (spiStateCopy() != CONNECTED) {
                     if (log.isDebugEnabled())
@@ -1455,7 +1457,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getLocalNodeId());
+                TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(nodeId);
 
                 msg.verify(getLocalNodeId());
 
@@ -1593,39 +1595,47 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // Addresses registered in IP finder.
                 Collection<InetSocketAddress> regAddrs = spi.registeredAddresses();
 
-                // Remove all addresses that belong to alive nodes, leave dead-node addresses.
-                Collection<InetSocketAddress> rmvAddrs = F.view(
-                    regAddrs,
-                    F.notContains(currAddrs),
-                    new P1<InetSocketAddress>() {
-                        private final Map<InetSocketAddress, Boolean> pingResMap = new HashMap<>();
+                P1<InetSocketAddress> p = new P1<InetSocketAddress>() {
+                    private final Map<InetSocketAddress, Boolean> pingResMap = new HashMap<>();
 
-                        @Override public boolean apply(InetSocketAddress addr) {
-                            Boolean res = pingResMap.get(addr);
+                    @Override public boolean apply(InetSocketAddress addr) {
+                        Boolean res = pingResMap.get(addr);
 
-                            if (res == null) {
-                                try {
-                                    res = pingNode(addr, null).get1() != null;
-                                }
-                                catch (IgniteCheckedException e) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Failed to ping node [addr=" + addr +
-                                            ", err=" + e.getMessage() + ']');
-
-                                    res = false;
-                                }
-                                finally {
-                                    pingResMap.put(addr, res);
-                                }
+                        if (res == null) {
+                            try {
+                                res = pingNode(addr, null).get1() != null;
                             }
+                            catch (IgniteCheckedException e) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Failed to ping node [addr=" + addr +
+                                        ", err=" + e.getMessage() + ']');
 
-                            return !res;
+                                res = false;
+                            }
+                            finally {
+                                pingResMap.put(addr, res);
+                            }
                         }
+
+                        return !res;
                     }
-                );
+                };
+
+                ArrayList<InetSocketAddress> rmvAddrs = null;
+
+                for (InetSocketAddress addr : regAddrs) {
+                    boolean rmv = !F.contains(currAddrs, addr) && p.apply(addr);
+
+                    if (rmv) {
+                        if (rmvAddrs == null)
+                            rmvAddrs = new ArrayList<>();
+
+                        rmvAddrs.add(addr);
+                    }
+                }
 
                 // Unregister dead-nodes addresses.
-                if (!rmvAddrs.isEmpty()) {
+                if (rmvAddrs != null) {
                     spi.ipFinder.unregisterAddresses(rmvAddrs);
 
                     if (log.isDebugEnabled())
@@ -4077,7 +4087,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
-            UUID locNodeId = getLocalNodeId();
+            UUID locNodeId = getConfiguredNodeId();
 
             ClientMessageWorker clientMsgWrk = null;
 
@@ -4170,6 +4180,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                     TcpDiscoveryHandshakeResponse res =
                         new TcpDiscoveryHandshakeResponse(locNodeId, locNode.internalOrder());
 
+                    if (req.client())
+                        res.clientAck(true);
+
                     spi.writeToSocket(sock, res);
 
                     // It can happen if a remote node is stopped and it has a loopback address in the list of addresses,
@@ -4313,7 +4326,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                 if (state == CONNECTED) {
                                     spi.writeToSocket(msg, sock, RES_OK);
 
-                                    if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW)
+                                    if (clientMsgWrk.getState() == State.NEW)
                                         clientMsgWrk.start();
 
                                     msgWorker.addMessage(msg);
@@ -4457,7 +4470,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         msgWorker.addMessage(msg);
 
                         // Send receipt back.
-                        if (clientMsgWrk == null)
+                        if (clientMsgWrk != null) {
+                            TcpDiscoveryClientAckResponse ack = new TcpDiscoveryClientAckResponse(locNodeId, msg.id());
+
+                            ack.verify(locNodeId);
+
+                            clientMsgWrk.addMessage(ack);
+                        }
+                        else
                             spi.writeToSocket(msg, sock, RES_OK);
                     }
                     catch (IgniteCheckedException e) {
@@ -4567,8 +4587,11 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 msg.responded(true);
 
-                if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW)
+                if (clientMsgWrk != null && clientMsgWrk.getState() == State.NEW) {
+                    clientMsgWrk.clientVersion(U.productVersion(msg.node()));
+
                     clientMsgWrk.start();
+                }
 
                 msgWorker.addMessage(msg);
 
@@ -4679,6 +4702,9 @@ class ServerImpl extends TcpDiscoveryImpl {
         /** */
         private final AtomicReference<GridFutureAdapter<Boolean>> pingFut = new AtomicReference<>();
 
+        /** */
+        private IgniteProductVersion clientVer;
+
         /**
          * @param sock Socket.
          * @param clientNodeId Node ID.
@@ -4691,6 +4717,13 @@ class ServerImpl extends TcpDiscoveryImpl {
         }
 
         /**
+         * @param clientVer Client version.
+         */
+        void clientVersion(IgniteProductVersion clientVer) {
+            this.clientVer = clientVer;
+        }
+
+        /**
          * @return Current client metrics.
          */
         ClusterMetrics metrics() {
@@ -4709,17 +4742,40 @@ class ServerImpl extends TcpDiscoveryImpl {
             try {
                 assert msg.verified() : msg;
 
-                if (log.isDebugEnabled())
-                    log.debug("Redirecting message to client [sock=" + sock + ", locNodeId="
-                        + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']');
+                if (msg instanceof TcpDiscoveryClientAckResponse) {
+                    if (clientVer == null) {
+                        ClusterNode node = spi.getNode(clientNodeId);
 
-                try {
-                    prepareNodeAddedMessage(msg, clientNodeId, null, null);
+                        if (node != null)
+                            clientVer = IgniteUtils.productVersion(node);
+                        else if (log.isDebugEnabled())
+                            log.debug("Skip sending message ack to client, fail to get client node " +
+                                "[sock=" + sock + ", locNodeId=" + getLocalNodeId() +
+                                ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']');
+                    }
+
+                    if (clientVer != null &&
+                        clientVer.compareTo(TcpDiscoveryClientAckResponse.CLIENT_ACK_SINCE_VERSION) >= 0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Sending message ack to client [sock=" + sock + ", locNodeId="
+                                + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']');
 
-                    writeToSocket(sock, msg);
+                        writeToSocket(sock, msg);
+                    }
                 }
-                finally {
-                    clearNodeAddedMessage(msg);
+                else {
+                    try {
+                        if (log.isDebugEnabled())
+                            log.debug("Redirecting message to client [sock=" + sock + ", locNodeId="
+                                + getLocalNodeId() + ", rmtNodeId=" + clientNodeId + ", msg=" + msg + ']');
+
+                        prepareNodeAddedMessage(msg, clientNodeId, null, null);
+
+                        writeToSocket(sock, msg);
+                    }
+                    finally {
+                        clearNodeAddedMessage(msg);
+                    }
                 }
             }
             catch (IgniteCheckedException | IOException e) {
@@ -4829,7 +4885,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException {
             if (log.isDebugEnabled())
-                log.debug("Message worker started [locNodeId=" + getLocalNodeId() + ']');
+                log.debug("Message worker started [locNodeId=" + getConfiguredNodeId() + ']');
 
             while (!isInterrupted()) {
                 TcpDiscoveryAbstractMessage msg = queue.poll(2000, TimeUnit.MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index ace917f..c271b7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -112,7 +112,14 @@ abstract class TcpDiscoveryImpl {
      * @return Local node ID.
      */
     public UUID getLocalNodeId() {
-        return spi.getLocalNodeId();
+        return spi.locNode.id();
+    }
+
+    /**
+     * @return Configured node ID (actual node ID can be different if client reconnects).
+     */
+    public UUID getConfiguredNodeId() {
+        return spi.cfgNodeId;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 7663fe6..431d198 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -260,6 +260,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** Local node. */
     protected TcpDiscoveryNode locNode;
 
+    /** */
+    protected UUID cfgNodeId;
+
     /** Local host. */
     protected InetAddress locHost;
 
@@ -327,6 +330,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** */
     private boolean forceSrvMode;
 
+    /** */
+    private boolean clientReconnectDisabled;
+
     /** {@inheritDoc} */
     @Override public String getSpiState() {
         return impl.getSpiState();
@@ -417,6 +423,29 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
+     * If {@code true} client does not try to reconnect after
+     * server detected client node failure.
+     *
+     * @return Client reconnect disabled flag.
+     */
+    public boolean isClientReconnectDisabled() {
+        return clientReconnectDisabled;
+    }
+
+    /**
+     * Sets client reconnect disabled flag.
+     * <p>
+     * If {@code true} client does not try to reconnect after
+     * server detected client node failure.
+     *
+     * @param clientReconnectDisabled Client reconnect disabled flag.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setClientReconnectDisabled(boolean clientReconnectDisabled) {
+        this.clientReconnectDisabled = clientReconnectDisabled;
+    }
+
+    /**
      * Inject resources
      *
      * @param ignite Ignite.
@@ -844,7 +873,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
         }
 
         locNode = new TcpDiscoveryNode(
-            getLocalNodeId(),
+            ignite.configuration().getNodeId(),
             addrs.get1(),
             addrs.get2(),
             srvPort,
@@ -1615,6 +1644,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
                 mcastIpFinder.setLocalAddress(locAddr);
         }
 
+        cfgNodeId = ignite.configuration().getNodeId();
+
         impl.spiStart(gridName);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
index 22f56c3..032cf01 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryNode.java
@@ -441,6 +441,25 @@ public class TcpDiscoveryNode extends GridMetadataAwareAdapter implements Cluste
         this.clientRouterNodeId = clientRouterNodeId;
     }
 
+    /**
+     * @param newId New node ID.
+     */
+    public void onClientDisconnected(UUID newId) {
+        id = newId;
+    }
+
+    /**
+     * @return Copy of local node for client reconnect request.
+     */
+    public TcpDiscoveryNode clientReconnectNode() {
+        TcpDiscoveryNode node = new TcpDiscoveryNode(id, addrs, hostNames, discPort, metricsProvider, ver);
+
+        node.attrs = attrs;
+        node.clientRouterNodeId = clientRouterNodeId;
+
+        return node;
+    }
+
     /** {@inheritDoc} */
     @Override public int compareTo(@Nullable TcpDiscoveryNode node) {
         if (node == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
index 21dbf4f..6f52152 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryAbstractMessage.java
@@ -40,6 +40,9 @@ public abstract class TcpDiscoveryAbstractMessage implements Serializable {
     /** */
     protected static final int CLIENT_RECON_SUCCESS_FLAG_POS = 2;
 
+    /** */
+    protected static final int CLIENT_ACK_FLAG_POS = 4;
+
     /** Sender of the message (transient). */
     private transient UUID sndNodeId;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
new file mode 100644
index 0000000..ce3943a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryClientAckResponse.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.discovery.tcp.messages;
+
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class TcpDiscoveryClientAckResponse extends TcpDiscoveryAbstractMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    public static final IgniteProductVersion CLIENT_ACK_SINCE_VERSION = IgniteProductVersion.fromString("1.4.1");
+
+    /** */
+    private final IgniteUuid msgId;
+
+    /**
+     * @param creatorNodeId Creator node ID.
+     * @param msgId Message ID to ack.
+     */
+    public TcpDiscoveryClientAckResponse(UUID creatorNodeId, IgniteUuid msgId) {
+        super(creatorNodeId);
+
+        this.msgId = msgId;
+    }
+
+    /**
+     * @return Acknowledged message ID.
+     */
+    public IgniteUuid messageId() {
+        return msgId;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean highPriority() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(TcpDiscoveryClientAckResponse.class, this, "super", super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java
index 5c2f798..ac4be50 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryHandshakeResponse.java
@@ -61,6 +61,20 @@ public class TcpDiscoveryHandshakeResponse extends TcpDiscoveryAbstractMessage {
         this.order = order;
     }
 
+    /**
+     * @return {@code True} if server supports client message acknowledge.
+     */
+    public boolean clientAck() {
+        return getFlag(CLIENT_ACK_FLAG_POS);
+    }
+
+    /**
+     * @param clientAck {@code True} if server supports client message acknowledge.
+     */
+    public void clientAck(boolean clientAck) {
+        setFlag(CLIENT_ACK_FLAG_POS, clientAck);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TcpDiscoveryHandshakeResponse.class, this, "super", super.toString());

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
index 7a88426..000782a 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -257,7 +257,7 @@ public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi,
 
         registerMBean(gridName, this, FileSwapSpaceSpiMBean.class);
 
-        String path = baseDir + File.separator + gridName + File.separator + getLocalNodeId();
+        String path = baseDir + File.separator + gridName + File.separator + ignite.configuration().getNodeId();
 
         try {
             dir = U.resolveWorkDirectory(path, true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
index abc9109..bf499c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;
@@ -104,8 +105,6 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
      * Test kernal gateway that always return uninitialized user stack trace.
      */
     private static final GridKernalGateway TEST_GATEWAY = new GridKernalGateway() {
-        @Override public void lightCheck() throws IllegalStateException {}
-
         @Override public void readLock() throws IllegalStateException {}
 
         @Override public void readLockAnyway() {}
@@ -122,10 +121,6 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
 
         @Override public void writeUnlock() {}
 
-        @Override public void addStopListener(Runnable lsnr) {}
-
-        @Override public void removeStopListener(Runnable lsnr) {}
-
         @Override public String userStackTrace() {
             return null;
         }
@@ -133,5 +128,13 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
         @Override public boolean tryWriteLock(long timeout) {
             return false;
         }
+
+        @Override public GridFutureAdapter<?> onDisconnected() {
+            return null;
+        }
+
+        @Override public void onReconnected() {
+            // No-op.
+        }
     };
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
new file mode 100644
index 0000000..fbaea11
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
@@ -0,0 +1,363 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jetbrains.annotations.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.net.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final long RECONNECT_TIMEOUT = 10_000;
+
+    /** */
+    protected boolean clientMode;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TestTcpDiscoverySpi disco = new TestTcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+        disco.setJoinTimeout(2 * 60_000);
+        disco.setSocketTimeout(1000);
+        disco.setNetworkTimeout(2000);
+
+        cfg.setDiscoverySpi(disco);
+
+        BlockTpcCommunicationSpi commSpi = new BlockTpcCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        if (clientMode)
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /**
+     * @param latch Latch.
+     * @throws Exception If failed.
+     */
+    protected void waitReconnectEvent(CountDownLatch latch) throws Exception {
+        if (!latch.await(RECONNECT_TIMEOUT, MILLISECONDS)) {
+            log.error("Failed to wait for reconnect event, will dump threads, latch count: " + latch.getCount());
+
+            U.dumpThreads(log);
+
+            fail("Failed to wait for disconnect/reconnect event.");
+        }
+    }
+
+    /**
+     * @return Number of server nodes started before tests.
+     */
+    protected abstract int serverCount();
+
+    /**
+     * @return Number of client nodes started before tests.
+     */
+    protected int clientCount() {
+        return 0;
+    }
+
+    /**
+     * @param ignite Node.
+     * @return Discovery SPI.
+     */
+    protected TestTcpDiscoverySpi spi(Ignite ignite) {
+        return ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
+    }
+
+    /**
+     * @param ignite Node.
+     * @return Communication SPI.
+     */
+    protected BlockTpcCommunicationSpi commSpi(Ignite ignite) {
+        return ((BlockTpcCommunicationSpi)ignite.configuration().getCommunicationSpi());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        int srvs = serverCount();
+
+        if (srvs > 0)
+            startGrids(srvs);
+
+        int clients = clientCount();
+
+        if (clients > 0) {
+            clientMode = true;
+
+            startGridsMultiThreaded(srvs, clients);
+
+            clientMode = false;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @param client Client.
+     * @return Server node client connected to.
+     */
+    protected Ignite clientRouter(Ignite client) {
+        TcpDiscoveryNode node = (TcpDiscoveryNode)client.cluster().localNode();
+
+        assertTrue(node.isClient());
+        assertNotNull(node.clientRouterNodeId());
+
+        Ignite srv = G.ignite(node.clientRouterNodeId());
+
+        assertNotNull(srv);
+
+        return srv;
+    }
+
+    /**
+     * @param fut Future.
+     * @throws Exception If failed.
+     */
+    protected void assertNotDone(IgniteInternalFuture<?> fut) throws Exception {
+        assertNotNull(fut);
+
+        if (fut.isDone())
+            fail("Future completed with result: " + fut.get());
+    }
+
+    /**
+     * Reconnect client node.
+     *
+     * @param client Client.
+     * @param srv Server.
+     * @param disconnectedC Closure which will be run when client node disconnected.
+     * @throws Exception If failed.
+     */
+    protected void reconnectClientNode(Ignite client, Ignite srv, @Nullable Runnable disconnectedC)
+        throws Exception {
+        final TestTcpDiscoverySpi clientSpi = spi(client);
+        final TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        log.info("Block reconnect.");
+
+        clientSpi.writeLatch = new CountDownLatch(1);
+
+        IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        };
+
+        client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(disconnectLatch);
+
+        if (disconnectedC != null)
+            disconnectedC.run();
+
+        log.info("Allow reconnect.");
+
+        clientSpi.writeLatch.countDown();
+
+        waitReconnectEvent(reconnectLatch);
+
+        client.events().stopLocalListen(p);
+    }
+
+    /**
+     * @param e Client disconnected exception.
+     * @return Reconnect future.
+     */
+    protected IgniteFuture<?> check(CacheException e) {
+        log.info("Expected exception: " + e);
+
+        if (!(e.getCause() instanceof IgniteClientDisconnectedException))
+            log.error("Unexpected cause: " + e.getCause(), e);
+
+        assertTrue("Unexpected cause: " + e.getCause(), e.getCause() instanceof IgniteClientDisconnectedException);
+
+        IgniteClientDisconnectedException e0 = (IgniteClientDisconnectedException)e.getCause();
+
+        assertNotNull(e0.reconnectFuture());
+
+        return e0.reconnectFuture();
+    }
+
+    /**
+     * @param e Client disconnected exception.
+     */
+    protected void checkAndWait(CacheException e) {
+        check(e).get();
+    }
+
+    /**
+     * @param e Client disconnected exception.
+     */
+    protected void checkAndWait(IgniteClientDisconnectedException e) {
+        log.info("Expected exception: " + e);
+
+        assertNotNull(e.reconnectFuture());
+
+        e.reconnectFuture().get();
+    }
+
+    /**
+     *
+     */
+    protected static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
+        /** */
+        volatile CountDownLatch writeLatch;
+
+        /** {@inheritDoc} */
+        @Override protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg)
+            throws IOException, IgniteCheckedException {
+            if (msg instanceof TcpDiscoveryJoinRequestMessage) {
+                CountDownLatch writeLatch0 = writeLatch;
+
+                if (writeLatch0 != null) {
+                    log.info("Block join request send: " + msg);
+
+                    U.await(writeLatch0);
+                }
+            }
+
+            super.writeToSocket(sock, msg);
+        }
+    }
+
+    /**
+     *
+     */
+    protected static class BlockTpcCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        volatile Class msgCls;
+
+        /** */
+        AtomicBoolean collectStart = new AtomicBoolean(false);
+
+        /** */
+        ConcurrentHashMap<String, ClusterNode> classes = new ConcurrentHashMap<>();
+
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            Class msgCls0 = msgCls;
+
+            if (collectStart.get() && msg instanceof GridIoMessage)
+                classes.put(((GridIoMessage)msg).message().getClass().getName(), node);
+
+            if (msgCls0 != null && msg instanceof GridIoMessage
+                && ((GridIoMessage)msg).message().getClass().equals(msgCls)) {
+                log.info("Block message: " + msg);
+
+                return;
+            }
+
+            super.sendMessage(node, msg);
+        }
+
+        /**
+         * @param clazz Class of messages which will be block.
+         */
+        public void blockMessage(Class clazz) {
+            msgCls = clazz;
+        }
+
+        /**
+         * Unlock all message.
+         */
+        public void unblockMessage() {
+            msgCls = null;
+        }
+
+        /**
+         * Start collect messages.
+         */
+        public void start() {
+            collectStart.set(true);
+        }
+
+        /**
+         * Print collected messages.
+         */
+        public void print() {
+            for (String s : classes.keySet())
+                log.error(s);
+        }
+    }
+}


[42/50] [abbrv] ignite git commit: minor

Posted by nt...@apache.org.
minor


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

Branch: refs/heads/ignite-788-dev
Commit: d24c2da35ac300484566bf2fe40d2664e4942686
Parents: 05672fe
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jul 20 11:53:41 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jul 20 11:53:41 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/IgniteCacheProxy.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d24c2da3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index 0b2eba0..e532778 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -556,7 +556,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                 if (isReplicatedDataNode() || ctx.isLocal() || qry.isLocal())
                     return (QueryCursor<R>)new QueryCursorImpl<>(new Iterable<Cache.Entry<K, V>>() {
                         @Override public Iterator<Cache.Entry<K, V>> iterator() {
-                            return ctx.kernalContext().query().<K, V>queryLocal(ctx, p);
+                            return ctx.kernalContext().query().queryLocal(ctx, p);
                         }
                     });
 


[47/50] [abbrv] ignite git commit: # master fixed test

Posted by nt...@apache.org.
# master fixed test


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

Branch: refs/heads/ignite-788-dev
Commit: d623d7e4913998b29a3ba91e78e8409cd1d89d46
Parents: cf08181
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 14:42:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 14:42:52 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/IgniteClientReconnectFailoverTest.java     | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d623d7e4/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
index b7de3eb..ab0e726 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
@@ -148,6 +148,12 @@ public class IgniteClientReconnectFailoverTest extends IgniteClientReconnectFail
 
                     assertEquals(map, res);
                 }
+                catch (IgniteClientDisconnectedException e) {
+                    throw e;
+                }
+                catch (IgniteException e) {
+                    log.info("Ignore error: " + e);
+                }
                 catch (CacheException e) {
                     if (e.getCause() instanceof IgniteClientDisconnectedException)
                         throw e;


[34/50] [abbrv] ignite git commit: GG-10556 Fixed stuck on unmarshal security subject.

Posted by nt...@apache.org.
GG-10556 Fixed stuck on unmarshal security subject.


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

Branch: refs/heads/ignite-788-dev
Commit: 6711d2c799b99d7a0ae3855475cccafbf362f5dd
Parents: 7d27103
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jul 15 10:30:12 2015 +0300
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jul 17 00:05:31 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 8 +++-----
 1 file changed, 3 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6711d2c7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index d51293e..a5ae5a9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -631,8 +631,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                         Map<String, Object> attrs = new HashMap<>(locNode.attributes());
 
-                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                            spi.ignite().configuration().getMarshaller().marshal(subj));
+                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marsh.marshal(subj));
                         attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
 
                         locNode.setAttributes(attrs);
@@ -2586,8 +2585,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             // Stick in authentication subject to node (use security-safe attributes for copy).
                             Map<String, Object> attrs = new HashMap<>(node.getAttributes());
 
-                            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                                spi.ignite().configuration().getMarshaller().marshal(subj));
+                            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT, spi.marsh.marshal(subj));
 
                             node.setAttributes(attrs);
                         }
@@ -2935,7 +2933,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         else {
                             SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
-                            SecurityContext coordSubj = spi.ignite().configuration().getMarshaller().unmarshal(
+                            SecurityContext coordSubj = spi.marsh.unmarshal(
                                 node.<byte[]>attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT),
                                 U.gridClassLoader());
 


[30/50] [abbrv] 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/fbc38d77
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/fbc38d77
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/fbc38d77

Branch: refs/heads/ignite-788-dev
Commit: fbc38d779cbbc2125f1837501a87ebec9c587f9d
Parents: 954ae15 17f8c85
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Jul 16 15:26:27 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Jul 16 15:26:27 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteCluster.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------



[17/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectQueriesTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectQueriesTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectQueriesTest.java
new file mode 100644
index 0000000..547adcb
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectQueriesTest.java
@@ -0,0 +1,427 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.query.*;
+import org.apache.ignite.internal.processors.query.h2.twostep.messages.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectQueriesTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    public static final String QUERY_CACHE = "query";
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<Integer, Person> ccfg = new CacheConfiguration<Integer, Person>(QUERY_CACHE)
+            .setCacheMode(PARTITIONED)
+            .setAtomicityMode(ATOMIC)
+            .setBackups(1)
+            .setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        grid(0).getOrCreateCache(QUERY_CACHE).removeAll();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueryReconnect() throws Exception {
+        Ignite cln = grid(serverCount());
+
+        assertTrue(cln.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(cln);
+
+        final IgniteCache<Integer, Person> clnCache = cln.getOrCreateCache(QUERY_CACHE);
+
+        final IgniteCache<Integer, Person> srvCache = srv.getOrCreateCache(QUERY_CACHE);
+
+        clnCache.put(1, new Person(1, "name1", "surname1"));
+        clnCache.put(2, new Person(2, "name2", "surname2"));
+        clnCache.put(3, new Person(3, "name3", "surname3"));
+
+        final SqlQuery<Integer, Person> qry = new SqlQuery<>(Person.class, "_key <> 0");
+
+        qry.setPageSize(1);
+
+        QueryCursor<Cache.Entry<Integer, Person>> cur = clnCache.query(qry);
+
+        reconnectClientNode(cln, srv, new Runnable() {
+            @Override public void run() {
+                srvCache.put(4, new Person(4, "name4", "surname4"));
+
+                try {
+                    clnCache.query(qry);
+
+                    fail();
+                } catch (CacheException e) {
+                    check(e);
+                }
+            }
+        });
+
+        List<Cache.Entry<Integer, Person>> res = cur.getAll();
+
+        assertNotNull(res);
+        assertEquals(4, res.size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectQueryInProgress() throws Exception {
+        Ignite cln = grid(serverCount());
+
+        assertTrue(cln.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(cln);
+
+        final IgniteCache<Integer, Person> clnCache = cln.getOrCreateCache(QUERY_CACHE);
+
+        clnCache.put(1, new Person(1, "name1", "surname1"));
+        clnCache.put(2, new Person(2, "name2", "surname2"));
+        clnCache.put(3, new Person(3, "name3", "surname3"));
+
+        blockMessage(GridQueryNextPageResponse.class);
+
+        final SqlQuery<Integer, Person> qry = new SqlQuery<>(Person.class, "_key <> 0");
+
+        qry.setPageSize(1);
+
+        final QueryCursor<Cache.Entry<Integer, Person>> cur1 = clnCache.query(qry);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    cur1.getAll();
+                }
+                catch (CacheException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        unblockMessage();
+
+        reconnectClientNode(cln, srv, null);
+
+        assertTrue((Boolean) fut.get(2, SECONDS));
+
+        QueryCursor<Cache.Entry<Integer, Person>> cur2 = clnCache.query(qry);
+
+        assertEquals(3, cur2.getAll().size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testScanQueryReconnect() throws Exception {
+        Ignite cln = grid(serverCount());
+
+        assertTrue(cln.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(cln);
+
+        final IgniteCache<Integer, Person> clnCache = cln.getOrCreateCache(QUERY_CACHE);
+
+        final IgniteCache<Integer, Person> srvCache = srv.getOrCreateCache(QUERY_CACHE);
+
+        for (int i = 0; i < 10_000; i++)
+            clnCache.put(i, new Person(i, "name-" + i, "surname-" + i));
+
+        final ScanQuery<Integer, Person> scanQry = new ScanQuery<>();
+
+        scanQry.setPageSize(1);
+
+        scanQry.setFilter(new IgniteBiPredicate<Integer, Person>() {
+            @Override public boolean apply(Integer integer, Person person) {
+                return true;
+            }
+        });
+
+        QueryCursor<Cache.Entry<Integer, Person>> qryCursor = clnCache.query(scanQry);
+
+        reconnectClientNode(cln, srv, new Runnable() {
+            @Override public void run() {
+                srvCache.put(10_001, new Person(10_001, "name", "surname"));
+
+                try {
+                    clnCache.query(scanQry);
+
+                    fail();
+                } catch (CacheException e) {
+                    check(e);
+                }
+            }
+        });
+
+        try {
+            qryCursor.getAll();
+
+            fail();
+        }
+        catch (CacheException e) {
+            checkAndWait(e);
+        }
+
+        qryCursor = clnCache.query(scanQry);
+
+        assertEquals(10_001, qryCursor.getAll().size());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testScanQueryReconnectInProgress1() throws Exception {
+        scanQueryReconnectInProgress(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testScanQueryReconnectInProgress2() throws Exception {
+        scanQueryReconnectInProgress(true);
+    }
+
+    /**
+     * @param setPart If {@code true} sets partition for scan query.
+     * @throws Exception If failed.
+     */
+    private void scanQueryReconnectInProgress(boolean setPart) throws Exception {
+        Ignite cln = grid(serverCount());
+
+        assertTrue(cln.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(cln);
+
+        final IgniteCache<Integer, Person> clnCache = cln.getOrCreateCache(QUERY_CACHE);
+
+        clnCache.put(1, new Person(1, "name1", "surname1"));
+        clnCache.put(2, new Person(2, "name2", "surname2"));
+        clnCache.put(3, new Person(3, "name3", "surname3"));
+
+        final ScanQuery<Integer, Person> scanQry = new ScanQuery<>();
+
+        scanQry.setPageSize(1);
+
+        scanQry.setFilter(new IgniteBiPredicate<Integer, Person>() {
+            @Override public boolean apply(Integer integer, Person person) {
+                return true;
+            }
+        });
+
+        if (setPart)
+            scanQry.setPartition(1);
+
+        blockMessage(GridCacheQueryResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    QueryCursor<Cache.Entry<Integer, Person>> qryCursor = clnCache.query(scanQry);
+
+                    qryCursor.getAll();
+                }
+                catch (CacheException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        unblockMessage();
+
+        reconnectClientNode(cln, srv, null);
+
+        assertTrue((Boolean)fut.get(2, SECONDS));
+
+        QueryCursor<Cache.Entry<Integer, Person>> qryCursor2 = clnCache.query(scanQry);
+
+        assertEquals(setPart ? 1 : 3, qryCursor2.getAll().size());
+    }
+
+    /**
+     * @param clazz Message class.
+     */
+    private void blockMessage(Class<?> clazz) {
+        for (int i = 0; i < serverCount(); i++) {
+            BlockTpcCommunicationSpi commSpi = commSpi(grid(i));
+
+            commSpi.blockMessage(clazz);
+        }
+    }
+
+    /**
+     *
+     */
+    private void unblockMessage() {
+        for (int i = 0; i < serverCount(); i++) {
+            BlockTpcCommunicationSpi commSpi = commSpi(grid(i));
+
+            commSpi.unblockMessage();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class Person {
+        /** */
+        @QuerySqlField
+        public int id;
+
+        /** */
+        @QuerySqlField
+        public String name;
+
+        /** */
+        @QuerySqlField
+        public String surname;
+
+        /**
+         * @param id Id.
+         * @param name Name.
+         * @param surname Surname.
+         */
+        public Person(int id, String name, String surname) {
+            this.id = id;
+            this.name = name;
+            this.surname = surname;
+        }
+
+        /**
+         * @return Id.
+         */
+        public int getId() {
+            return id;
+        }
+
+        /**
+         * @param id Set id.
+         */
+        public void setId(int id) {
+            this.id = id;
+        }
+
+        /**
+         * @return Name.
+         */
+        public String getName() {
+            return name;
+        }
+
+        /**
+         * @param name Name.
+         */
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        /**
+         * @return Surname.
+         */
+        public String getSurname() {
+            return surname;
+        }
+
+        /**
+         * @param surname Surname.
+         */
+        public void setSurname(String surname) {
+            this.surname = surname;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return this == o || !(o == null || getClass() != o.getClass()) && id == ((Person)o).id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java
index 8ab2485..ca80acf 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/reducefields/GridCacheAbstractReduceFieldsQuerySelfTest.java
@@ -375,6 +375,7 @@ public abstract class GridCacheAbstractReduceFieldsQuerySelfTest extends GridCom
         /** */
         private int cnt;
 
+        /** {@inheritDoc} */
         @Override public boolean collect(List<?> e) {
             sum += (Integer)e.get(0);
 
@@ -383,6 +384,7 @@ public abstract class GridCacheAbstractReduceFieldsQuerySelfTest extends GridCom
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override public IgniteBiTuple<Integer, Integer> reduce() {
             return F.t(sum, cnt);
         }
@@ -398,6 +400,7 @@ public abstract class GridCacheAbstractReduceFieldsQuerySelfTest extends GridCom
         /** */
         private int cnt;
 
+        /** {@inheritDoc} */
         @Override public boolean collect(IgniteBiTuple<Integer, Integer> t) {
             sum += t.get1();
             cnt += t.get2();
@@ -405,6 +408,7 @@ public abstract class GridCacheAbstractReduceFieldsQuerySelfTest extends GridCom
             return true;
         }
 
+        /** {@inheritDoc} */
         @Override public Integer reduce() {
             return cnt == 0 ? 0 : sum / cnt;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
index 67ebda9..cc01540 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheWithIndexingTestSuite.java
@@ -52,6 +52,7 @@ public class IgniteCacheWithIndexingTestSuite extends TestSuite {
         suite.addTestSuite(CacheConfigurationP2PTest.class);
 
         suite.addTestSuite(IgniteCacheConfigurationPrimitiveTypesSelfTest.class);
+        suite.addTestSuite(IgniteClientReconnectQueriesTest.class);
 
         return suite;
     }


[25/50] [abbrv] ignite git commit: checksum spaces fix

Posted by nt...@apache.org.
checksum spaces fix


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

Branch: refs/heads/ignite-788-dev
Commit: 700600d82d5b8faa474cbd6686f33f2db36f3c35
Parents: aef4063
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Jul 16 13:16:44 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Jul 16 13:16:44 2015 +0300

----------------------------------------------------------------------
 pom.xml | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/700600d8/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 344db40..a6ca9f5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -607,14 +607,12 @@
                                         </copy>
 
                                         <!-- appending filename to md5 and sha1 files. to be improved. -->
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
-
-                                        <copy file="${basedir}/KEYS" todir="${basedir}/target/site" failonerror="false" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message="  ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message="  ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message="  ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message="  ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message="  ${project.artifactId}-${project.version}-src.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message="  ${project.artifactId}-${project.version}-src.zip" />
                                     </target>
                                 </configuration>
                             </execution>


[24/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
# ignite-901 client reconnect support


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

Branch: refs/heads/ignite-788-dev
Commit: 57ac2b3bf437c037904624d411fd89b28b22c944
Parents: aef4063
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jul 16 13:06:04 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jul 16 13:06:05 2015 +0300

----------------------------------------------------------------------
 .../IgniteClientDisconnectedException.java      |   61 +
 .../java/org/apache/ignite/IgniteCluster.java   |    5 +
 .../apache/ignite/internal/GridComponent.java   |   18 +
 .../ignite/internal/GridJobSiblingImpl.java     |    2 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |   31 +-
 .../ignite/internal/GridKernalGateway.java      |   46 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   85 +-
 .../apache/ignite/internal/GridKernalState.java |    3 +
 .../ignite/internal/GridPluginComponent.java    |   11 +
 ...gniteClientDisconnectedCheckedException.java |   49 +
 .../apache/ignite/internal/IgniteKernal.java    |  222 +++-
 .../cluster/IgniteClusterAsyncImpl.java         |    5 +
 .../internal/cluster/IgniteClusterImpl.java     |   18 +
 .../internal/managers/GridManagerAdapter.java   |   19 +-
 .../deployment/GridDeploymentCommunication.java |    2 +-
 .../deployment/GridDeploymentManager.java       |   95 +-
 .../discovery/GridDiscoveryManager.java         |  163 ++-
 .../processors/GridProcessorAdapter.java        |   11 +
 .../affinity/GridAffinityAssignmentCache.java   |   26 +-
 .../cache/CacheOsConflictResolutionManager.java |    6 +
 .../cache/DynamicCacheChangeBatch.java          |   17 +
 .../processors/cache/GridCacheAdapter.java      |   25 +-
 .../cache/GridCacheAffinityManager.java         |   21 +-
 .../cache/GridCacheConcurrentMap.java           |   15 +-
 .../processors/cache/GridCacheGateway.java      |  116 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +
 .../processors/cache/GridCacheManager.java      |    6 +
 .../cache/GridCacheManagerAdapter.java          |    6 +
 .../processors/cache/GridCacheMvccManager.java  |   41 +-
 .../GridCachePartitionExchangeManager.java      |   81 +-
 .../processors/cache/GridCachePreloader.java    |    5 +
 .../cache/GridCachePreloaderAdapter.java        |    5 +
 .../processors/cache/GridCacheProcessor.java    |  311 ++++-
 .../cache/GridCacheSharedContext.java           |  113 +-
 .../cache/GridCacheSharedManager.java           |   11 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   20 +-
 .../processors/cache/GridCacheUtils.java        |   11 +
 .../processors/cache/IgniteCacheFutureImpl.java |    5 +
 .../processors/cache/IgniteCacheProxy.java      |    2 +-
 .../CacheDataStructuresManager.java             |   35 +
 .../distributed/GridCacheTxFinishSync.java      |   46 +
 .../distributed/dht/GridDhtCacheAdapter.java    |   14 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   24 +
 .../dht/GridPartitionedGetFuture.java           |   13 +-
 .../dht/preloader/GridDhtPreloader.java         |   16 +-
 .../distributed/near/GridNearCacheAdapter.java  |    8 +
 .../distributed/near/GridNearGetFuture.java     |   13 +-
 .../cache/dr/GridOsCacheDrManager.java          |    7 +-
 .../query/GridCacheDistributedQueryManager.java |   22 +
 .../cache/query/GridCacheQueryAdapter.java      |   11 +-
 .../query/GridCacheQueryFutureAdapter.java      |    2 +-
 .../continuous/CacheContinuousQueryHandler.java |    5 +
 .../transactions/IgniteTransactionsImpl.java    |   59 +-
 .../cache/transactions/IgniteTxManager.java     |   19 +-
 .../transactions/TransactionProxyImpl.java      |    2 +-
 .../cache/version/GridCacheVersionManager.java  |    9 +-
 .../clock/GridClockSyncProcessor.java           |    6 +-
 .../processors/cluster/ClusterProcessor.java    |   11 +
 .../continuous/GridContinuousHandler.java       |    9 +-
 .../continuous/GridContinuousProcessor.java     |  127 +-
 .../datastreamer/DataStreamProcessor.java       |   24 +-
 .../datastreamer/DataStreamerImpl.java          |   90 +-
 .../datastructures/DataStructuresProcessor.java |   33 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   33 +-
 .../GridCacheAtomicReferenceImpl.java           |   34 +-
 .../GridCacheAtomicSequenceImpl.java            |   33 +-
 .../GridCacheAtomicStampedImpl.java             |   33 +-
 .../GridCacheCountDownLatchImpl.java            |   51 +-
 .../datastructures/GridCacheRemovable.java      |    6 +-
 .../datastructures/GridCacheSetImpl.java        |   15 +-
 .../datastructures/GridCacheSetProxy.java       |   47 +-
 .../processors/job/GridJobProcessor.java        |    2 +-
 .../internal/processors/job/GridJobWorker.java  |    2 +-
 .../processors/query/GridQueryIndexing.java     |    7 +
 .../processors/query/GridQueryProcessor.java    |    6 +
 .../service/GridServiceProcessor.java           |   45 +-
 .../processors/service/GridServiceProxy.java    |   13 +-
 .../processors/task/GridTaskProcessor.java      |   55 +-
 .../processors/task/GridTaskWorker.java         |   59 +-
 .../ignite/internal/util/IgniteUtils.java       |   28 +
 .../shmem/IpcSharedMemoryClientEndpoint.java    |    5 +-
 .../ignite/internal/util/lang/GridFunc.java     |    2 +
 .../java/org/apache/ignite/spi/IgniteSpi.java   |   15 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  354 ++++--
 .../spi/discovery/DiscoverySpiDataExchange.java |    3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  408 ++++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  134 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    9 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   33 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   19 +
 .../messages/TcpDiscoveryAbstractMessage.java   |    3 +
 .../messages/TcpDiscoveryClientAckResponse.java |   64 +
 .../messages/TcpDiscoveryHandshakeResponse.java |   14 +
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   15 +-
 .../IgniteClientReconnectAbstractTest.java      |  363 ++++++
 .../IgniteClientReconnectApiExceptionTest.java  |  846 ++++++++++++
 .../IgniteClientReconnectAtomicsTest.java       |  672 ++++++++++
 .../IgniteClientReconnectCacheTest.java         | 1202 ++++++++++++++++++
 .../IgniteClientReconnectCollectionsTest.java   |  443 +++++++
 .../IgniteClientReconnectComputeTest.java       |  192 +++
 ...eClientReconnectContinuousProcessorTest.java |  372 ++++++
 ...IgniteClientReconnectDiscoveryStateTest.java |  123 ++
 ...niteClientReconnectFailoverAbstractTest.java |  231 ++++
 .../IgniteClientReconnectFailoverTest.java      |  212 +++
 .../IgniteClientReconnectServicesTest.java      |  260 ++++
 .../internal/IgniteClientReconnectStopTest.java |  106 ++
 .../IgniteClientReconnectStreamerTest.java      |  233 ++++
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 .../GridDeploymentManagerStopSelfTest.java      |    7 +
 .../IgniteCacheAbstractStopBusySelfTest.java    |    2 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |    6 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    1 +
 .../IgniteCacheSystemTransactionsSelfTest.java  |    2 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |    3 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  562 +++++++-
 .../multijvm/IgniteClusterProcessProxy.java     |    5 +
 .../IgniteClientReconnectTestSuite.java         |   48 +
 .../processors/query/h2/IgniteH2Indexing.java   |    5 +
 .../query/h2/twostep/GridMergeIndex.java        |   45 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   70 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |  225 ++++
 .../cache/IgniteClientReconnectQueriesTest.java |  427 +++++++
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |    4 +
 .../IgniteCacheWithIndexingTestSuite.java       |    1 +
 128 files changed, 9751 insertions(+), 815 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/IgniteClientDisconnectedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteClientDisconnectedException.java b/modules/core/src/main/java/org/apache/ignite/IgniteClientDisconnectedException.java
new file mode 100644
index 0000000..2089db0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteClientDisconnectedException.java
@@ -0,0 +1,61 @@
+/*
+ * 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;
+
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Exception thrown from Ignite API when client node disconnected from cluster.
+ */
+public class IgniteClientDisconnectedException extends IgniteException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private final IgniteFuture<?> reconnectFut;
+
+    /**
+     * @param reconnectFut Reconnect future.
+     * @param msg Error message.
+     */
+    public IgniteClientDisconnectedException(IgniteFuture<?> reconnectFut, String msg) {
+        this(reconnectFut, msg, null);
+    }
+
+    /**
+     * @param reconnectFut Reconnect future.
+     * @param msg Error message.
+     * @param cause Optional nested exception (can be {@code null}).
+     */
+    public IgniteClientDisconnectedException(
+        IgniteFuture<?> reconnectFut,
+        String msg,
+        @Nullable Throwable cause) {
+        super(msg, cause);
+
+        this.reconnectFut = reconnectFut;
+    }
+
+    /**
+     * @return Future that will be completed when client reconnected.
+     */
+    public IgniteFuture<?> reconnectFuture() {
+        return reconnectFut;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index 72be3fb..d3ce0e7 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -328,6 +328,11 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
      */
     public void resetMetrics();
 
+    /**
+     * @return Future that will be completed when client reconnected.
+     */
+    @Nullable public IgniteFuture<?> clientReconnectFuture();
+
     /** {@inheritDoc} */
     @Override public IgniteCluster withAsync();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
index fb227cd..65e0644 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridComponent.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
 
@@ -87,6 +88,7 @@ public interface GridComponent {
     /**
      * Receives discovery data object from remote nodes (called
      * on new node during discovery process).
+     *
      * @param joiningNodeId Joining node ID.
      * @param rmtNodeId Remote node ID for which data is provided.
      * @param data Discovery data object or {@code null} if nothing was
@@ -116,4 +118,20 @@ public interface GridComponent {
      * @return Unique component type for discovery data exchange.
      */
     @Nullable public DiscoveryDataExchangeType discoveryDataType();
+
+    /**
+     * Client disconnected callback.
+     *
+     * @param reconnectFut Reconnect future.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException;
+
+    /**
+     * Client reconnected callback.
+     *
+     * @param clusterRestarted Cluster restarted flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingImpl.java
index 62adf52..b4e0f01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSiblingImpl.java
@@ -167,7 +167,7 @@ public class GridJobSiblingImpl implements ComputeJobSibling, Externalizable {
                 }
                 catch (IgniteCheckedException e) {
                     // Avoid stack trace for left nodes.
-                    if (ctx.discovery().node(node.id()) != null && ctx.discovery().pingNode(node.id()))
+                    if (ctx.discovery().node(node.id()) != null && ctx.discovery().pingNodeNoError(node.id()))
                         U.error(ctx.log(GridJobSiblingImpl.class), "Failed to send cancel request to node " +
                             "[nodeId=" + node.id() + ", ses=" + ses + ']', e);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index d6542f3..f4da333 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -557,4 +557,9 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      * @return {@code True} if local node is client node (has flag {@link IgniteConfiguration#isClientMode()} set).
      */
     public boolean clientNode();
+
+    /**
+     * @return {@code True} if local node in disconnected state.
+     */
+    public boolean clientDisconnected();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 8abb135..fd8b50c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.managers.checkpoint.*;
 import org.apache.ignite.internal.managers.collision.*;
@@ -303,6 +304,12 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     /** Marshaller context. */
     private MarshallerContextImpl marshCtx;
 
+    /** */
+    private ClusterNode locNode;
+
+    /** */
+    private volatile boolean disconnected;
+
     /**
      * No-arg constructor is required by externalization.
      */
@@ -325,6 +332,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      * @param mgmtExecSvc Management executor service.
      * @param igfsExecSvc IGFS executor service.
      * @param restExecSvc REST executor service.
+     * @param plugins Plugin providers.
      * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("TypeMayBeWeakened")
@@ -503,7 +511,13 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** {@inheritDoc} */
     @Override public UUID localNodeId() {
-        return cfg.getNodeId();
+        if (locNode != null)
+            return locNode.id();
+
+        if (discoMgr != null)
+            locNode = discoMgr.localNode();
+
+        return locNode != null ? locNode.id() : config().getNodeId();
     }
 
     /** {@inheritDoc} */
@@ -903,6 +917,21 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public boolean clientDisconnected() {
+        if (locNode == null)
+            locNode = discoMgr != null ? discoMgr.localNode() : null;
+
+        return locNode != null ? (locNode.isClient() && disconnected) : false;
+    }
+
+    /**
+     * @param disconnected Disconnected flag.
+     */
+    void disconnected(boolean disconnected) {
+        this.disconnected = disconnected;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridKernalContextImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java
index 0156136..1d50aa2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java
@@ -17,7 +17,9 @@
 
 package org.apache.ignite.internal;
 
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.tostring.*;
+import org.jetbrains.annotations.*;
 
 /**
  * This interface guards access to implementations of public methods that access kernal
@@ -39,22 +41,6 @@ import org.apache.ignite.internal.util.tostring.*;
 @GridToStringExclude
 public interface GridKernalGateway {
     /**
-     * Performs light-weight check on the kernal state at the moment of this call.
-     * <p>
-     * This method should only be used when the kernal state should be checked just once
-     * at the beginning of the method and the fact that <b>kernal state can change in the middle
-     * of such method's execution</b> should not matter.
-     * <p>
-     * For example, when a method returns a constant value its implementation doesn't depend
-     * on the kernal being valid throughout its execution. In such case it is enough to check
-     * the kernal's state just once at the beginning of this method to provide consistent behavior
-     * of the API without incurring overhead of <code>lock-based</code> guard methods.
-     *
-     * @throws IllegalStateException Thrown in case when no kernal calls are allowed.
-     */
-    public void lightCheck() throws IllegalStateException;
-
-    /**
      * Should be called on entering every kernal related call
      * <b>originated directly or indirectly via public API</b>.
      * <p>
@@ -113,31 +99,29 @@ public interface GridKernalGateway {
     public void writeUnlock();
 
     /**
-     * Adds stop listener. Note that the identity set will be used to store listeners for
-     * performance reasons. Futures can register a listener to be notified when they need to
-     * be internally interrupted.
+     * Gets user stack trace through the first call of grid public API.
      *
-     * @param lsnr Listener to add.
+     * @return User stack trace.
      */
-    public void addStopListener(Runnable lsnr);
+    public String userStackTrace();
 
     /**
-     * Removes previously added stop listener.
-     *
-     * @param lsnr Listener to remove.
+     * @param timeout Timeout.
+     * @return {@code True} if write lock has been acquired.
+     * @throws InterruptedException If interrupted.
      */
-    public void removeStopListener(Runnable lsnr);
+    public boolean tryWriteLock(long timeout) throws InterruptedException;
 
     /**
-     * Gets user stack trace through the first call of grid public API.
+     * Disconnected callback.
+     *
+     * @return Reconnect future.
      */
-    public String userStackTrace();
+    @Nullable public GridFutureAdapter<?> onDisconnected();
 
     /**
-     * @param timeout Timeout.
-     * @return {@code True} if write lock has been acquired.
-     * @throws InterruptedException If interrupted.
+     * Reconnected callback.
      */
-    public boolean tryWriteLock(long timeout) throws InterruptedException;
+    public void onReconnected();
 }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java
index 35bbbed..f6a9e51 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java
@@ -17,13 +17,15 @@
 
 package org.apache.ignite.internal;
 
+import org.apache.ignite.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import java.io.*;
-import java.util.*;
 import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
 
 /**
  *
@@ -39,10 +41,10 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
 
     /** */
     @GridToStringExclude
-    private final Collection<Runnable> lsnrs = new GridSetWrapper<>(new IdentityHashMap<Runnable, Object>());
+    private IgniteFutureImpl<?> reconnectFut;
 
     /** */
-    private volatile GridKernalState state = GridKernalState.STOPPED;
+    private final AtomicReference<GridKernalState> state = new AtomicReference<>(GridKernalState.STOPPED);
 
     /** */
     @GridToStringExclude
@@ -63,12 +65,6 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
     }
 
     /** {@inheritDoc} */
-    @Override public void lightCheck() throws IllegalStateException {
-        if (state != GridKernalState.STARTED)
-            throw illegalState();
-    }
-
-    /** {@inheritDoc} */
     @SuppressWarnings({"LockAcquiredButNotSafelyReleased", "BusyWait"})
     @Override public void readLock() throws IllegalStateException {
         if (stackTrace == null)
@@ -76,10 +72,18 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
 
         rwLock.readLock();
 
+        GridKernalState state = this.state.get();
+
         if (state != GridKernalState.STARTED) {
             // Unlock just acquired lock.
             rwLock.readUnlock();
 
+            if (state == GridKernalState.DISCONNECTED) {
+                assert reconnectFut != null;
+
+                throw new IgniteClientDisconnectedException(reconnectFut, "Client node disconnected: " + gridName);
+            }
+
             throw illegalState();
         }
     }
@@ -90,6 +94,9 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
             stackTrace = stackTrace();
 
         rwLock.readLock();
+
+        if (state.get() == GridKernalState.DISCONNECTED)
+            throw new IgniteClientDisconnectedException(reconnectFut, "Client node disconnected: " + gridName);
     }
 
     /** {@inheritDoc} */
@@ -137,6 +144,27 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
         return false;
     }
 
+    /** {@inheritDoc} */
+    @Override public GridFutureAdapter<?> onDisconnected() {
+        GridFutureAdapter<?> fut = new GridFutureAdapter<>();
+
+        reconnectFut = new IgniteFutureImpl<>(fut);
+
+        if (!state.compareAndSet(GridKernalState.STARTED, GridKernalState.DISCONNECTED)) {
+            ((GridFutureAdapter<?>)reconnectFut.internalFuture()).onDone(new IgniteCheckedException("Node stopped."));
+
+            return null;
+        }
+
+        return fut;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected() {
+        if (state.compareAndSet(GridKernalState.DISCONNECTED, GridKernalState.STARTED))
+            ((GridFutureAdapter<?>)reconnectFut.internalFuture()).onDone();
+    }
+
     /**
      * Retrieves user stack trace.
      *
@@ -171,46 +199,15 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable {
         assert state != null;
 
         // NOTE: this method should always be called within write lock.
-        this.state = state;
+        this.state.set(state);
 
-        if (state == GridKernalState.STOPPING) {
-            Runnable[] runs;
-
-            synchronized (lsnrs) {
-                lsnrs.toArray(runs = new Runnable[lsnrs.size()]);
-            }
-
-            // In the same thread.
-            for (Runnable r : runs)
-                r.run();
-        }
+        if (reconnectFut != null)
+            ((GridFutureAdapter<?>)reconnectFut.internalFuture()).onDone(new IgniteCheckedException("Node stopped."));
     }
 
     /** {@inheritDoc} */
     @Override public GridKernalState getState() {
-        return state;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void addStopListener(Runnable lsnr) {
-        assert lsnr != null;
-
-        if (state == GridKernalState.STARTING || state == GridKernalState.STARTED)
-            synchronized (lsnrs) {
-                lsnrs.add(lsnr);
-            }
-        else
-            // Call right away in the same thread.
-            lsnr.run();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void removeStopListener(Runnable lsnr) {
-        assert lsnr != null;
-
-        synchronized (lsnrs) {
-            lsnrs.remove(lsnr);
-        }
+        return state.get();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalState.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalState.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalState.java
index fbb8f45..7d63578 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalState.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalState.java
@@ -32,6 +32,9 @@ public enum GridKernalState {
     /** Kernal is stopping. */
     STOPPING,
 
+    /** Kernal is disconnected. */
+    DISCONNECTED,
+
     /** Kernal is stopped.
      * <p>
      * This is also the initial state of the kernal.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
index b438bc1..55a84c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPluginComponent.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
@@ -64,6 +65,16 @@ public class GridPluginComponent implements GridComponent {
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
         plugin.onIgniteStop(cancel);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClientDisconnectedCheckedException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClientDisconnectedCheckedException.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClientDisconnectedCheckedException.java
new file mode 100644
index 0000000..e58530d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClientDisconnectedCheckedException.java
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+
+/**
+ *
+ */
+public class IgniteClientDisconnectedCheckedException extends IgniteCheckedException {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private IgniteFuture<?> reconnectFut;
+
+    /**
+     * @param reconnectFut Reconnect future.
+     * @param msg Message.
+     */
+    public IgniteClientDisconnectedCheckedException(IgniteFuture<?> reconnectFut, String msg) {
+        super(msg);
+
+        this.reconnectFut = reconnectFut;
+    }
+
+    /**
+     * @return Reconnect future.
+     */
+    public IgniteFuture<?> reconnectFuture() {
+        return reconnectFut;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 024dc7b..0d4ce32 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -60,6 +60,7 @@ import org.apache.ignite.internal.processors.session.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -902,82 +903,87 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
                 @Override public void run() {
                     if (log.isInfoEnabled()) {
-                        ClusterMetrics m = cluster().localNode().metrics();
+                        try {
+                            ClusterMetrics m = cluster().localNode().metrics();
 
-                        double cpuLoadPct = m.getCurrentCpuLoad() * 100;
-                        double avgCpuLoadPct = m.getAverageCpuLoad() * 100;
-                        double gcPct = m.getCurrentGcCpuLoad() * 100;
+                            double cpuLoadPct = m.getCurrentCpuLoad() * 100;
+                            double avgCpuLoadPct = m.getAverageCpuLoad() * 100;
+                            double gcPct = m.getCurrentGcCpuLoad() * 100;
 
-                        long heapUsed = m.getHeapMemoryUsed();
-                        long heapMax = m.getHeapMemoryMaximum();
+                            long heapUsed = m.getHeapMemoryUsed();
+                            long heapMax = m.getHeapMemoryMaximum();
 
-                        long heapUsedInMBytes = heapUsed / 1024 / 1024;
-                        long heapCommInMBytes = m.getHeapMemoryCommitted() / 1024 / 1024;
+                            long heapUsedInMBytes = heapUsed / 1024 / 1024;
+                            long heapCommInMBytes = m.getHeapMemoryCommitted() / 1024 / 1024;
 
-                        double freeHeapPct = heapMax > 0 ? ((double)((heapMax - heapUsed) * 100)) / heapMax : -1;
+                            double freeHeapPct = heapMax > 0 ? ((double)((heapMax - heapUsed) * 100)) / heapMax : -1;
 
-                        int hosts = 0;
-                        int nodes = 0;
-                        int cpus = 0;
+                            int hosts = 0;
+                            int nodes = 0;
+                            int cpus = 0;
 
-                        try {
-                            ClusterMetrics metrics = cluster().metrics();
+                            try {
+                                ClusterMetrics metrics = cluster().metrics();
 
-                            Collection<ClusterNode> nodes0 = cluster().nodes();
+                                Collection<ClusterNode> nodes0 = cluster().nodes();
 
-                            hosts = U.neighborhood(nodes0).size();
-                            nodes = metrics.getTotalNodes();
-                            cpus = metrics.getTotalCpus();
-                        }
-                        catch (IgniteException ignore) {
-                            // No-op.
-                        }
+                                hosts = U.neighborhood(nodes0).size();
+                                nodes = metrics.getTotalNodes();
+                                cpus = metrics.getTotalCpus();
+                            }
+                            catch (IgniteException ignore) {
+                                // No-op.
+                            }
 
-                        int pubPoolActiveThreads = 0;
-                        int pubPoolIdleThreads = 0;
-                        int pubPoolQSize = 0;
+                            int pubPoolActiveThreads = 0;
+                            int pubPoolIdleThreads = 0;
+                            int pubPoolQSize = 0;
 
-                        if (execSvc instanceof ThreadPoolExecutor) {
-                            ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+                            if (execSvc instanceof ThreadPoolExecutor) {
+                                ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
 
-                            int poolSize = exec.getPoolSize();
+                                int poolSize = exec.getPoolSize();
 
-                            pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                            pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
-                            pubPoolQSize = exec.getQueue().size();
-                        }
+                                pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+                                pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
+                                pubPoolQSize = exec.getQueue().size();
+                            }
 
-                        int sysPoolActiveThreads = 0;
-                        int sysPoolIdleThreads = 0;
-                        int sysPoolQSize = 0;
+                            int sysPoolActiveThreads = 0;
+                            int sysPoolIdleThreads = 0;
+                            int sysPoolQSize = 0;
 
-                        if (sysExecSvc instanceof ThreadPoolExecutor) {
-                            ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
+                            if (sysExecSvc instanceof ThreadPoolExecutor) {
+                                ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
 
-                            int poolSize = exec.getPoolSize();
+                                int poolSize = exec.getPoolSize();
 
-                            sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                            sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
-                            sysPoolQSize = exec.getQueue().size();
-                        }
+                                sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+                                sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
+                                sysPoolQSize = exec.getQueue().size();
+                            }
 
-                        String id = U.id8(localNode().id());
+                            String id = U.id8(localNode().id());
 
-                        String msg = NL +
-                            "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
-                            "    ^-- Node [id=" + id + ", name=" + name() + "]" + NL +
-                            "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
-                            "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
+                            String msg = NL +
+                                "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
+                                "    ^-- Node [id=" + id + ", name=" + name() + "]" + NL +
+                                "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
+                                "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
                                 dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
-                            "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
+                                "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
                                 dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
-                            "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
+                                "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
                                 pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
-                            "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
+                                "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
                                 sysPoolIdleThreads + ", qSize=" + sysPoolQSize + "]" + NL +
-                            "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
+                                "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
 
-                        log.info(msg);
+                            log.info(msg);
+                        }
+                        catch (IgniteClientDisconnectedException ignore) {
+                            // No-op.
+                        }
                     }
                 }
             }, metricsLogFreq, metricsLogFreq);
@@ -1676,7 +1682,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             GridKernalState state = gw.getState();
 
-            if (state == STARTED)
+            if (state == STARTED || state == DISCONNECTED)
                 firstStop = true;
             else if (state == STARTING)
                 U.warn(log, "Attempt to stop starting grid. This operation " +
@@ -1753,7 +1759,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 if (cache != null)
                     cache.blockGateways();
 
-                assert gw.getState() == STARTED || gw.getState() == STARTING;
+                assert gw.getState() == STARTED || gw.getState() == STARTING || gw.getState() == DISCONNECTED;
 
                 // No more kernal calls from this point on.
                 gw.setState(STOPPING);
@@ -2186,6 +2192,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             return false;
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -2801,6 +2810,109 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
     }
 
     /**
+     *
+     */
+    public void onDisconnected() {
+        Throwable err = null;
+
+        GridFutureAdapter<?> reconnectFut = ctx.gateway().onDisconnected();
+
+        if (reconnectFut == null) {
+            assert ctx.gateway().getState() != STARTED : ctx.gateway().getState();
+
+            return;
+        }
+
+        IgniteFuture<?> userFut = new IgniteFutureImpl<>(reconnectFut);
+
+        ctx.cluster().get().clientReconnectFuture(userFut);
+
+        ctx.disconnected(true);
+
+        List<GridComponent> comps = ctx.components();
+
+        for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {
+            GridComponent comp = it.previous();
+
+            try {
+                if (!skipDaemon(comp))
+                    comp.onDisconnected(userFut);
+            }
+            catch (IgniteCheckedException e) {
+                err = e;
+            }
+            catch (Throwable e) {
+                err = e;
+
+                if (e instanceof Error)
+                    throw e;
+            }
+        }
+
+        for (GridCacheContext cctx : ctx.cache().context().cacheContexts()) {
+            cctx.gate().writeLock();
+
+            cctx.gate().writeUnlock();
+        }
+
+        ctx.gateway().writeLock();
+
+        ctx.gateway().writeUnlock();
+
+        if (err != null) {
+            reconnectFut.onDone(err);
+
+            U.error(log, "Failed to reconnect, will stop node", err);
+
+            close();
+        }
+    }
+
+    /**
+     * @param clusterRestarted {@code True} if all cluster nodes restarted while client was disconnected.
+     */
+    public void onReconnected(final boolean clusterRestarted) {
+        Throwable err = null;
+
+        try {
+            ctx.disconnected(false);
+
+            for (GridComponent comp : ctx.components())
+                comp.onReconnected(clusterRestarted);
+
+            ctx.cache().context().exchange().reconnectExchangeFuture().listen(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
+                    try {
+                        fut.get();
+
+                        ctx.gateway().onReconnected();
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to reconnect, will stop node", e);
+
+                        close();
+                    }
+                }
+            });
+        }
+        catch (IgniteCheckedException e) {
+            err = e;
+        }
+        catch (Throwable e) {
+            err = e;
+
+            if (e instanceof Error)
+                throw e;
+        }
+
+        if (err != null) {
+            U.error(log, "Failed to reconnect, will stop node", err);
+
+            close();
+        }
+    }
+
+    /**
      * Creates optional component.
      *
      * @param cls Component interface.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
index 26c704c..51cf523 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterAsyncImpl.java
@@ -287,6 +287,11 @@ public class IgniteClusterAsyncImpl extends AsyncSupportAdapter<IgniteCluster>
     }
 
     /** {@inheritDoc} */
+    @Nullable @Override public IgniteFuture<?> clientReconnectFuture() {
+        return cluster.clientReconnectFuture();
+    }
+
+    /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         cluster = (IgniteClusterImpl)in.readObject();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
index 3c937b0..0287ca7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/cluster/IgniteClusterImpl.java
@@ -52,6 +52,9 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
     @GridToStringExclude
     private ConcurrentMap nodeLoc;
 
+    /** Client reconnect future. */
+    private IgniteFuture<?> reconnecFut;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -120,6 +123,9 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
         try {
             return ctx.discovery().pingNode(nodeId);
         }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
         finally {
             unguard();
         }
@@ -501,6 +507,18 @@ public class IgniteClusterImpl extends ClusterGroupAdapter implements IgniteClus
         nodeLoc.clear();
     }
 
+    /**
+     * @param reconnecFut Reconnect future.
+     */
+    public void clientReconnectFuture(IgniteFuture<?> reconnecFut) {
+        this.reconnecFut = reconnecFut;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteFuture<?> clientReconnectFuture() {
+        return reconnecFut;
+    }
+
     /** {@inheritDoc} */
     @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
         ctx = (GridKernalContext)in.readObject();

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
index 40a5ea5..298ff24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/GridManagerAdapter.java
@@ -166,6 +166,18 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        for (T t : spis)
+            t.onClientDisconnected(reconnectFut);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        for (T t : spis)
+            t.onClientReconnected(clusterRestarted);
+    }
+
     /**
      * Starts wrapped SPI.
      *
@@ -318,7 +330,12 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                     @Override public boolean pingNode(UUID nodeId) {
                         A.notNull(nodeId, "nodeId");
 
-                        return ctx.discovery().pingNode(nodeId);
+                        try {
+                            return ctx.discovery().pingNode(nodeId);
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw U.convertException(e);
+                        }
                     }
 
                     @Override public void send(ClusterNode node, Serializable msg, String topic)

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
index 443b221..3b886a5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentCommunication.java
@@ -293,7 +293,7 @@ class GridDeploymentCommunication {
                     log.debug("Sent peer class loading response [node=" + node.id() + ", res=" + res + ']');
             }
             catch (IgniteCheckedException e) {
-                if (ctx.discovery().pingNode(nodeId))
+                if (ctx.discovery().pingNodeNoError(nodeId))
                     U.error(log, "Failed to send peer class loading response to node: " + nodeId, e);
                 else if (log.isDebugEnabled())
                     log.debug("Failed to send peer class loading response to node " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
index 75fe98f..75fb41e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManager.java
@@ -94,13 +94,7 @@ public class GridDeploymentManager extends GridManagerAdapter<DeploymentSpi> {
 
         comm.start();
 
-        locStore = new GridDeploymentLocalStore(getSpi(), ctx, comm);
-        ldrStore = new GridDeploymentPerLoaderStore(getSpi(), ctx, comm);
-        verStore = new GridDeploymentPerVersionStore(getSpi(), ctx, comm);
-
-        locStore.start();
-        ldrStore.start();
-        verStore.start();
+        startStores();
 
         if (log.isDebugEnabled()) {
             log.debug("Local deployment: " + locDep);
@@ -110,17 +104,24 @@ public class GridDeploymentManager extends GridManagerAdapter<DeploymentSpi> {
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws IgniteCheckedException {
-        GridProtocolHandler.deregisterDeploymentManager();
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        storesOnKernalStop();
 
-        if (verStore != null)
-            verStore.stop();
+        storesStop();
 
-        if (ldrStore != null)
-            ldrStore.stop();
+        startStores();
+    }
 
-        if (locStore != null)
-            locStore.stop();
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        storesOnKernalStart();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
+        GridProtocolHandler.deregisterDeploymentManager();
+
+        storesStop();
 
         if (comm != null)
             comm.stop();
@@ -135,21 +136,12 @@ public class GridDeploymentManager extends GridManagerAdapter<DeploymentSpi> {
 
     /** {@inheritDoc} */
     @Override public void onKernalStart0() throws IgniteCheckedException {
-        locStore.onKernalStart();
-        ldrStore.onKernalStart();
-        verStore.onKernalStart();
+        storesOnKernalStart();
     }
 
     /** {@inheritDoc} */
     @Override public void onKernalStop0(boolean cancel) {
-        if (verStore != null)
-            verStore.onKernalStop();
-
-        if (ldrStore != null)
-            ldrStore.onKernalStop();
-
-        if (locStore != null)
-            locStore.onKernalStop();
+        storesOnKernalStop();
     }
 
     /** {@inheritDoc} */
@@ -547,6 +539,57 @@ public class GridDeploymentManager extends GridManagerAdapter<DeploymentSpi> {
         return ldr instanceof GridDeploymentClassLoader;
     }
 
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void startStores() throws IgniteCheckedException {
+        locStore = new GridDeploymentLocalStore(getSpi(), ctx, comm);
+        ldrStore = new GridDeploymentPerLoaderStore(getSpi(), ctx, comm);
+        verStore = new GridDeploymentPerVersionStore(getSpi(), ctx, comm);
+
+        locStore.start();
+        ldrStore.start();
+        verStore.start();
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    private void storesOnKernalStart() throws IgniteCheckedException {
+        locStore.onKernalStart();
+        ldrStore.onKernalStart();
+        verStore.onKernalStart();
+    }
+
+    /**
+     *
+     */
+    private void storesOnKernalStop() {
+        if (verStore != null)
+            verStore.onKernalStop();
+
+        if (ldrStore != null)
+            ldrStore.onKernalStop();
+
+        if (locStore != null)
+            locStore.onKernalStop();
+    }
+
+    /**
+     *
+     */
+    private void storesStop() {
+        if (verStore != null)
+            verStore.stop();
+
+        if (ldrStore != null)
+            ldrStore.stop();
+
+        if (locStore != null)
+            locStore.stop();
+    }
+
     /**
      *
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index b35628c..068d374 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -188,14 +188,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** Received custom messages history. */
     private final ArrayDeque<IgniteUuid> rcvdCustomMsgs = new ArrayDeque<>();
 
+    /** */
+    private final CountDownLatch startLatch = new CountDownLatch(1);
+
     /** @param ctx Context. */
     public GridDiscoveryManager(GridKernalContext ctx) {
         super(ctx, ctx.config().getDiscoverySpi());
     }
 
-    /** */
-    private final CountDownLatch startLatch = new CountDownLatch(1);
-
     /**
      * @return Memory usage of non-heap memory.
      */
@@ -337,7 +337,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         isLocDaemon = ctx.isDaemon();
 
-        hasRslvrs = !F.isEmpty(ctx.config().getSegmentationResolvers());
+        hasRslvrs = !ctx.config().isClientMode() && !F.isEmpty(ctx.config().getSegmentationResolvers());
 
         segChkFreq = ctx.config().getSegmentCheckFrequency();
 
@@ -380,14 +380,24 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         spi.setListener(new DiscoverySpiListener() {
+            private long gridStartTime;
+
             @Override public void onDiscovery(
-                int type,
-                long topVer,
-                ClusterNode node,
-                Collection<ClusterNode> topSnapshot,
-                Map<Long, Collection<ClusterNode>> snapshots,
+                final int type,
+                final long topVer,
+                final ClusterNode node,
+                final Collection<ClusterNode> topSnapshot,
+                final Map<Long, Collection<ClusterNode>> snapshots,
                 @Nullable DiscoverySpiCustomMessage spiCustomMsg
             ) {
+                if (type == EVT_NODE_JOINED && node.isLocal() && ctx.clientDisconnected()) {
+                    discoCacheHist.clear();
+
+                    topHist.clear();
+
+                    topSnap.set(new Snapshot(AffinityTopologyVersion.ZERO, null));
+                }
+
                 DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
                     : ((CustomMessageWrapper)spiCustomMsg).delegate();
 
@@ -415,7 +425,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
                 else {
-                    if (type != EVT_NODE_SEGMENTED) {
+                    if (type != EVT_NODE_SEGMENTED &&
+                        type != EVT_CLIENT_NODE_DISCONNECTED &&
+                        type != EVT_CLIENT_NODE_RECONNECTED) {
                         minorTopVer = 0;
 
                         verChanged = true;
@@ -424,7 +436,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
 
-                AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
+                final AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
 
                 if (type == EVT_NODE_FAILED || type == EVT_NODE_LEFT) {
                     for (DiscoCache c : discoCacheHist.values())
@@ -467,11 +479,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                 // If this is a local join event, just save it and do not notify listeners.
                 if (type == EVT_NODE_JOINED && node.id().equals(locNode.id())) {
+                    if (gridStartTime == 0)
+                        gridStartTime = getSpi().getGridStartTime();
+
                     updateTopologyVersionIfGreater(new AffinityTopologyVersion(locNode.order()),
                         new DiscoCache(localNode(), getSpi().getRemoteNodes()));
 
-                    assert startLatch.getCount() == 1;
-
                     startLatch.countDown();
 
                     DiscoveryEvent discoEvt = new DiscoveryEvent();
@@ -491,6 +504,46 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
                     return;
                 }
+                else if (type == EVT_CLIENT_NODE_DISCONNECTED) {
+                    /*
+                     * Notify all components from discovery thread to avoid concurrent
+                     * reconnect while disconnect handling is in progress.
+                     */
+
+                    assert locNode.isClient() : locNode;
+                    assert node.isClient() : node;
+
+                    ((IgniteKernal)ctx.grid()).onDisconnected();
+
+                    locJoinEvt = new GridFutureAdapter<>();
+
+                    registeredCaches.clear();
+                }
+                else if (type == EVT_CLIENT_NODE_RECONNECTED) {
+                    assert locNode.isClient() : locNode;
+                    assert node.isClient() : node;
+
+                    boolean clusterRestarted = gridStartTime != getSpi().getGridStartTime();
+
+                    gridStartTime = getSpi().getGridStartTime();
+
+                    ((IgniteKernal)ctx.grid()).onReconnected(clusterRestarted);
+
+                    ctx.cluster().clientReconnectFuture().listen(new CI1<IgniteFuture<?>>() {
+                        @Override public void apply(IgniteFuture<?> fut) {
+                            try {
+                                fut.get();
+
+                                discoWrk.addEvent(type, nextTopVer, node, topSnapshot, null);
+                            }
+                            catch (IgniteException ignore) {
+                                // No-op.
+                            }
+                        }
+                    });
+
+                    return;
+                }
 
                 discoWrk.addEvent(type, nextTopVer, node, topSnapshot, customMsg);
             }
@@ -967,7 +1020,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
         Collection<ClusterNode> rmtNodes = discoCache.remoteNodes();
 
-        Collection<ClusterNode> serverNodes = F.view(discoCache.allNodes(), F.not(clientFilter));
+        Collection<ClusterNode> srvNodes = F.view(discoCache.allNodes(), F.not(clientFilter));
 
         Collection<ClusterNode> clientNodes = F.view(discoCache.allNodes(), clientFilter);
 
@@ -987,7 +1040,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         double heap = U.heapSize(allNodes, 2);
 
         if (log.isQuiet())
-            U.quiet(false, topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
+            U.quiet(false, topologySnapshotMessage(srvNodes.size(), clientNodes.size(), totalCpus, heap));
 
         if (log.isDebugEnabled()) {
             String dbg = "";
@@ -997,7 +1050,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 ">>> " + PREFIX + "." + U.nl() +
                 ">>> +----------------+" + U.nl() +
                 ">>> Grid name: " + (ctx.gridName() == null ? "default" : ctx.gridName()) + U.nl() +
-                ">>> Number of server nodes: " + serverNodes.size() + U.nl() +
+                ">>> Number of server nodes: " + srvNodes.size() + U.nl() +
                 ">>> Number of client nodes: " + clientNodes.size() + U.nl() +
                 (discoOrdered ? ">>> Topology version: " + topVer + U.nl() : "") +
                 ">>> Topology hash: 0x" + Long.toHexString(hash).toUpperCase() + U.nl();
@@ -1031,20 +1084,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             log.debug(dbg);
         }
         else if (log.isInfoEnabled())
-            log.info(topologySnapshotMessage(serverNodes.size(), clientNodes.size(), totalCpus, heap));
+            log.info(topologySnapshotMessage(srvNodes.size(), clientNodes.size(), totalCpus, heap));
     }
 
     /**
-     * @param serverNodesNum Server nodes number.
+     * @param srvNodesNum Server nodes number.
      * @param clientNodesNum Client nodes number.
      * @param totalCpus Total cpu number.
      * @param heap Heap size.
      * @return Topology snapshot message.
      */
-    private String topologySnapshotMessage(int serverNodesNum, int clientNodesNum, int totalCpus, double heap) {
+    private String topologySnapshotMessage(int srvNodesNum, int clientNodesNum, int totalCpus, double heap) {
         return PREFIX + " [" +
             (discoOrdered ? "ver=" + topSnap.get().topVer.topologyVersion() + ", " : "") +
-            "server nodes=" + serverNodesNum +
+            "server nodes=" + srvNodesNum +
             ", client nodes=" + clientNodesNum +
             ", CPUs=" + totalCpus +
             ", heap=" + heap + "GB" +
@@ -1134,8 +1187,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * @param nodeId ID of the node.
      * @return {@code True} if ping succeeded.
+     * @throws IgniteClientDisconnectedCheckedException If ping failed.
      */
-    public boolean pingNode(UUID nodeId) {
+    public boolean pingNode(UUID nodeId) throws IgniteClientDisconnectedCheckedException {
         assert nodeId != null;
 
         if (!busyLock.enterBusy())
@@ -1144,6 +1198,36 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         try {
             return getSpi().pingNode(nodeId);
         }
+        catch (IgniteException e) {
+            if (e.hasCause(IgniteClientDisconnectedCheckedException.class)) {
+                IgniteFuture<?> reconnectFut = ctx.cluster().clientReconnectFuture();
+
+                throw new IgniteClientDisconnectedCheckedException(reconnectFut, e.getMessage());
+            }
+
+            throw e;
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
+     * @param nodeId ID of the node.
+     * @return {@code True} if ping succeeded.
+     */
+    public boolean pingNodeNoError(UUID nodeId) {
+        assert nodeId != null;
+
+        if (!busyLock.enterBusy())
+            return false;
+
+        try {
+            return getSpi().pingNode(nodeId);
+        }
+        catch (IgniteException e) {
+            return false;
+        }
         finally {
             busyLock.leaveBusy();
         }
@@ -1519,9 +1603,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param msg Custom message.
+     * @throws IgniteCheckedException If failed.
      */
-    public void sendCustomEvent(DiscoveryCustomMessage msg) {
-        getSpi().sendCustomEvent(new CustomMessageWrapper(msg));
+    public void sendCustomEvent(DiscoveryCustomMessage msg) throws IgniteCheckedException {
+        try {
+            getSpi().sendCustomEvent(new CustomMessageWrapper(msg));
+        }
+        catch (IgniteClientDisconnectedException e) {
+            IgniteFuture<?> reconnectFut = ctx.cluster().clientReconnectFuture();
+
+            throw new IgniteClientDisconnectedCheckedException(reconnectFut, e.getMessage());
+        }
+        catch (IgniteException e) {
+            throw new IgniteCheckedException(e);
+        }
     }
 
     /**
@@ -1743,6 +1838,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 else if (type == EVT_NODE_SEGMENTED)
                     evt.message("Node segmented: " + node);
 
+                else if (type == EVT_CLIENT_NODE_DISCONNECTED)
+                    evt.message("Client node disconnected: " + node);
+
+                else if (type == EVT_CLIENT_NODE_RECONNECTED)
+                    evt.message("Client node reconnected: " + node);
+
                 else
                     assert false;
 
@@ -1755,6 +1856,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
          * @param topVer Topology version.
          * @param node Node.
          * @param topSnapshot Topology snapshot.
+         * @param data Custom message.
          */
         void addEvent(
             int type,
@@ -1864,6 +1966,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     break;
                 }
 
+                case EVT_CLIENT_NODE_DISCONNECTED: {
+                    // No-op.
+
+                    break;
+                }
+
+                case EVT_CLIENT_NODE_RECONNECTED: {
+                    if (log.isInfoEnabled())
+                        log.info("Client node reconnected to topology: " + node);
+
+                    ackTopology(topVer.topologyVersion(), true);
+
+                    break;
+                }
+
                 case EVT_NODE_FAILED: {
                     // Check only if resolvers were configured.
                     if (hasRslvrs)

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
index a84c48a..8baf95c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/GridProcessorAdapter.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
 
@@ -62,6 +63,16 @@ public abstract class GridProcessorAdapter implements GridProcessor {
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Nullable @Override public DiscoveryDataExchangeType discoveryDataType() {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 6989385..d40128c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -69,7 +69,7 @@ public class GridAffinityAssignmentCache {
     private IgniteLogger log;
 
     /** Node stop flag. */
-    private volatile boolean stopping;
+    private volatile IgniteCheckedException stopErr;
 
     /**
      * Constructs affinity cached calculations.
@@ -130,18 +130,28 @@ public class GridAffinityAssignmentCache {
 
     /**
      * Kernal stop callback.
+     *
+     * @param err Error.
      */
-    public void onKernalStop() {
-        stopping = true;
-
-        IgniteCheckedException err =
-            new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping.");
+    public void onKernalStop(IgniteCheckedException err) {
+        stopErr = err;
 
         for (AffinityReadyFuture fut : readyFuts.values())
             fut.onDone(err);
     }
 
     /**
+     *
+     */
+    public void onReconnected() {
+        affCache.clear();
+
+        head.set(new GridAffinityAssignment(AffinityTopologyVersion.NONE));
+
+        stopErr = null;
+    }
+
+    /**
      * Calculates affinity cache for given topology version.
      *
      * @param topVer Topology version to calculate affinity cache for.
@@ -312,8 +322,8 @@ public class GridAffinityAssignmentCache {
 
             fut.onDone(topVer);
         }
-        else if (stopping)
-            fut.onDone(new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping."));
+        else if (stopErr != null)
+            fut.onDone(stopErr);
 
         return fut;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOsConflictResolutionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOsConflictResolutionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOsConflictResolutionManager.java
index 29e50b6..9e765d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOsConflictResolutionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheOsConflictResolutionManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.cache.version.*;
+import org.apache.ignite.lang.*;
 
 /**
  * OS conflict resolver manager.
@@ -55,4 +56,9 @@ public class CacheOsConflictResolutionManager<K ,V> implements CacheConflictReso
     @Override public void printMemoryStats() {
         // No-op.
     }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
index dfc39c1..1e8184d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java
@@ -43,6 +43,9 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
     /** Custom message ID. */
     private IgniteUuid id = IgniteUuid.randomUuid();
 
+    /** */
+    private boolean clientReconnect;
+
     /**
      * @param reqs Requests.
      */
@@ -93,6 +96,20 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage {
         return false;
     }
 
+    /**
+     * @param clientReconnect {@code True} if this is discovery data sent on client reconnect.
+     */
+    public void clientReconnect(boolean clientReconnect) {
+        this.clientReconnect = clientReconnect;
+    }
+
+    /**
+     * @return {@code True} if this is discovery data sent on client reconnect.
+     */
+    public boolean clientReconnect() {
+        return clientReconnect;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(DynamicCacheChangeBatch.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index e138520..d2a730a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -212,7 +212,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     @SuppressWarnings("OverriddenMethodCallDuringObjectConstruction")
     protected GridCacheAdapter(GridCacheContext<K, V> ctx, int startSize) {
-        this(ctx, new GridCacheConcurrentMap(ctx, startSize, 0.75F));
+        this(ctx, new GridCacheConcurrentMap(ctx, startSize, 0.75F, null));
     }
 
     /**
@@ -2868,7 +2868,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(val);
 
-                return (GridCacheReturn)tx.removeAllAsync(ctx,
+                return tx.removeAllAsync(ctx,
                     Collections.singletonList(key),
                     null,
                     true,
@@ -2934,7 +2934,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 if (ctx.deploymentEnabled())
                     ctx.deploy().registerClass(oldVal);
 
-                return (GridCacheReturn) tx.putAllAsync(ctx,
+                return tx.putAllAsync(ctx,
                         F.t(key, newVal),
                         true,
                         null,
@@ -3036,7 +3036,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                     ctx.deploy().registerClass(val);
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, false,
-                        ctx.equalsValArray(val)).get().success();
+                    ctx.equalsValArray(val)).get().success();
             }
 
             @Override public String toString() {
@@ -3250,10 +3250,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         TransactionConfiguration cfg = ctx.gridConfig().getTransactionConfiguration();
 
         return txStart(
-                concurrency,
-                isolation,
-                cfg.getDefaultTxTimeout(),
-                0
+            concurrency,
+            isolation,
+            cfg.getDefaultTxTimeout(),
+            0
         );
     }
 
@@ -3689,7 +3689,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         return F.iterator(iterator(),
             new IgniteClosure<Cache.Entry<K, V>, Cache.Entry<K, V>>() {
                 private IgniteCacheExpiryPolicy expiryPlc =
-                        ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
+                    ctx.cache().expiryPolicy(opCtx != null ? opCtx.expiry() : null);
 
                 @Override public Cache.Entry<K, V> apply(Cache.Entry<K, V> lazyEntry) {
                     CacheOperationContext prev = ctx.gate().enter(opCtx);
@@ -4443,6 +4443,13 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     public abstract void onDeferredDelete(GridCacheEntryEx entry, GridCacheVersion ver);
 
     /**
+     *
+     */
+    public void onReconnected() {
+        // No-op.
+    }
+
+    /**
      * Validates that given cache value implements {@link Externalizable}.
      *
      * @param val Cache value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index ea17df1..88e5e40 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -54,7 +55,25 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter {
 
     /** {@inheritDoc} */
     @Override protected void onKernalStop0(boolean cancel) {
-        aff.onKernalStop();
+        IgniteCheckedException err =
+            new IgniteCheckedException("Failed to wait for topology update, cache (or node) is stopping.");
+
+        aff.onKernalStop(err);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture reconnectFut) {
+        IgniteCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
+            "Failed to wait for topology update, client disconnected.");
+
+        aff.onKernalStop(err);
+    }
+
+    /**
+     *
+     */
+    public void onReconnected() {
+        aff.onReconnected();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index db5eed1..966dcc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -279,11 +279,17 @@ public class GridCacheConcurrentMap {
      * @param loadFactor  the load factor threshold, used to control resizing.
      *      Resizing may be performed when the average number of elements per
      *      bin exceeds this threshold.
+     * @param factory Entries factory.
      * @throws IllegalArgumentException if the initial capacity of
      *      elements is negative or the load factor is non-positive.
      */
-    public GridCacheConcurrentMap(GridCacheContext ctx, int initCap, float loadFactor) {
+    public GridCacheConcurrentMap(GridCacheContext ctx,
+        int initCap,
+        float loadFactor,
+        @Nullable GridCacheMapEntryFactory factory) {
         this(ctx, initCap, loadFactor, DFLT_CONCUR_LEVEL);
+
+        this.factory = factory;
     }
 
     /**
@@ -312,6 +318,13 @@ public class GridCacheConcurrentMap {
     }
 
     /**
+     * @return Entries factory.
+     */
+    public GridCacheMapEntryFactory getEntryFactory() {
+        return factory;
+    }
+
+    /**
      * @return Non-internal predicate.
      */
     private static CacheEntryPredicate[] nonInternal() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
index f2beb0a..a3c8da6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java
@@ -22,8 +22,12 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.*;
+import java.util.concurrent.atomic.*;
+
 /**
  * Cache gateway.
  */
@@ -33,7 +37,10 @@ public class GridCacheGateway<K, V> {
     private final GridCacheContext<K, V> ctx;
 
     /** Stopped flag for dynamic caches. */
-    private volatile boolean stopped;
+    private final AtomicReference<State> state = new AtomicReference<>(State.STARTED);
+
+    /** */
+    private IgniteFuture<?> reconnectFut;
 
     /** */
     private GridSpinReadWriteLock rwLock = new GridSpinReadWriteLock();
@@ -56,11 +63,36 @@ public class GridCacheGateway<K, V> {
 
         rwLock.readLock();
 
-        if (stopped) {
-            rwLock.readUnlock();
+        checkState(true, true);
+    }
 
-            throw new IllegalStateException("Dynamic cache has been stopped: " + ctx.name());
+    /**
+     * @param lock {@code True} if lock is held.
+     * @param stopErr {@code True} if throw exception if stopped.
+     * @return {@code True} if cache is in started state.
+     */
+    private boolean checkState(boolean lock, boolean stopErr) {
+        State state = this.state.get();
+
+        if (state != State.STARTED) {
+            if (lock)
+                rwLock.readUnlock();
+
+            if (state == State.STOPPED) {
+                if (stopErr)
+                    throw new IllegalStateException("Cache has been stopped: " + ctx.name());
+                else
+                    return false;
+            }
+            else {
+                assert reconnectFut != null;
+
+                throw new CacheException(
+                    new IgniteClientDisconnectedException(reconnectFut, "Client node disconnected: " + ctx.gridName()));
+            }
         }
+
+        return true;
     }
 
     /**
@@ -71,17 +103,11 @@ public class GridCacheGateway<K, V> {
     public boolean enterIfNotStopped() {
         onEnter();
 
-        // Must unlock in case of unexpected errors to avoid
-        // deadlocks during kernal stop.
+        // Must unlock in case of unexpected errors to avoid deadlocks during kernal stop.
         rwLock.readLock();
 
-        if (stopped) {
-            rwLock.readUnlock();
-
-            return false;
-        }
+        return checkState(true, false);
 
-        return true;
     }
 
     /**
@@ -92,7 +118,7 @@ public class GridCacheGateway<K, V> {
     public boolean enterIfNotStoppedNoLock() {
         onEnter();
 
-        return !stopped;
+        return checkState(false, false);
     }
 
     /**
@@ -144,11 +170,7 @@ public class GridCacheGateway<K, V> {
 
         rwLock.readLock();
 
-        if (stopped) {
-            rwLock.readUnlock();
-
-            throw new IllegalStateException("Cache has been stopped: " + ctx.name());
-        }
+        checkState(true, true);
 
         // Must unlock in case of unexpected errors to avoid
         // deadlocks during kernal stop.
@@ -169,8 +191,7 @@ public class GridCacheGateway<K, V> {
     @Nullable public CacheOperationContext enterNoLock(@Nullable CacheOperationContext opCtx) {
         onEnter();
 
-        if (stopped)
-            throw new IllegalStateException("Cache has been stopped: " + ctx.name());
+        checkState(false, false);
 
         return setOperationContextPerCall(opCtx);
     }
@@ -229,8 +250,42 @@ public class GridCacheGateway<K, V> {
     /**
      *
      */
-    public void block() {
-        stopped = true;
+    public void stopped() {
+        state.set(State.STOPPED);
+    }
+
+    /**
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        assert reconnectFut != null;
+
+        this.reconnectFut = reconnectFut;
+
+        state.compareAndSet(State.STARTED, State.DISCONNECTED);
+    }
+
+    /**
+     *
+     */
+    public void writeLock(){
+        rwLock.writeLock();
+    }
+
+    /**
+     *
+     */
+    public void writeUnlock() {
+        rwLock.writeUnlock();
+    }
+
+    /**
+     * @param stopped Cache stopped flag.
+     */
+    public void reconnected(boolean stopped) {
+        State newState = stopped ? State.STOPPED : State.STARTED;
+
+        state.compareAndSet(State.DISCONNECTED, newState);
     }
 
     /**
@@ -256,11 +311,24 @@ public class GridCacheGateway<K, V> {
             Thread.currentThread().interrupt();
 
         try {
-            // No-op.
-            stopped = true;
+            state.set(State.STOPPED);
         }
         finally {
             rwLock.writeUnlock();
         }
     }
+
+    /**
+     *
+     */
+    private enum State {
+        /** */
+        STARTED,
+
+        /** */
+        DISCONNECTED,
+
+        /** */
+        STOPPED
+    }
 }


[44/50] [abbrv] ignite git commit: # master minor

Posted by nt...@apache.org.
# master minor


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

Branch: refs/heads/ignite-788-dev
Commit: 7494a86b9ce166e547e7e5cad14f7202ab597b97
Parents: d8f86f1
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 12:31:01 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 12:36:02 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/GridDhtTxPrepareFuture.java       | 9 ++++-----
 .../near/GridCachePartitionedMultiNodeFullApiSelfTest.java  | 4 +++-
 2 files changed, 7 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7494a86b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index af0fbdf..6efa4d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -256,7 +256,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                 MiniFuture f = (MiniFuture)fut;
 
                 if (f.node().id().equals(nodeId)) {
-                    f.onResult(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
+                    f.onNodeLeft(new ClusterTopologyCheckedException("Remote node left grid: " + nodeId));
 
                     return true;
                 }
@@ -873,12 +873,11 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
 
                     assert req.transactionNodes() != null;
 
-                    //noinspection TryWithIdenticalCatches
                     try {
                         cctx.io().send(n, req, tx.ioPolicy());
                     }
                     catch (ClusterTopologyCheckedException e) {
-                        fut.onResult(e);
+                        fut.onNodeLeft(e);
                     }
                     catch (IgniteCheckedException e) {
                         fut.onResult(e);
@@ -933,7 +932,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                             cctx.io().send(nearMapping.node(), req, tx.system() ? UTILITY_CACHE_POOL : SYSTEM_POOL);
                         }
                         catch (ClusterTopologyCheckedException e) {
-                            fut.onResult(e);
+                            fut.onNodeLeft(e);
                         }
                         catch (IgniteCheckedException e) {
                             fut.onResult(e);
@@ -1148,7 +1147,7 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         /**
          * @param e Node failure.
          */
-        void onResult(ClusterTopologyCheckedException e) {
+        void onNodeLeft(ClusterTopologyCheckedException e) {
             if (log.isDebugEnabled())
                 log.debug("Remote node left grid while sending or waiting for reply (will ignore): " + this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/7494a86b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index a5e7a31..7572aaf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -372,7 +372,9 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
         boolean nearEnabled = cache2.getConfiguration(CacheConfiguration.class).getNearConfiguration() != null;
 
         assertEquals(nearEnabled ? 2 : 0, cache2.localSize(NEAR));
-        assertEquals(0, cache2.localSize(CachePeekMode.ALL) - cache2.localSize(NEAR));
+
+        if (cacheMode() != REPLICATED)
+            assertEquals(0, cache2.localSize(CachePeekMode.ALL) - cache2.localSize(NEAR));
     }
 
     /**


[29/50] [abbrv] ignite git commit: ignite-1067

Posted by nt...@apache.org.
ignite-1067


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

Branch: refs/heads/ignite-788-dev
Commit: 954ae1543116ab538a5a04ee64172fa780debe80
Parents: dd246e5
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Jul 16 15:26:01 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Jul 16 15:26:01 2015 +0300

----------------------------------------------------------------------
 assembly/LICENSE_FABRIC                            | 17 +++--------------
 assembly/LICENSE_HADOOP                            | 11 -----------
 .../src/main/resources/META-INF/licenses.txt.vm    |  4 +---
 modules/core/licenses/jsr166-license.txt           |  3 +++
 modules/core/licenses/snaptree-bsd-license.txt     |  2 +-
 modules/jta/licenses/jta-license.txt               |  2 ++
 6 files changed, 10 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/assembly/LICENSE_FABRIC
----------------------------------------------------------------------
diff --git a/assembly/LICENSE_FABRIC b/assembly/LICENSE_FABRIC
index 794d721..152cc1e 100644
--- a/assembly/LICENSE_FABRIC
+++ b/assembly/LICENSE_FABRIC
@@ -227,17 +227,6 @@ For details, see CC0 1.0 Universal (1.0), Public Domain Dedication,
 http://creativecommons.org/publicdomain/zero/1.0/
 
 ==============================================================================
-For books used for tests in "org.apache.ignite.internal.processors.hadoop.books"
-==============================================================================
-This code bundles book text files used for testing purposes which contain
-the following header:
-
-This eBook is for the use of anyone anywhere at no cost and with
-almost no restrictions whatsoever.  You may copy it, give it away or
-re-use it under the terms of the Project Gutenberg License included
-with this eBook or online at www.gutenberg.org
-
-==============================================================================
 For JSR107 API and SPI (https://github.com/jsr107/jsr107spec) javax.cache:cache-api:jar:1.0.0
 ==============================================================================
 This product bundles JSR107 API and SPI which is available under a:
@@ -256,7 +245,7 @@ This product bundles JLine which is available under a:
 The BSD License. For details, see http://www.opensource.org/licenses/bsd-license.php.
 
 ==============================================================================
-For Scala Library (http://www.scala-lang.org/) org.scala-lang:scala-library:jar:2.11.2
+For Scala Library (http://www.scala-lang.org/) org.scala-lang:scala-library:jar:2.*
 ==============================================================================
 This product bundles Scala Library which is available under a:
 BSD 3-Clause. For details, see http://www.scala-lang.org/license.html.
@@ -294,8 +283,8 @@ Eclipse Public License - v 1.0. For details, see http://www.eclipse.org/legal/ep
 ==============================================================================
 For Java Transaction API (http://java.sun.com/products/jta) javax.transaction:jta:jar:1.1
 ==============================================================================
-This product bundles Java Transaction API which is available under a:
-No licenses.
+This product bundles Java Transaction API which is available under it's own license.
+See http://download.oracle.com/otndocs/jcp/jta-1.1-classes-oth-JSpec/jta-1.1-classes-oth-JSpec-license.html for details.
 
 ==============================================================================
 For ASM All (http://asm.objectweb.org/asm-all/) org.ow2.asm:asm-all:jar:4.2

http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/assembly/LICENSE_HADOOP
----------------------------------------------------------------------
diff --git a/assembly/LICENSE_HADOOP b/assembly/LICENSE_HADOOP
index f6ed34c..cfbaea3 100644
--- a/assembly/LICENSE_HADOOP
+++ b/assembly/LICENSE_HADOOP
@@ -227,17 +227,6 @@ For details, see CC0 1.0 Universal (1.0), Public Domain Dedication,
 http://creativecommons.org/publicdomain/zero/1.0/
 
 ==============================================================================
-For books used for tests in "org.apache.ignite.internal.processors.hadoop.books"
-==============================================================================
-This code bundles book text files used for testing purposes which contain
-the following header:
-
-This eBook is for the use of anyone anywhere at no cost and with
-almost no restrictions whatsoever.  You may copy it, give it away or
-re-use it under the terms of the Project Gutenberg License included
-with this eBook or online at www.gutenberg.org
-
-==============================================================================
 For JSR107 API and SPI (https://github.com/jsr107/jsr107spec) javax.cache:cache-api:jar:1.0.0
 ==============================================================================
 This product bundles JSR107 API and SPI which is available under a:

http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/modules/apache-license-gen/src/main/resources/META-INF/licenses.txt.vm
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/src/main/resources/META-INF/licenses.txt.vm b/modules/apache-license-gen/src/main/resources/META-INF/licenses.txt.vm
index f02d9bc..ca1988b 100644
--- a/modules/apache-license-gen/src/main/resources/META-INF/licenses.txt.vm
+++ b/modules/apache-license-gen/src/main/resources/META-INF/licenses.txt.vm
@@ -25,7 +25,7 @@
 
 #foreach ( $organizationName in $projectsSortedByOrganization.keySet() )
 #foreach ( $project in $projectsSortedByOrganization.get( $organizationName ) )
-#if($project.licenses.size() == 1 && $project.licenses.get(0).url.contains("www.apache.org/licenses/LICENSE-2.0") && !$project.licenses.get(0).url.contains("and"))#else
+#if((!($project.licenses.size() == 1 && $project.licenses.get(0).url.contains("www.apache.org/licenses/LICENSE-2.0") && !$project.licenses.get(0).url.contains("and"))) && $project.licenses.size() > 0)
 ==============================================================================
 For $project.name #if ($project.url)($project.url)#end $project.artifact
 ==============================================================================
@@ -34,8 +34,6 @@ This product bundles $project.name which is available under a:
 $license.name.replaceAll("[ ]{2,}"," ").replaceAll("\n",""). #if ($license.url)For details, see $license.url.replaceAll("[ ]{2,}"," ").replaceAll("\n","").#end
 
 #end
-#if ($project.licenses.size() == 0)No licenses.
-#end
 
 #end
 #end

http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/modules/core/licenses/jsr166-license.txt
----------------------------------------------------------------------
diff --git a/modules/core/licenses/jsr166-license.txt b/modules/core/licenses/jsr166-license.txt
new file mode 100644
index 0000000..24cf176
--- /dev/null
+++ b/modules/core/licenses/jsr166-license.txt
@@ -0,0 +1,3 @@
+This product sources bundles JSR-166 classes which are donated to public domain.
+For details, see CC0 1.0 Universal (1.0), Public Domain Dedication,
+http://creativecommons.org/publicdomain/zero/1.0/
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/modules/core/licenses/snaptree-bsd-license.txt
----------------------------------------------------------------------
diff --git a/modules/core/licenses/snaptree-bsd-license.txt b/modules/core/licenses/snaptree-bsd-license.txt
index 5e7d327..a4bd0d9 100644
--- a/modules/core/licenses/snaptree-bsd-license.txt
+++ b/modules/core/licenses/snaptree-bsd-license.txt
@@ -1,4 +1,4 @@
-SNAPTREE LICENSE
+This product sources bundles SnapTree classes, which is available under a SNAPTREE LICENSE:
 
 Copyright (c) 2009 Stanford University, unless otherwise specified.
 All rights reserved.

http://git-wip-us.apache.org/repos/asf/ignite/blob/954ae154/modules/jta/licenses/jta-license.txt
----------------------------------------------------------------------
diff --git a/modules/jta/licenses/jta-license.txt b/modules/jta/licenses/jta-license.txt
new file mode 100644
index 0000000..77d0633
--- /dev/null
+++ b/modules/jta/licenses/jta-license.txt
@@ -0,0 +1,2 @@
+This product bundles Java Transaction API, which is available under it's own license.
+See http://download.oracle.com/otndocs/jcp/jta-1.1-classes-oth-JSpec/jta-1.1-classes-oth-JSpec-license.html for details.
\ No newline at end of file


[11/50] [abbrv] 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/cd3a9b61
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/cd3a9b61
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/cd3a9b61

Branch: refs/heads/ignite-788-dev
Commit: cd3a9b61bbad20dc0cd2ba239de6399aa2754dc3
Parents: d38a70d bdd1762
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 15 20:04:53 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 15 20:04:53 2015 -0700

----------------------------------------------------------------------
 .../internal/processors/query/GridQueryProcessor.java   |  4 +++-
 .../GridCacheAbstractNodeRestartSelfTest.java           | 12 ++++++------
 .../IgniteCacheAtomicReplicatedNodeRestartSelfTest.java |  5 +++++
 3 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------



[04/50] [abbrv] ignite git commit: ignite-1085: reimplemented

Posted by nt...@apache.org.
ignite-1085: reimplemented


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

Branch: refs/heads/ignite-788-dev
Commit: f0ace1f997b3e7b7a0692870aba4fd80302f9813
Parents: 24aae82
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 14:03:50 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 14:03:50 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         | 20 ++++++++------------
 1 file changed, 8 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f0ace1f9/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 1bac4d0..b35628c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -193,6 +193,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         super(ctx, ctx.config().getDiscoverySpi());
     }
 
+    /** */
+    private final CountDownLatch startLatch = new CountDownLatch(1);
+
     /**
      * @return Memory usage of non-heap memory.
      */
@@ -376,8 +379,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
             });
         }
 
-        final CountDownLatch startLatch = new CountDownLatch(1);
-
         spi.setListener(new DiscoverySpiListener() {
             @Override public void onDiscovery(
                 int type,
@@ -1052,6 +1053,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /** {@inheritDoc} */
     @Override public void onKernalStop0(boolean cancel) {
+        startLatch.countDown();
+
         // Stop segment check worker.
         if (segChkWrk != null) {
             segChkWrk.cancel();
@@ -1224,16 +1227,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @return Discovery collection cache.
      */
     public DiscoCache discoCache() {
-        Snapshot cur;
-
-        while ((cur = topSnap.get()) == null) {
-            // Wrap the SPI collection to avoid possible floating collection.
-            if (topSnap.compareAndSet(null, cur = new Snapshot(
-                AffinityTopologyVersion.ZERO,
-                new DiscoCache(localNode(), getSpi().getRemoteNodes())))) {
-                return cur.discoCache;
-            }
-        }
+        Snapshot cur = topSnap.get();
+
+        assert cur != null;
 
         return cur.discoCache;
     }


[37/50] [abbrv] ignite git commit: # master fix tests

Posted by nt...@apache.org.
# master fix tests


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

Branch: refs/heads/ignite-788-dev
Commit: 192fefc2382d479a62504517cf3ddc8d5188657c
Parents: f557728
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 10:09:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 10:09:24 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheUtils.java        |  4 +-
 .../ignite/internal/util/IgniteUtils.java       |  4 +-
 .../IgniteClientReconnectFailoverTest.java      | 47 ++++++++++++--------
 .../cache/IgniteCacheNearLockValueSelfTest.java |  2 +
 4 files changed, 32 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/192fefc2/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index bd2623d..e16e30d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1561,9 +1561,7 @@ public class GridCacheUtils {
      */
     @NotNull public static RuntimeException convertToCacheException(IgniteCheckedException e) {
         IgniteClientDisconnectedCheckedException disconnectedErr =
-            e instanceof IgniteClientDisconnectedCheckedException ?
-            (IgniteClientDisconnectedCheckedException)e
-            : e.getCause(IgniteClientDisconnectedCheckedException.class);
+            e.getCause(IgniteClientDisconnectedCheckedException.class);
 
         if (disconnectedErr != null) {
             assert disconnectedErr.reconnectFuture() != null : disconnectedErr;

http://git-wip-us.apache.org/repos/asf/ignite/blob/192fefc2/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 66eb596..42982b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -691,9 +691,7 @@ public abstract class IgniteUtils {
         }
 
         IgniteClientDisconnectedCheckedException disconnectedErr =
-            e instanceof IgniteClientDisconnectedCheckedException ?
-            (IgniteClientDisconnectedCheckedException)e
-            : e.getCause(IgniteClientDisconnectedCheckedException.class);
+            e.getCause(IgniteClientDisconnectedCheckedException.class);
 
         if (disconnectedErr != null) {
             assert disconnectedErr.reconnectFuture() != null : disconnectedErr;

http://git-wip-us.apache.org/repos/asf/ignite/blob/192fefc2/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
index 7cfc329..b7de3eb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 
+import javax.cache.*;
 import java.util.*;
 import java.util.concurrent.*;
 
@@ -114,37 +115,45 @@ public class IgniteClientReconnectFailoverTest extends IgniteClientReconnectFail
 
         reconnectFailover(new Callable<Void>() {
             @Override public Void call() throws Exception {
-                TreeMap<Integer, Integer> map = new TreeMap<>();
+                try {
+                    TreeMap<Integer, Integer> map = new TreeMap<>();
 
-                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
-                for (int i = 0; i < 5; i++) {
-                    Integer key = rnd.nextInt(0, 100_000);
+                    for (int i = 0; i < 5; i++) {
+                        Integer key = rnd.nextInt(0, 100_000);
 
-                    cache.put(key, key);
+                        cache.put(key, key);
 
-                    assertEquals(key, cache.get(key));
+                        assertEquals(key, cache.get(key));
 
-                    map.put(key, key);
-                }
+                        map.put(key, key);
+                    }
 
-                for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) {
-                    try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
-                        for (Map.Entry<Integer, Integer> e : map.entrySet()) {
-                            cache.put(e.getKey(), e.getValue());
+                    for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) {
+                        try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
+                            for (Map.Entry<Integer, Integer> e : map.entrySet()) {
+                                cache.put(e.getKey(), e.getValue());
 
-                            assertNotNull(cache.get(e.getKey()));
-                        }
+                                assertNotNull(cache.get(e.getKey()));
+                            }
 
-                        tx.commit();
+                            tx.commit();
+                        }
                     }
-                }
 
-                cache.putAll(map);
+                    cache.putAll(map);
 
-                Map<Integer, Integer> res = cache.getAll(map.keySet());
+                    Map<Integer, Integer> res = cache.getAll(map.keySet());
 
-                assertEquals(map, res);
+                    assertEquals(map, res);
+                }
+                catch (CacheException e) {
+                    if (e.getCause() instanceof IgniteClientDisconnectedException)
+                        throw e;
+                    else
+                        log.info("Ignore error: " + e);
+                }
 
                 return null;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/192fefc2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index cd04433..c500bbf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -49,6 +49,8 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         super.afterTestsStopped();
+
+        stopAllGrids();
     }
 
     /** {@inheritDoc} */


[40/50] [abbrv] ignite git commit: # master minor

Posted by nt...@apache.org.
# master minor


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

Branch: refs/heads/ignite-788-dev
Commit: d8f86f18315da2c6882eae72117f5f2033e3b67a
Parents: f9d2a2e
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 11:40:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 11:40:52 2015 +0300

----------------------------------------------------------------------
 .../IgniteCachePutRetryAbstractSelfTest.java    | 52 +++++++++++++-------
 ...gniteCachePutRetryTransactionalSelfTest.java | 17 ++++---
 2 files changed, 44 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d8f86f18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index bfddbe7..7f2d173 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -71,6 +71,8 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
     public void testPut() throws Exception {
         final AtomicBoolean finished = new AtomicBoolean();
 
+        int keysCnt = keysCount();
+
         IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finished.get()) {
@@ -85,13 +87,18 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
             }
         });
 
-        int keysCnt = keysCount();
 
-        for (int i = 0; i < keysCnt; i++)
-            ignite(0).cache(null).put(i, i);
+        try {
+            for (int i = 0; i < keysCnt; i++)
+                ignite(0).cache(null).put(i, i);
 
-        finished.set(true);
-        fut.get();
+            finished.set(true);
+
+            fut.get();
+        }
+        finally {
+            finished.set(true);
+        }
 
         for (int i = 0; i < keysCnt; i++)
             assertEquals(i, ignite(0).cache(null).get(i));
@@ -117,27 +124,34 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCacheAbstr
             }
         });
 
-        int keysCnt = keysCount();
+        try {
+            int keysCnt = keysCount();
 
-        boolean exceptionThrown = false;
+            boolean eThrown = false;
 
-        for (int i = 0; i < keysCnt; i++) {
-            try {
-                ignite(0).cache(null).withNoRetries().put(i, i);
-            }
-            catch (Exception e) {
-                assertTrue("Invalid exception: " + e, X.hasCause(e, ClusterTopologyCheckedException.class) || X.hasCause(e, CachePartialUpdateException.class));
+            for (int i = 0; i < keysCnt; i++) {
+                try {
+                    ignite(0).cache(null).withNoRetries().put(i, i);
+                }
+                catch (Exception e) {
+                    assertTrue("Invalid exception: " + e, X.hasCause(e, ClusterTopologyCheckedException.class) ||
+                        X.hasCause(e, CachePartialUpdateException.class));
 
-                exceptionThrown = true;
+                    eThrown = true;
 
-                break;
+                    break;
+                }
             }
-        }
 
-        assertTrue(exceptionThrown);
+            assertTrue(eThrown);
 
-        finished.set(true);
-        fut.get();
+            finished.set(true);
+
+            fut.get();
+        }
+        finally {
+            finished.set(true);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/d8f86f18/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 91c454a..1d14dec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -49,8 +49,7 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
         IgniteAtomicLong atomic = ignite(0).atomicLong("TestAtomic", 0, true);
 
         IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
-            @Override
-            public Object call() throws Exception {
+            @Override public Object call() throws Exception {
                 while (!finished.get()) {
                     stopGrid(3);
 
@@ -65,10 +64,16 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
 
         int keysCnt = keysCount();
 
-        for (int i = 0; i < keysCnt; i++)
-            atomic.incrementAndGet();
+        try {
+            for (int i = 0; i < keysCnt; i++)
+                atomic.incrementAndGet();
 
-        finished.set(true);
-        fut.get();
+            finished.set(true);
+
+            fut.get();
+        }
+        finally {
+            finished.set(true);
+        }
     }
 }


[45/50] [abbrv] 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/cf081811
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/cf081811
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/cf081811

Branch: refs/heads/ignite-788-dev
Commit: cf08181144e4a0394cedffd844988cf7fdc75093
Parents: 7494a86 7ae2b0a
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 12:37:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 12:37:32 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/GridCacheAdapter.java       | 2 +-
 .../ignite/internal/processors/cache/IgniteCacheProxy.java       | 2 +-
 .../ignite/internal/processors/cache/IgniteInternalCache.java    | 1 +
 .../main/java/org/apache/ignite/internal/util/IgniteUtils.java   | 4 ++--
 4 files changed, 5 insertions(+), 4 deletions(-)
----------------------------------------------------------------------



[43/50] [abbrv] ignite git commit: minor

Posted by nt...@apache.org.
minor


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

Branch: refs/heads/ignite-788-dev
Commit: 7ae2b0af43f914f7af4975da76db3ea6acc0e43d
Parents: d24c2da
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jul 20 12:24:39 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jul 20 12:24:39 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/IgniteInternalCache.java    | 1 +
 .../main/java/org/apache/ignite/internal/util/IgniteUtils.java   | 4 ++--
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7ae2b0af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 9972f92..7396c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -1473,6 +1473,7 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
      * @return Internal projection.
      */
     IgniteInternalCache<K, V> forSubjectId(UUID subjId);
+
     /**
      * Store DR data.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/7ae2b0af/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 42982b9..c4f6712 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -984,7 +984,7 @@ public abstract class IgniteUtils {
     }
 
     /**
-     * Returns current JVM maxMemory in the same format as {@link #heapSize(org.apache.ignite.cluster.ClusterNode, int)}.
+     * Returns current JVM maxMemory in the same format as {@link #heapSize(ClusterNode, int)}.
      *
      * @param precision Precision.
      * @return Maximum memory size in GB.
@@ -8069,7 +8069,7 @@ public abstract class IgniteUtils {
     /**
      * @param addrs Node's addresses.
      * @param port Port discovery number.
-     * @return A string compatible with {@link org.apache.ignite.cluster.ClusterNode#consistentId()} requirements.
+     * @return A string compatible with {@link ClusterNode#consistentId()} requirements.
      */
     public static String consistentId(Collection<String> addrs, int port) {
         assert !F.isEmpty(addrs);


[02/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-mys-26'

Posted by nt...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-mys-26'


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

Branch: refs/heads/ignite-788-dev
Commit: 7147f3d8ec736e013e6b2945f3ad6359d01c6ea7
Parents: de7e06f 322e5df
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 09:18:51 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 09:18:51 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryProcessor.java     | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[18/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
new file mode 100644
index 0000000..7cfc329
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverTest.java
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.transactions.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectFailoverTest extends IgniteClientReconnectFailoverAbstractTest {
+    /** */
+    protected static final String ATOMIC_CACHE = "ATOMIC_CACHE";
+
+    /** */
+    protected static final String TX_CACHE = "TX_CACHE";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setName(ATOMIC_CACHE);
+        ccfg1.setBackups(1);
+        ccfg1.setAtomicityMode(ATOMIC);
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setName(TX_CACHE);
+        ccfg2.setBackups(1);
+        ccfg2.setAtomicityMode(TRANSACTIONAL);
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectAtomicCache() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        final IgniteCache<Integer, Integer> cache = client.cache(ATOMIC_CACHE);
+
+        assertNotNull(cache);
+
+        assertEquals(ATOMIC, cache.getConfiguration(CacheConfiguration.class).getAtomicityMode());
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                TreeMap<Integer, Integer> map = new TreeMap<>();
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                for (int i = 0; i < 10; i++) {
+                    Integer key = rnd.nextInt(0, 100_000);
+
+                    cache.put(key, key);
+
+                    assertEquals(key, cache.get(key));
+
+                    map.put(key, key);
+                }
+
+                cache.putAll(map);
+
+                Map<Integer, Integer> res = cache.getAll(map.keySet());
+
+                assertEquals(map, res);
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectTxCache() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        final IgniteCache<Integer, Integer> cache = client.cache(TX_CACHE);
+
+        assertNotNull(cache);
+
+        assertEquals(TRANSACTIONAL, cache.getConfiguration(CacheConfiguration.class).getAtomicityMode());
+
+        final IgniteTransactions txs = client.transactions();
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                TreeMap<Integer, Integer> map = new TreeMap<>();
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                for (int i = 0; i < 5; i++) {
+                    Integer key = rnd.nextInt(0, 100_000);
+
+                    cache.put(key, key);
+
+                    assertEquals(key, cache.get(key));
+
+                    map.put(key, key);
+                }
+
+                for (TransactionConcurrency txConcurrency : TransactionConcurrency.values()) {
+                    try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
+                        for (Map.Entry<Integer, Integer> e : map.entrySet()) {
+                            cache.put(e.getKey(), e.getValue());
+
+                            assertNotNull(cache.get(e.getKey()));
+                        }
+
+                        tx.commit();
+                    }
+                }
+
+                cache.putAll(map);
+
+                Map<Integer, Integer> res = cache.getAll(map.keySet());
+
+                assertEquals(map, res);
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectComputeApi() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        final IgniteCompute comp = client.compute();
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                comp.call(new DummyClosure());
+
+                comp.broadcast(new DummyClosure());
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectStreamerApi() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                stream(ATOMIC_CACHE);
+
+                stream(TX_CACHE);
+
+                return null;
+            }
+
+            private void stream(String cacheName) {
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                try (IgniteDataStreamer<Integer, Integer> streamer = client.dataStreamer(cacheName)) {
+                    streamer.allowOverwrite(true);
+
+                    streamer.perNodeBufferSize(10);
+
+                    for (int i = 0; i < 100; i++)
+                        streamer.addData(rnd.nextInt(100_000), 0);
+                }
+            }
+        });
+    }
+
+    /**
+     *
+     */
+    public static class DummyClosure implements IgniteCallable<Object> {
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            return 1;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
new file mode 100644
index 0000000..31b4192
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectServicesTest.java
@@ -0,0 +1,260 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.processors.service.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.services.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectServicesTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteServices services = client.services();
+
+        services.deployClusterSingleton("testReconnect", new TestServiceImpl());
+
+        TestService srvc = services.serviceProxy("testReconnect", TestService.class, false);
+
+        assertNotNull(srvc);
+
+        long topVer = grid(0).cluster().topologyVersion();
+
+        assertEquals((Object)topVer, srvc.test());
+
+        Ignite srv = clientRouter(client);
+
+        reconnectClientNode(client, srv, null);
+
+        CountDownLatch latch = new CountDownLatch(1);
+
+        DummyService.exeLatch("testReconnect2", latch);
+
+        services.deployClusterSingleton("testReconnect2", new DummyService());
+
+        assertTrue(latch.await(5000, TimeUnit.MILLISECONDS));
+
+        assertEquals((Object)(topVer + 2), srvc.test());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testServiceRemove() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteServices clnServices = client.services();
+
+        final IgniteServices srvServices = srv.services();
+
+        srvServices.deployClusterSingleton("testServiceRemove", new TestServiceImpl());
+
+        final TestService srvc = clnServices.serviceProxy("testServiceRemove", TestService.class, false);
+
+        assertNotNull(srvc);
+
+        assertNotNull(srvc.test());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvServices.cancel("testServiceRemove");
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return srvc.test();
+            }
+        }, IgniteException.class, null);
+
+        clnServices.deployClusterSingleton("testServiceRemove", new TestServiceImpl());
+
+        TestService newSrvc = clnServices.serviceProxy("testServiceRemove", TestService.class, false);
+
+        assertNotNull(newSrvc);
+        assertNotNull(newSrvc.test());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectInDeploying() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteServices services = client.services();
+
+        Ignite srv = clientRouter(client);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(GridNearTxPrepareResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    services.deployClusterSingleton("testReconnectInDeploying", new TestServiceImpl());
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final IgniteServices services = client.services();
+
+        final Ignite srv = clientRouter(client);
+
+        services.deployClusterSingleton("testReconnectInProgress", new TestServiceImpl());
+
+        final TestService srvc = services.serviceProxy("testReconnectInProgress", TestService.class, false);
+
+        assertNotNull(srvc);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(GridJobExecuteResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    srvc.test();
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+    }
+
+    /**
+     *
+     */
+    public static interface TestService {
+        /**
+         * @return Topology version.
+         */
+        public Long test();
+    }
+
+    /**
+     *
+     */
+    public static class TestServiceImpl implements Service, TestService {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public void cancel(ServiceContext ctx) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void init(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void execute(ServiceContext ctx) throws Exception {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Long test() {
+            assertFalse(ignite.cluster().localNode().isClient());
+
+            return ignite.cluster().topologyVersion();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
new file mode 100644
index 0000000..98c3d0f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStopTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectStopTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStopWhenDisconnected() throws Exception {
+        clientMode = true;
+
+        Ignite client = startGrid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        final TestTcpDiscoverySpi clientSpi = spi(client);
+
+        log.info("Block reconnect.");
+
+        clientSpi.writeLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                } else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(disconnectLatch);
+
+        IgniteFuture<?> reconnectFut = null;
+
+        try {
+            client.getOrCreateCache(new CacheConfiguration<>());
+
+            fail();
+        }
+        catch (IgniteClientDisconnectedException e) {
+            log.info("Expected operation exception: " + e);
+
+            reconnectFut = e.reconnectFuture();
+        }
+
+        assertNotNull(reconnectFut);
+
+        client.close();
+
+        try {
+            reconnectFut.get();
+
+            fail();
+        }
+        catch (IgniteException e) {
+            log.info("Expected reconnect exception: " + e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
new file mode 100644
index 0000000..a4cf77f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectStreamerTest.java
@@ -0,0 +1,233 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.datastreamer.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.testframework.*;
+
+import javax.cache.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectStreamerTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    public static final String CACHE_NAME = "streamer";
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<Integer, Integer>(CACHE_NAME)
+            .setAtomicityMode(ATOMIC)
+            .setCacheMode(PARTITIONED);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStreamerReconnect() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
+
+        IgniteDataStreamer<Integer, Integer> streamer = client.dataStreamer(CACHE_NAME);
+
+        for (int i = 0; i < 50; i++)
+            streamer.addData(i, i);
+
+        streamer.flush();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return srvCache.localSize() == 50;
+            }
+        }, 2000L);
+
+        assertEquals(50, srvCache.localSize());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                try {
+                    client.dataStreamer(CACHE_NAME);
+
+                    fail();
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    assertNotNull(e.reconnectFuture());
+                }
+            }
+        });
+
+        checkStreamerClosed(streamer);
+
+        streamer = client.dataStreamer(CACHE_NAME);
+
+        for (int i = 50; i < 100; i++)
+            streamer.addData(i, i);
+
+        streamer.flush();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return srvCache.localSize() == 100;
+            }
+        }, 2000L);
+
+        assertEquals(100, srvCache.localSize());
+
+        streamer.close();
+
+        streamer.future().get(2, TimeUnit.SECONDS);
+
+        srvCache.removeAll();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStreamerReconnectInProgress() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final IgniteCache<Object, Object> srvCache = srv.cache(CACHE_NAME);
+
+        final IgniteDataStreamer<Integer, Integer> streamer = client.dataStreamer(CACHE_NAME);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(DataStreamerResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    for (int i = 0; i < 50; i++)
+                        streamer.addData(i, i);
+
+                    streamer.flush();
+                }
+                catch (CacheException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+                finally {
+                    streamer.close();
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        checkStreamerClosed(streamer);
+
+        IgniteDataStreamer<Integer, Integer> streamer2 = client.dataStreamer(CACHE_NAME);
+
+        for (int i = 0; i < 50; i++)
+            streamer2.addData(i, i);
+
+        streamer2.close();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return srvCache.localSize() == 50;
+            }
+        }, 2000L);
+
+        assertEquals(50, srvCache.localSize());
+    }
+
+    /**
+     * @param streamer Streamer.
+     */
+    private void checkStreamerClosed(IgniteDataStreamer<Integer, Integer> streamer) {
+        try {
+            streamer.addData(100, 100);
+
+            fail();
+        }
+        catch (CacheException e) {
+            checkAndWait(e);
+        }
+
+        try {
+            streamer.flush();
+
+            fail();
+        }
+        catch (CacheException e) {
+            checkAndWait(e);
+        }
+
+        try {
+            streamer.future().get();
+
+            fail();
+        }
+        catch (CacheException e) {
+            checkAndWait(e);
+        }
+
+        streamer.tryFlush();
+
+        streamer.close();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 27c2a61..a392245 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -62,6 +62,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientReconnectDisabled(true);
 
         if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
             cfg.setClientMode(true);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManagerStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManagerStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManagerStopSelfTest.java
index 9780080..62f5d41 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManagerStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentManagerStopSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.managers.deployment;
 
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.resource.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.deployment.*;
@@ -95,5 +96,11 @@ public class GridDeploymentManagerStopSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Override public boolean unregister(String rsrcName) { return false; }
+
+        /** {@inheritDoc} */
+        @Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) { /* No-op. */ }
+
+        /** {@inheritDoc} */
+        @Override public void onClientReconnected(boolean clusterRestarted) { /* No-op. */ }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index 074f6ff..9c30f23 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -263,7 +263,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
 
         e.printStackTrace(pw);
 
-        assertTrue(sw.toString().contains("grid is stopping"));
+        assertTrue(sw.toString().contains("node is stopping"));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
index 071341e..8703d32 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheDynamicStopSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.IgniteException;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -27,7 +26,7 @@ import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 
-import javax.cache.Cache;
+import javax.cache.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 
@@ -89,7 +88,8 @@ public class IgniteCacheDynamicStopSelfTest extends GridCommonAbstractTest {
                             @Override public void apply(IgniteFuture<?> f) {
                                 try {
                                     f.get();
-                                } catch (IgniteException ignore) {
+                                }
+                                catch (CacheException ignore) {
                                     // This may be debugged.
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
index af3ea9d..30bf5dc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.transactions.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.indexing.*;
 import org.apache.ignite.testframework.*;

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
index d78add6..53404cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSystemTransactionsSelfTest.java
@@ -75,7 +75,7 @@ public class IgniteCacheSystemTransactionsSelfTest extends GridCacheAbstractSelf
             jcache.get("1");
             jcache.put("1", "11");
 
-            GridCacheAdapter<Object, Object> utilityCache = ignite.context().cache().utilityCache();
+            IgniteInternalCache<Object, Object> utilityCache = ignite.context().cache().utilityCache();
 
             utilityCache.getAndPutIfAbsent("2", "2");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedInvalidateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedInvalidateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedInvalidateSelfTest.java
index 19e40bf..7a2e8b3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedInvalidateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedInvalidateSelfTest.java
@@ -220,7 +220,8 @@ public class GridCacheReplicatedInvalidateSelfTest extends GridCommonAbstractTes
             Object msg0 = ((GridIoMessage)msg).message();
 
             if (!(msg0 instanceof GridClockDeltaSnapshotMessage)) {
-                info("Sending message [locNodeId=" + getLocalNodeId() + ", destNodeId= " + destNode.id()
+                info("Sending message [locNodeId=" + ignite.cluster().localNode().id() +
+                    ", destNodeId= " + destNode.id()
                     + ", msg=" + msg + ']');
 
                 synchronized (msgCntMap) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
index e9d7a45..9a883b3 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/hashmap/GridCacheTestContext.java
@@ -55,8 +55,8 @@ public class GridCacheTestContext<K, V> extends GridCacheContext<K, V> {
                 new GridCacheDeploymentManager<K, V>(),
                 new GridCachePartitionExchangeManager<K, V>(),
                 new GridCacheIoManager(),
-                null,
-                new CacheNoopJtaManager()
+                new CacheNoopJtaManager(),
+                null
             ),
             defaultCacheConfiguration(),
             CacheType.USER,

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index ec6a526..63db0c1 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -111,6 +111,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /** */
     private IgniteInClosure2X<TcpDiscoveryAbstractMessage, Socket> afterWrite;
 
+    /** */
+    private boolean reconnectDisabled;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -159,6 +162,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         disco.setJoinTimeout(joinTimeout);
         disco.setNetworkTimeout(netTimeout);
 
+        disco.setClientReconnectDisabled(reconnectDisabled);
+
         disco.afterWrite(afterWrite);
 
         cfg.setDiscoverySpi(disco);
@@ -524,7 +529,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         TestTcpDiscoverySpi spi = ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
 
-        spi.pauseAll();
+        spi.pauseAll(false);
 
         try {
             spi.brakeConnection();
@@ -568,7 +573,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         TestTcpDiscoverySpi spi = ((TestTcpDiscoverySpi)ignite.configuration().getDiscoverySpi());
 
-        spi.pauseAll();
+        spi.pauseAll(false);
 
         try {
             spi.brakeConnection();
@@ -606,7 +611,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         attachListeners(2, 2);
 
-        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).pauseAll();
+        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).pauseAll(true);
 
         stopGrid("server-2");
 
@@ -633,6 +638,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     public void testClientSegmentation() throws Exception {
         clientsPerSrv = 1;
 
+        reconnectDisabled = true;
+
         startServerNodes(3);
         startClientNodes(3);
 
@@ -656,6 +663,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         final TcpDiscoverySpi disco = (TcpDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi();
 
         try {
+            log.info("Fail server: " + 2);
+
             failServer(2);
 
             await(srvFailedLatch);
@@ -886,8 +895,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         try {
             startClientNodes(1);
 
-            assertEquals(G.ignite("server-0").cluster().localNode().id(), ((TcpDiscoveryNode)G.ignite("client-0")
-                .cluster().localNode()).clientRouterNodeId());
+            assertEquals(G.ignite("server-0").cluster().localNode().id(),
+                ((TcpDiscoveryNode) G.ignite("client-0").cluster().localNode()).clientRouterNodeId());
 
             checkNodes(2, 1);
 
@@ -1206,6 +1215,528 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectAfterFail() throws Exception {
+        reconnectAfterFail(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectAfterFailTopologyChanged() throws Exception {
+        reconnectAfterFail(true);
+    }
+
+    /**
+     * @param changeTop If {@code true} topology is changed after client disconnects.
+     * @throws Exception If failed.
+     */
+    private void reconnectAfterFail(final boolean changeTop) throws Exception {
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        Ignite srv = G.ignite("server-0");
+
+        TestTcpDiscoverySpi srvSpi = ((TestTcpDiscoverySpi)srv.configuration().getDiscoverySpi());
+
+        Ignite client = G.ignite("client-0");
+
+        final ClusterNode clientNode = client.cluster().localNode();
+
+        final UUID clientId = clientNode.id();
+
+        final TestTcpDiscoverySpi clientSpi = ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi());
+
+        assertEquals(2L, clientNode.order());
+
+        final CountDownLatch failLatch = new CountDownLatch(1);
+
+        final CountDownLatch joinLatch = new CountDownLatch(1);
+
+        srv.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                info("Server event: " + evt);
+
+                DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+                if (evt0.eventNode().id().equals(clientId) && (evt.type() == EVT_NODE_FAILED)) {
+                    if (evt.type() == EVT_NODE_FAILED)
+                        failLatch.countDown();
+                }
+                else if (evt.type() == EVT_NODE_JOINED) {
+                    TcpDiscoveryNode node = (TcpDiscoveryNode)evt0.eventNode();
+
+                    if ("client-0".equals(node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME))) {
+                        assertEquals(changeTop ? 5L : 4L, node.order());
+
+                        joinLatch.countDown();
+                    }
+                }
+
+                return true;
+            }
+        }, EVT_NODE_FAILED, EVT_NODE_JOINED);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                info("Client event: " + evt);
+
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    assertEquals(1, reconnectLatch.getCount());
+
+                    disconnectLatch.countDown();
+
+                    if (changeTop)
+                        clientSpi.pauseAll(false);
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    assertEquals(0, disconnectLatch.getCount());
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        if (changeTop) {
+            Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
+
+            srvNodeIds.add(g.cluster().localNode().id());
+
+            clientSpi.resumeAll();
+        }
+
+        assertTrue(disconnectLatch.await(5000, MILLISECONDS));
+        assertTrue(reconnectLatch.await(5000, MILLISECONDS));
+        assertTrue(failLatch.await(5000, MILLISECONDS));
+        assertTrue(joinLatch.await(5000, MILLISECONDS));
+
+        long topVer = changeTop ? 5L : 4L;
+
+        assertEquals(topVer, client.cluster().localNode().order());
+
+        assertEquals(topVer, client.cluster().topologyVersion());
+
+        Collection<ClusterNode> clientTop = client.cluster().topology(topVer);
+
+        assertEquals(changeTop ? 3 : 2, clientTop.size());
+
+        clientNodeIds.remove(clientId);
+
+        clientNodeIds.add(client.cluster().localNode().id());
+
+        checkNodes(changeTop ? 2 : 1, 1);
+
+        Ignite g = startGrid("server-" + srvIdx.getAndIncrement());
+
+        srvNodeIds.add(g.cluster().localNode().id());
+
+        checkNodes(changeTop ? 3 : 2, 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectAfterFailConcurrentJoin() throws Exception {
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        Ignite srv = G.ignite("server-0");
+
+        TestTcpDiscoverySpi srvSpi = ((TestTcpDiscoverySpi)srv.configuration().getDiscoverySpi());
+
+        Ignite client = G.ignite("client-0");
+
+        final ClusterNode clientNode = client.cluster().localNode();
+
+        assertEquals(2L, clientNode.order());
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                info("Client event: " + evt);
+
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    assertEquals(1, reconnectLatch.getCount());
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    assertEquals(0, disconnectLatch.getCount());
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        final int CLIENTS = 20;
+
+        clientsPerSrv = CLIENTS + 1;
+
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                latch.await();
+
+                Ignite g = startGrid("client-" + clientIdx.getAndIncrement());
+
+                clientNodeIds.add(g.cluster().localNode().id());
+
+                return null;
+            }
+        }, CLIENTS, "start-client");
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        latch.countDown();
+
+        assertTrue(disconnectLatch.await(10_000, MILLISECONDS));
+        assertTrue(reconnectLatch.await(10_000, MILLISECONDS));
+
+        clientNodeIds.add(client.cluster().localNode().id());
+
+        fut.get();
+
+        checkNodes(1, CLIENTS + 1);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientFailReconnectDisabled() throws Exception {
+        reconnectDisabled = true;
+
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        Ignite srv = G.ignite("server-0");
+
+        TestTcpDiscoverySpi srvSpi = ((TestTcpDiscoverySpi)srv.configuration().getDiscoverySpi());
+
+        Ignite client = G.ignite("client-0");
+
+        final CountDownLatch segmentedLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_NODE_SEGMENTED)
+                    segmentedLatch.countDown();
+
+                return false;
+            }
+        }, EVT_NODE_SEGMENTED);
+
+        srvFailedLatch = new CountDownLatch(1);
+
+        attachListeners(1, 0);
+
+        log.info("Fail client node.");
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        assertTrue(srvFailedLatch.await(5000, MILLISECONDS));
+        assertTrue(segmentedLatch.await(5000, MILLISECONDS));
+
+        checkNodes(1, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectSegmentedAfterJoinTimeoutServerFailed() throws Exception {
+        reconnectSegmentedAfterJoinTimeout(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectSegmentedAfterJoinTimeoutNetworkError() throws Exception {
+        reconnectSegmentedAfterJoinTimeout(false);
+    }
+
+    /**
+     * @param failSrv If {@code true} fails server, otherwise server does not send join message.
+     * @throws Exception If failed.
+     */
+    private void reconnectSegmentedAfterJoinTimeout(boolean failSrv) throws Exception {
+        netTimeout = 4000;
+        joinTimeout = 5000;
+
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        final Ignite srv = G.ignite("server-0");
+        Ignite client = G.ignite("client-0");
+
+        TestTcpDiscoverySpi srvSpi = ((TestTcpDiscoverySpi)srv.configuration().getDiscoverySpi());
+        TestTcpDiscoverySpi clientSpi = ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi());
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch segmentedLatch = new CountDownLatch(1);
+        final AtomicBoolean err = new AtomicBoolean(false);
+
+        if (!failSrv) {
+            srvFailedLatch = new CountDownLatch(1);
+
+            attachListeners(1, 0);
+        }
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    log.info("Disconnected event.");
+
+                    assertEquals(1, segmentedLatch.getCount());
+                    assertEquals(1, disconnectLatch.getCount());
+                    assertFalse(err.get());
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_NODE_SEGMENTED) {
+                    log.info("Segmented event.");
+
+                    assertEquals(1, segmentedLatch.getCount());
+                    assertEquals(0, disconnectLatch.getCount());
+                    assertFalse(err.get());
+
+                    segmentedLatch.countDown();
+                }
+                else {
+                    log.error("Unexpected event: " + evt);
+
+                    err.set(true);
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED, EVT_NODE_SEGMENTED);
+
+        if (failSrv) {
+            log.info("Fail server.");
+
+            failServer(0);
+        }
+        else {
+            log.info("Fail client connection.");
+
+            srvSpi.failClientReconnect.set(1_000_000);
+            srvSpi.skipNodeAdded = true;
+
+            clientSpi.brakeConnection();
+        }
+
+        assertTrue(disconnectLatch.await(10_000, MILLISECONDS));
+
+        assertTrue(segmentedLatch.await(10_000, MILLISECONDS));
+
+        waitSegmented(client);
+
+        assertFalse(err.get());
+
+        if (!failSrv) {
+            await(srvFailedLatch);
+
+            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                @Override public boolean apply() {
+                    return srv.cluster().nodes().size() == 1;
+                }
+            }, 10_000);
+
+            checkNodes(1, 0);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectClusterRestart() throws Exception {
+        netTimeout = 3000;
+        joinTimeout = 60_000;
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+        final AtomicBoolean err = new AtomicBoolean(false);
+
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        Ignite srv = G.ignite("server-0");
+        Ignite client = G.ignite("client-0");
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    log.info("Disconnected event.");
+
+                    assertEquals(1, reconnectLatch.getCount());
+                    assertEquals(1, disconnectLatch.getCount());
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    log.info("Reconnected event.");
+
+                    assertEquals(1, reconnectLatch.getCount());
+                    assertEquals(0, disconnectLatch.getCount());
+                    assertFalse(err.get());
+
+                    reconnectLatch.countDown();
+                }
+                else {
+                    log.error("Unexpected event: " + evt);
+
+                    err.set(true);
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED, EVT_NODE_SEGMENTED);
+
+        log.info("Stop server.");
+
+        srv.close();
+
+        assertTrue(disconnectLatch.await(10_000, MILLISECONDS));
+
+        srvNodeIds.clear();
+        srvIdx.set(0);
+
+        Thread.sleep(3000);
+
+        log.info("Restart server.");
+
+        startServerNodes(1);
+
+        assertTrue(reconnectLatch.await(10_000, MILLISECONDS));
+
+        clientNodeIds.clear();
+        clientNodeIds.add(client.cluster().localNode().id());
+
+        checkNodes(1, 1);
+
+        assertFalse(err.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDisconnectAfterNetworkTimeout() throws Exception {
+        netTimeout = 5000;
+        joinTimeout = 60_000;
+        maxMissedClientHbs = 2;
+
+        startServerNodes(1);
+
+        startClientNodes(1);
+
+        final Ignite srv = G.ignite("server-0");
+        Ignite client = G.ignite("client-0");
+
+        TestTcpDiscoverySpi srvSpi = ((TestTcpDiscoverySpi)srv.configuration().getDiscoverySpi());
+        TestTcpDiscoverySpi clientSpi = ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi());
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+        final AtomicBoolean err = new AtomicBoolean(false);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override
+            public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    log.info("Disconnected event.");
+
+                    assertEquals(1, reconnectLatch.getCount());
+                    assertEquals(1, disconnectLatch.getCount());
+                    assertFalse(err.get());
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    log.info("Reconnected event.");
+
+                    assertEquals(1, reconnectLatch.getCount());
+                    assertEquals(0, disconnectLatch.getCount());
+                    assertFalse(err.get());
+
+                    reconnectLatch.countDown();
+                }
+                else {
+                    log.error("Unexpected event: " + evt);
+
+                    err.set(true);
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED, EVT_NODE_SEGMENTED);
+
+        log.info("Fail client connection1.");
+
+        srvSpi.failClientReconnect.set(1_000_000);
+        srvSpi.skipNodeAdded = true;
+
+        clientSpi.brakeConnection();
+
+        assertTrue(disconnectLatch.await(10_000, MILLISECONDS));
+
+        log.info("Fail client connection2.");
+
+        srvSpi.failClientReconnect.set(0);
+        srvSpi.skipNodeAdded = false;
+
+        clientSpi.brakeConnection();
+
+        assertTrue(reconnectLatch.await(10_000, MILLISECONDS));
+
+        clientNodeIds.clear();
+
+        clientNodeIds.add(client.cluster().localNode().id());
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override
+            public boolean apply() {
+                return srv.cluster().nodes().size() == 2;
+            }
+        }, 10_000);
+
+        checkNodes(1, 1);
+
+        assertFalse(err.get());
+    }
+
+    /**
+     * @param ignite Ignite.
+     * @throws Exception If failed.
+     */
+    private void waitSegmented(final Ignite ignite) throws Exception {
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return IgniteState.STOPPED_ON_SEGMENTATION == Ignition.state(ignite.name());
+            }
+        }, 5000);
+
+        assertEquals(IgniteState.STOPPED_ON_SEGMENTATION, Ignition.state(ignite.name()));
+    }
+
+    /**
      * @param clientIdx Client index.
      * @param srvIdx Server index.
      * @throws Exception In case of error.
@@ -1401,7 +1932,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     private void checkRemoteNodes(Ignite ignite, int expCnt) {
         Collection<ClusterNode> nodes = ignite.cluster().forRemotes().nodes();
 
-        assertEquals(expCnt, nodes.size());
+        assertEquals("Unexpected state for node: " + ignite.name(), expCnt, nodes.size());
 
         for (ClusterNode node : nodes) {
             UUID id = node.id();
@@ -1420,7 +1951,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      * @throws InterruptedException If interrupted.
      */
     private void await(CountDownLatch latch) throws InterruptedException {
-        assertTrue("Latch count: " + latch.getCount(), latch.await(10000, MILLISECONDS));
+        assertTrue("Latch count: " + latch.getCount(), latch.await(10_000, MILLISECONDS));
     }
 
     /**
@@ -1471,6 +2002,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         /** */
         private volatile String delayJoinAckFor;
 
+        /** */
+        private volatile boolean skipNodeAdded;
+
         /**
          * @param lock Lock.
          */
@@ -1543,6 +2077,13 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
             boolean fail = false;
 
+            if (skipNodeAdded &&
+                (msg instanceof TcpDiscoveryNodeAddedMessage || msg instanceof TcpDiscoveryNodeAddFinishedMessage)) {
+                log.info("Skip message: " + msg);
+
+                return;
+            }
+
             if (msg instanceof TcpDiscoveryNodeAddedMessage)
                 fail = failNodeAdded.getAndDecrement() > 0;
             else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
@@ -1577,12 +2118,13 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param suspend If {@code true} suspends worker threads.
          */
-        public void pauseAll() {
+        public void pauseAll(boolean suspend) {
             pauseResumeOperation(true, openSockLock, writeLock);
 
-            impl.workerThread().suspend();
+            if (suspend)
+                impl.workerThread().suspend();
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
index 159c451..dacbf55 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/multijvm/IgniteClusterProcessProxy.java
@@ -317,4 +317,9 @@ public class IgniteClusterProcessProxy implements IgniteClusterEx {
     @Override public ClusterMetrics metrics() throws IgniteException {
         throw new UnsupportedOperationException("Operation is not supported yet.");
     }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public IgniteFuture<?> clientReconnectFuture() {
+        throw new UnsupportedOperationException("Operation is not supported yet.");
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
new file mode 100644
index 0000000..66c9835
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientReconnectTestSuite.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Client Reconnect Test Suite");
+
+        suite.addTestSuite(IgniteClientReconnectStopTest.class);
+        suite.addTestSuite(IgniteClientReconnectApiExceptionTest.class);
+        suite.addTestSuite(IgniteClientReconnectDiscoveryStateTest.class);
+        suite.addTestSuite(IgniteClientReconnectCacheTest.class);
+        suite.addTestSuite(IgniteClientReconnectContinuousProcessorTest.class);
+        suite.addTestSuite(IgniteClientReconnectComputeTest.class);
+        suite.addTestSuite(IgniteClientReconnectAtomicsTest.class);
+        suite.addTestSuite(IgniteClientReconnectCollectionsTest.class);
+        suite.addTestSuite(IgniteClientReconnectServicesTest.class);
+        suite.addTestSuite(IgniteClientReconnectStreamerTest.class);
+        suite.addTestSuite(IgniteClientReconnectFailoverTest.class);
+
+        return suite;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 06c0961..c76dbe7 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
@@ -1439,6 +1439,11 @@ public class IgniteH2Indexing implements GridQueryIndexing {
             fut.get();
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        rdcQryExec.onDisconnected(reconnectFut);
+    }
+
     /**
      * Wrapper to store connection and flag is schema set or not.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
index af29647..2b2996d 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMergeIndex.java
@@ -26,6 +26,7 @@ import org.h2.table.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
+import javax.cache.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -40,7 +41,7 @@ public abstract class GridMergeIndex extends BaseIndex {
     private static final int MAX_FETCH_SIZE = getInteger(IGNITE_SQL_MERGE_TABLE_MAX_SIZE, 10_000);
 
     /** All rows number. */
-    private final AtomicInteger expectedRowsCnt = new AtomicInteger(0);
+    private final AtomicInteger expRowsCnt = new AtomicInteger(0);
 
     /** Remaining rows per source node ID. */
     private final ConcurrentMap<UUID, Counter> remainingRows = new ConcurrentHashMap8<>();
@@ -75,8 +76,8 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /** {@inheritDoc} */
-    @Override public long getRowCount(Session session) {
-        return expectedRowsCnt.get();
+    @Override public long getRowCount(Session ses) {
+        return expRowsCnt.get();
     }
 
     /** {@inheritDoc} */
@@ -93,6 +94,23 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /**
+     * @param e Error.
+     */
+    public void fail(final CacheException e) {
+        for (UUID nodeId0 : remainingRows.keySet()) {
+            addPage0(new GridResultPage(null, nodeId0, null) {
+                @Override public boolean isFail() {
+                    return true;
+                }
+
+                @Override public void fetchNextPage() {
+                    throw e;
+                }
+            });
+        }
+    }
+
+    /**
      * @param nodeId Node ID.
      */
     public void fail(UUID nodeId) {
@@ -120,7 +138,7 @@ public abstract class GridMergeIndex extends BaseIndex {
             assert !cnt.initialized : "Counter is already initialized.";
 
             cnt.addAndGet(allRows);
-            expectedRowsCnt.addAndGet(allRows);
+            expRowsCnt.addAndGet(allRows);
 
             // We need this separate flag to handle case when the first source contains only one page
             // and it will signal that all remaining counters are zero and fetch is finished.
@@ -162,7 +180,7 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /** {@inheritDoc} */
-    @Override public Cursor find(Session session, SearchRow first, SearchRow last) {
+    @Override public Cursor find(Session ses, SearchRow first, SearchRow last) {
         if (fetched == null)
             throw new IgniteException("Fetched result set was too large.");
 
@@ -176,7 +194,7 @@ public abstract class GridMergeIndex extends BaseIndex {
      * @return {@code true} If we have fetched all the remote rows.
      */
     public boolean fetchedAll() {
-        return fetchedCnt == expectedRowsCnt.get();
+        return fetchedCnt == expRowsCnt.get();
     }
 
     /**
@@ -200,32 +218,32 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /** {@inheritDoc} */
-    @Override public void close(Session session) {
+    @Override public void close(Session ses) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void add(Session session, Row row) {
+    @Override public void add(Session ses, Row row) {
         throw DbException.getUnsupportedException("add");
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(Session session, Row row) {
+    @Override public void remove(Session ses, Row row) {
         throw DbException.getUnsupportedException("remove row");
     }
 
     /** {@inheritDoc} */
-    @Override public double getCost(Session session, int[] masks, TableFilter filter, SortOrder sortOrder) {
+    @Override public double getCost(Session ses, int[] masks, TableFilter filter, SortOrder sortOrder) {
         return getRowCountApproximation() + Constants.COST_ROW_OFFSET;
     }
 
     /** {@inheritDoc} */
-    @Override public void remove(Session session) {
+    @Override public void remove(Session ses) {
         throw DbException.getUnsupportedException("remove index");
     }
 
     /** {@inheritDoc} */
-    @Override public void truncate(Session session) {
+    @Override public void truncate(Session ses) {
         throw DbException.getUnsupportedException("truncate");
     }
 
@@ -235,7 +253,7 @@ public abstract class GridMergeIndex extends BaseIndex {
     }
 
     /** {@inheritDoc} */
-    @Override public Cursor findFirstOrLast(Session session, boolean first) {
+    @Override public Cursor findFirstOrLast(Session ses, boolean first) {
         throw DbException.getUnsupportedException("findFirstOrLast");
     }
 
@@ -299,6 +317,7 @@ public abstract class GridMergeIndex extends BaseIndex {
         private Iterator<Row> stream;
 
         /**
+         * @param stream Iterator.
          */
         public FetchingCursor(Iterator<Row> stream) {
             super(new FetchedIterator());

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 32d1c95..cde3288 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
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.processors.query.h2.twostep.messages.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.h2.command.*;
@@ -47,6 +48,7 @@ import org.h2.table.*;
 import org.h2.tools.*;
 import org.h2.util.*;
 import org.h2.value.*;
+import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
 import javax.cache.*;
@@ -234,10 +236,15 @@ public class GridReduceQueryExecutor {
                     Object errState = r.state.get();
 
                     if (errState != null) {
+                        CacheException err0 = errState instanceof CacheException ? (CacheException)errState : null;
+
+                        if (err0 != null && err0.getCause() instanceof IgniteClientDisconnectedException)
+                            throw err0;
+
                         CacheException e = new CacheException("Failed to fetch data from node: " + node.id());
 
-                        if (errState instanceof CacheException)
-                            e.addSuppressed((Throwable)errState);
+                        if (err0 != null)
+                            e.addSuppressed(err0);
 
                         throw e;
                     }
@@ -301,6 +308,7 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param cctx Cache context.
      * @return {@code true} If cache context
      */
     private boolean hasMovingPartitions(GridCacheContext<?,?> cctx) {
@@ -481,6 +489,12 @@ public class GridReduceQueryExecutor {
             runs.put(qryReqId, r);
 
             try {
+                if (ctx.clientDisconnected()) {
+                    throw new CacheException("Query was cancelled, client node disconnected.",
+                        new IgniteClientDisconnectedException(ctx.cluster().clientReconnectFuture(),
+                        "Client node disconnected."));
+                }
+
                 Collection<GridCacheSqlQuery> mapQrys = qry.mapQueries();
 
                 if (qry.explain()) {
@@ -506,8 +520,14 @@ public class GridReduceQueryExecutor {
                     Object state = r.state.get();
 
                     if (state != null) {
-                        if (state instanceof CacheException)
-                            throw new CacheException("Failed to run map query remotely.", (CacheException)state);
+                        if (state instanceof CacheException) {
+                            CacheException err = (CacheException)state;
+
+                            if (err.getCause() instanceof IgniteClientDisconnectedException)
+                                throw err;
+
+                            throw new CacheException("Failed to run map query remotely.", err);
+                        }
 
                         if (state instanceof AffinityTopologyVersion) {
                             retry = true;
@@ -550,7 +570,20 @@ public class GridReduceQueryExecutor {
             catch (IgniteCheckedException | RuntimeException e) {
                 U.closeQuiet(r.conn);
 
-                throw new CacheException("Failed to run reduce query locally.", e);
+                if (e instanceof CacheException)
+                    throw (CacheException)e;
+
+                Throwable cause = e;
+
+                if (e instanceof IgniteCheckedException) {
+                    Throwable disconnectedErr =
+                        ((IgniteCheckedException)e).getCause(IgniteClientDisconnectedException.class);
+
+                    if (disconnectedErr != null)
+                        cause = disconnectedErr;
+                }
+
+                throw new CacheException("Failed to run reduce query locally.", cause);
             }
             finally {
                 if (!runs.remove(qryReqId, r))
@@ -1082,6 +1115,17 @@ public class GridReduceQueryExecutor {
     }
 
     /**
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        CacheException err = new CacheException("Query was cancelled, client node disconnected.",
+            new IgniteClientDisconnectedException(reconnectFut, "Client node disconnected."));
+
+        for (Map.Entry<Long, QueryRun> e : runs.entrySet())
+            e.getValue().disconnected(err);
+    }
+
+    /**
      *
      */
     private static class QueryRun {
@@ -1104,7 +1148,7 @@ public class GridReduceQueryExecutor {
          * @param o Fail state object.
          * @param nodeId Node ID.
          */
-        void state(Object o, UUID nodeId) {
+        void state(Object o, @Nullable UUID nodeId) {
             assert o != null;
             assert o instanceof CacheException || o instanceof AffinityTopologyVersion : o.getClass();
 
@@ -1117,6 +1161,20 @@ public class GridReduceQueryExecutor {
             for (GridMergeTable tbl : tbls) // Fail all merge indexes.
                 tbl.getScanIndex(null).fail(nodeId);
         }
+
+        /**
+         * @param e Error.
+         */
+        void disconnected(CacheException e) {
+            if (!state.compareAndSet(null, e))
+                return;
+
+            while (latch.getCount() != 0) // We don't need to wait for all nodes to reply.
+                latch.countDown();
+
+            for (GridMergeTable tbl : tbls) // Fail all merge indexes.
+                tbl.getScanIndex(null).fail(e);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectCacheQueriesFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectCacheQueriesFailoverTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectCacheQueriesFailoverTest.java
new file mode 100644
index 0000000..23320ae
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClientReconnectCacheQueriesFailoverTest.java
@@ -0,0 +1,225 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectCacheQueriesFailoverTest extends IgniteClientReconnectFailoverAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setIndexedTypes(Integer.class, Person.class);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        final IgniteCache<Integer, Person> cache = grid(serverCount()).cache(null);
+
+        assertNotNull(cache);
+
+        for (int i = 0; i <= 10_000; i++)
+            cache.put(i, new Person(i, "name-" + i));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectCacheQueries() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        final IgniteCache<Integer, Person> cache = client.cache(null);
+
+        assertNotNull(cache);
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                SqlQuery<Integer, Person> sqlQry = new SqlQuery<>(Person.class, "where id > 1");
+
+                try {
+                    assertEquals(9999, cache.query(sqlQry).getAll().size());
+                }
+                catch (CacheException e) {
+                    if (e.getCause() instanceof IgniteClientDisconnectedException)
+                        throw e;
+                    else
+                        log.info("Ignore error: " + e);
+                }
+
+                try {
+                    SqlFieldsQuery fieldsQry = new SqlFieldsQuery("select avg(p.id) from Person p");
+
+                    List<List<?>> res = cache.query(fieldsQry).getAll();
+
+                    assertEquals(1, res.size());
+
+                    Double avg = (Double)res.get(0).get(0);
+
+                    assertEquals(5_000, avg.intValue());
+                }
+                catch (CacheException e) {
+                    if (e.getCause() instanceof IgniteClientDisconnectedException)
+                        throw e;
+                    else
+                        log.info("Ignore error: " + e);
+                }
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectScanQuery() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        final IgniteCache<Integer, Person> cache = client.cache(null);
+
+        assertNotNull(cache);
+
+        final Affinity<Integer> aff = client.affinity(null);
+
+        final Map<Integer, Integer> partMap = new HashMap<>();
+
+        for (int i = 0; i < aff.partitions(); i++)
+            partMap.put(i, 0);
+
+        for (int i = 0; i <= 10_000; i++) {
+            Integer part = aff.partition(i);
+
+            Integer size = partMap.get(part);
+
+            partMap.put(part, size + 1);
+        }
+
+        reconnectFailover(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                ScanQuery<Integer, Person> qry = new ScanQuery<>(new IgniteBiPredicate<Integer, Person>() {
+                    @Override public boolean apply(Integer key, Person val) {
+                        return val.getId() % 2 == 1;
+                    }
+                });
+
+                assertEquals(5000, cache.query(qry).getAll().size());
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                Integer part = rnd.nextInt(0, aff.partitions());
+
+                qry = new ScanQuery<>(part);
+
+                assertEquals((int)partMap.get(part), cache.query(qry).getAll().size());
+
+                return null;
+            }
+        });
+    }
+
+    /**
+     *
+     */
+    public static class Person {
+        /** */
+        @QuerySqlField
+        public int id;
+
+        /** */
+        @QuerySqlField
+        public String name;
+
+        /**
+         * @param id Id.
+         * @param name Name.
+         */
+        public Person(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+
+        /**
+         * @return Id.
+         */
+        public int getId() {
+            return id;
+        }
+
+        /**
+         * @param id Set id.
+         */
+        public void setId(int id) {
+            this.id = id;
+        }
+
+        /**
+         * @return Name.
+         */
+        public String getName() {
+            return name;
+        }
+
+        /**
+         * @param name Name.
+         */
+        public void setName(String name) {
+            this.name = name;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            return this == o || !(o == null || getClass() != o.getClass()) && id == ((Person)o).id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            return id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Person.class, this);
+        }
+    }
+}


[12/50] [abbrv] ignite git commit: ignite-890: fixes after the review

Posted by nt...@apache.org.
ignite-890: fixes after the review


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

Branch: refs/heads/ignite-788-dev
Commit: 5c5581290aba6d2958aa410d1f126f77facc626c
Parents: b601a62
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 16 09:01:45 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 16 09:01:45 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/IgniteSpiAdapter.java | 26 +++++++++++---------
 .../org/apache/ignite/spi/IgniteSpiContext.java |  6 ++---
 .../communication/tcp/TcpCommunicationSpi.java  |  2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  2 +-
 4 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/5c558129/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index df043dd..9bdf495 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -545,19 +545,17 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
     /**
      * @param obj Timeout object.
-     * @throws IgniteSpiException Thrown in case of any error.
      * @see IgniteSpiContext#addTimeoutObject(IgniteSpiTimeoutObject)
      */
-    protected void addTimeoutObject(IgniteSpiTimeoutObject obj) throws IgniteSpiException {
+    protected void addTimeoutObject(IgniteSpiTimeoutObject obj) {
         spiCtx.addTimeoutObject(obj);
     }
 
     /**
      * @param obj Timeout object.
-     * @throws IgniteSpiException Thrown in case of any error.
      * @see IgniteSpiContext#removeTimeoutObject(IgniteSpiTimeoutObject)
      */
-    protected void removeTimeoutObject(IgniteSpiTimeoutObject obj) throws IgniteSpiException {
+    protected void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
         spiCtx.removeTimeoutObject(obj);
     }
 
@@ -766,19 +764,23 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) throws IgniteSpiException {
-            if (!(ignite instanceof IgniteKernal))
-                throw new IgniteSpiException("Wrong Ignite instance is set: " + ignite);
+        @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
+            Ignite ignite0 = ignite;
 
-            ((IgniteKernal)ignite).context().timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
+            if (!(ignite0 instanceof IgniteKernal))
+                throw new IgniteSpiException("Wrong Ignite instance is set: " + ignite0);
+
+            ((IgniteKernal)ignite0).context().timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
         }
 
         /** {@inheritDoc} */
-        @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) throws IgniteSpiException {
-            if (!(ignite instanceof IgniteKernal))
-                throw new IgniteSpiException("Wrong Ignite instance is set: " + ignite);
+        @Override public void removeTimeoutObject(IgniteSpiTimeoutObject obj) {
+            Ignite ignite0 = ignite;
+
+            if (!(ignite0 instanceof IgniteKernal))
+                throw new IgniteSpiException("Wrong Ignite instance is set: " + ignite0);
 
-            ((IgniteKernal)ignite).context().timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj));
+            ((IgniteKernal)ignite0).context().timeout().removeTimeoutObject(new GridSpiTimeoutObject(obj));
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c558129/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
index addf63f..611702b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiContext.java
@@ -320,13 +320,11 @@ public interface IgniteSpiContext {
 
     /**
      * @param c Timeout object.
-     * @throws IgniteSpiException Thrown if any exception occurs.
      */
-    public void addTimeoutObject(IgniteSpiTimeoutObject c) throws IgniteSpiException;
+    public void addTimeoutObject(IgniteSpiTimeoutObject c);
 
     /**
      * @param c Timeout object.
-     * @throws IgniteSpiException Thrown if any exception occurs.
      */
-    public void removeTimeoutObject(IgniteSpiTimeoutObject c) throws IgniteSpiException;
+    public void removeTimeoutObject(IgniteSpiTimeoutObject c);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c558129/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index df7d098..b79ca55 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -2063,7 +2063,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         client = null;
                     }
 
-                    onException("Handshake timeout (will retry with increased timeout) [timeout=" + connTimeout0 +
+                    onException("Handshake timed out (will retry with increased timeout) [timeout=" + connTimeout0 +
                         ", addr=" + addr + ']', e);
 
                     if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/5c558129/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index bca2d98..6587006 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -338,7 +338,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         clientMsgWorkers.clear();
 
-        utilityPool.shutdownNow();
+        IgniteUtils.shutdownNow(ServerImpl.class, utilityPool, log);
 
         U.interrupt(statsPrinter);
         U.join(statsPrinter, log);


[41/50] [abbrv] ignite git commit: minor

Posted by nt...@apache.org.
minor


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

Branch: refs/heads/ignite-788-dev
Commit: 05672fedcf2ddd2adba764d7cb2ae75e8d607a3a
Parents: d8f86f1
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jul 20 11:51:49 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jul 20 11:51:49 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/processors/cache/GridCacheAdapter.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/05672fed/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index d2a730a..7a5a0aa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1627,7 +1627,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     ) {
         ctx.checkSecurity(SecurityPermission.CACHE_READ);
 
-       if (keyCheck)
+        if (keyCheck)
             validateCacheKeys(keys);
 
         return getAllAsync0(ctx.cacheKeysView(keys),


[36/50] [abbrv] ignite git commit: Merge branch ignite-1.3.2 into master.

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


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

Branch: refs/heads/ignite-788-dev
Commit: f557728b3108ccd3b5f8ecd1190a8591de27bbca
Parents: f0d24f6 6711d2c
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jul 17 11:00:53 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jul 17 11:00:53 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/plugin/security/SecurityPermission.java    | 7 +++++--
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java | 8 +++-----
 2 files changed, 8 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


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


[10/50] [abbrv] ignite git commit: Minor fixes in Spring Caching

Posted by nt...@apache.org.
Minor fixes in Spring Caching


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

Branch: refs/heads/ignite-788-dev
Commit: d38a70d2c12c640624c70bd0cb5634bb65b6d041
Parents: de7e06f
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jul 15 20:04:45 2015 -0700
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jul 15 20:04:45 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/cache/spring/SpringCache.java |  2 +-
 .../ignite/cache/spring/SpringCacheManager.java | 22 +++++++++++---------
 2 files changed, 13 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d38a70d2/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
index 7651fbe..08aa38b 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCache.java
@@ -48,7 +48,7 @@ class SpringCache implements Cache {
     }
 
     /** {@inheritDoc} */
-    @Override public Cache.ValueWrapper get(Object key) {
+    @Override public ValueWrapper get(Object key) {
         Object val = cache.get(key);
 
         return val != null ? new SimpleValueWrapper(val) : null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/d38a70d2/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
index afadaee..260026e 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringCacheManager.java
@@ -41,15 +41,17 @@ import java.util.concurrent.*;
  * you will need to do the following:
  * <ul>
  *     <li>
- *         Start a Ignite node with configured cache in the same JVM
- *         where you application is running.
+ *         Start an Ignite node with proper configuration in embedded mode
+ *         (i.e., in the same JVM where the application is running). It can
+ *         already have predefined caches, but it's not required - caches
+ *         will be created automatically on first access if needed.
  *     </li>
  *     <li>
- *         Configure {@code GridSpringCacheManager} as a cache provider
- *         in Spring application context.
+ *         Configure {@code SpringCacheManager} as a cache provider
+ *         in the Spring application context.
  *     </li>
  * </ul>
- * {@code GridSpringCacheManager} can start a node itself on its startup
+ * {@code SpringCacheManager} can start a node itself on its startup
  * based on provided Ignite configuration. You can provide path to a
  * Spring configuration XML file, like below (path can be absolute or
  * relative to {@code IGNITE_HOME}):
@@ -61,9 +63,9 @@ import java.util.concurrent.*;
  *         http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
  *         http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd"&gt;
  *     &lt;-- Provide configuration file path. --&gt;
- *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.GridSpringCacheManager"&gt;
+ *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager"&gt;
  *         &lt;property name="configurationPath" value="examples/config/spring-cache.xml"/&gt;
- *     &lt;/bean>
+ *     &lt;/bean&gt;
  *
  *     &lt;-- Use annotation-driven caching configuration. --&gt;
  *     &lt;cache:annotation-driven/&gt;
@@ -78,7 +80,7 @@ import java.util.concurrent.*;
  *         http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
  *         http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd"&gt;
  *     &lt;-- Provide configuration bean. --&gt;
- *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.GridSpringCacheManager"&gt;
+ *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager"&gt;
  *         &lt;property name="configuration"&gt;
  *             &lt;bean id="gridCfg" class="org.apache.ignite.configuration.IgniteConfiguration"&gt;
  *                 ...
@@ -104,7 +106,7 @@ import java.util.concurrent.*;
  *         http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans.xsd
  *         http://www.springframework.org/schema/cache http://www.springframework.org/schema/cache/spring-cache.xsd"&gt;
  *     &lt;-- Provide Grid name. --&gt;
- *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.GridSpringCacheManager"&gt;
+ *     &lt;bean id="cacheManager" class="org.apache.ignite.cache.spring.SpringCacheManager"&gt;
  *         &lt;property name="gridName" value="myGrid"/&gt;
  *     &lt;/bean>
  *
@@ -247,7 +249,7 @@ public class SpringCacheManager implements CacheManager, InitializingBean {
         if (cfgPath != null && cfg != null) {
             throw new IllegalArgumentException("Both 'configurationPath' and 'configuration' are " +
                 "provided. Set only one of these properties if you need to start a Ignite node inside of " +
-                "GridSpringCacheManager. If you already have a node running, omit both of them and set" +
+                "SpringCacheManager. If you already have a node running, omit both of them and set" +
                 "'gridName' property.");
         }
 


[15/50] [abbrv] ignite git commit: 1.4.1-SNAPSHOT

Posted by nt...@apache.org.
1.4.1-SNAPSHOT


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

Branch: refs/heads/ignite-788-dev
Commit: 69e0ad2a8eb0d834f07a5549e18331325fcc42c7
Parents: 13e55b2
Author: Ignite Teamcity <ig...@apache.org>
Authored: Thu Jul 16 12:35:56 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Thu Jul 16 12:35:56 2015 +0300

----------------------------------------------------------------------
 modules/apache-license-gen/pom.xml | 2 +-
 modules/yarn/pom.xml               | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/69e0ad2a/modules/apache-license-gen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/apache-license-gen/pom.xml b/modules/apache-license-gen/pom.xml
index 6e22668..3025eab 100644
--- a/modules/apache-license-gen/pom.xml
+++ b/modules/apache-license-gen/pom.xml
@@ -31,7 +31,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>ignite-apache-license-gen</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.4.1-SNAPSHOT</version>
 
     <build>
         <plugins>

http://git-wip-us.apache.org/repos/asf/ignite/blob/69e0ad2a/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 2679603..63aa3cd 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yarn</artifactId>
-    <version>1.3.1-SNAPSHOT</version>
+    <version>1.4.1-SNAPSHOT</version>
 
     <properties>
         <hadoop.version>2.7.0</hadoop.version>


[23/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 29e3551..84e4dc2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -893,6 +893,14 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * @param cacheId Cache ID to remove handlers for.
+     * @param type Message type.
+     */
+    public void removeHandler(int cacheId, Class<? extends GridCacheMessage> type) {
+        clsHandlers.remove(new ListenerKey(cacheId, type));
+    }
+
+    /**
      * @param msgCls Message class to check.
      * @return Message index.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManager.java
index 775daf5..ae7e9d7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManager.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
 
 /**
  * Interface for cache managers.
@@ -49,6 +50,11 @@ public interface GridCacheManager<K, V> {
     public void onKernalStop(boolean cancel);
 
     /**
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut);
+
+    /**
      * Prints memory statistics (sizes of internal data structures, etc.).
      *
      * NOTE: this method is for testing and profiling purposes only.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
index 52fade8..54b1915 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheManagerAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 
 import java.util.concurrent.atomic.*;
 
@@ -127,6 +128,11 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index f24cf01..36e108f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -208,10 +208,7 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         exchLog = cctx.logger(getClass().getName() + ".exchange");
 
         pendingExplicit = GridConcurrentFactory.newMap();
-    }
 
-    /** {@inheritDoc} */
-    @Override public void onKernalStart0() throws IgniteCheckedException {
         cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_FAILED, EVT_NODE_LEFT);
     }
 
@@ -295,15 +292,39 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
      * Cancels all client futures.
      */
     public void cancelClientFutures() {
-        IgniteCheckedException e = new IgniteCheckedException("Operation has been cancelled (grid is stopping).");
+        cancelClientFutures(new IgniteCheckedException("Operation has been cancelled (node is stopping)."));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture reconnectFut) {
+        IgniteClientDisconnectedCheckedException err = disconnectedError(reconnectFut);
+
+        cancelClientFutures(err);
+    }
 
+    /**
+     * @param err Error.
+     */
+    private void cancelClientFutures(IgniteCheckedException err) {
         for (Collection<GridCacheFuture<?>> futures : futs.values()) {
             for (GridCacheFuture<?> future : futures)
-                ((GridFutureAdapter)future).onDone(e);
+                ((GridFutureAdapter)future).onDone(err);
         }
 
         for (GridCacheAtomicFuture<?> future : atomicFuts.values())
-            ((GridFutureAdapter)future).onDone(e);
+            ((GridFutureAdapter)future).onDone(err);
+    }
+
+    /**
+     * @param reconnectFut Reconnect future.
+     * @return Client disconnected exception.
+     */
+    private IgniteClientDisconnectedCheckedException disconnectedError(@Nullable IgniteFuture<?> reconnectFut) {
+        if (reconnectFut == null)
+            reconnectFut = cctx.kernalContext().cluster().clientReconnectFuture();
+
+        return new IgniteClientDisconnectedCheckedException(reconnectFut,
+            "Operation has been cancelled (client node disconnected).");
     }
 
     /**
@@ -339,6 +360,9 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         IgniteInternalFuture<?> old = atomicFuts.put(futVer, fut);
 
         assert old == null : "Old future is not null [futVer=" + futVer + ", fut=" + fut + ", old=" + old + ']';
+
+        if (cctx.kernalContext().clientDisconnected())
+            ((GridFutureAdapter)fut).onDone(disconnectedError(null));
     }
 
     /**
@@ -459,7 +483,10 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
                 fut.onNodeLeft(n.id());
         }
 
-        // Just in case if future was complete before it was added.
+        if (cctx.kernalContext().clientDisconnected())
+            ((GridFutureAdapter)fut).onDone(disconnectedError(null));
+
+        // Just in case if future was completed before it was added.
         if (fut.isDone())
             removeFuture(fut);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 4398b4c..1f6a8bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -97,6 +97,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     private final AtomicReference<AffinityTopologyVersion> readyTopVer =
         new AtomicReference<>(AffinityTopologyVersion.NONE);
 
+    /** */
+    private GridFutureAdapter<?> reconnectExchangeFut;
+
     /**
      * Partition map futures.
      * This set also contains already completed exchange futures to address race conditions when coordinator
@@ -237,9 +240,16 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             });
     }
 
+    /**
+     * @return Reconnect partition exchange future.
+     */
+    public IgniteInternalFuture<?> reconnectExchangeFuture() {
+        return reconnectExchangeFut;
+    }
+
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws IgniteCheckedException {
-        super.onKernalStart0();
+    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
+        super.onKernalStart0(reconnect);
 
         ClusterNode loc = cctx.localNode();
 
@@ -260,6 +270,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         GridDhtPartitionsExchangeFuture fut = exchangeFuture(exchId, discoEvt, null);
 
+        if (reconnect)
+            reconnectExchangeFut = new GridFutureAdapter<>();
+
         new IgniteThread(cctx.gridName(), "exchange-worker", exchWorker).start();
 
         onDiscoveryEvent(cctx.localNodeId(), fut);
@@ -267,10 +280,30 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         // Allow discovery events to get processed.
         locExchFut.onDone();
 
-        if (log.isDebugEnabled())
-            log.debug("Beginning to wait on local exchange future: " + fut);
+        if (reconnect) {
+            fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> fut) {
+                    try {
+                        fut.get();
+
+                        for (GridCacheContext cacheCtx : cctx.cacheContexts())
+                            cacheCtx.preloader().onInitialExchangeComplete(null);
+
+                        reconnectExchangeFut.onDone();
+                    }
+                    catch (IgniteCheckedException e) {
+                        for (GridCacheContext cacheCtx : cctx.cacheContexts())
+                            cacheCtx.preloader().onInitialExchangeComplete(e);
+
+                        reconnectExchangeFut.onDone(e);
+                    }
+                }
+            });
+        }
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Beginning to wait on local exchange future: " + fut);
 
-        try {
             boolean first = true;
 
             while (true) {
@@ -296,28 +329,35 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
             for (GridCacheContext cacheCtx : cctx.cacheContexts())
                 cacheCtx.preloader().onInitialExchangeComplete(null);
-        }
-        catch (IgniteFutureTimeoutCheckedException e) {
-            IgniteCheckedException err = new IgniteCheckedException("Timed out waiting for exchange future: " + fut, e);
 
-            for (GridCacheContext cacheCtx : cctx.cacheContexts())
-                cacheCtx.preloader().onInitialExchangeComplete(err);
-
-            throw err;
+            if (log.isDebugEnabled())
+                log.debug("Finished waiting for initial exchange: " + fut.exchangeId());
         }
-
-        if (log.isDebugEnabled())
-            log.debug("Finished waiting on local exchange: " + fut.exchangeId());
     }
 
     /** {@inheritDoc} */
     @Override protected void onKernalStop0(boolean cancel) {
+        cctx.gridEvents().removeLocalEventListener(discoLsnr);
+
+        cctx.io().removeHandler(0, GridDhtPartitionsSingleMessage.class);
+        cctx.io().removeHandler(0, GridDhtPartitionsFullMessage.class);
+        cctx.io().removeHandler(0, GridDhtPartitionsSingleRequest.class);
+
+        IgniteCheckedException err = cctx.kernalContext().clientDisconnected() ?
+            new IgniteClientDisconnectedCheckedException(cctx.kernalContext().cluster().clientReconnectFuture(),
+                "Client node disconnected: " + cctx.gridName()) :
+            new IgniteInterruptedCheckedException("Node is stopping: " + cctx.gridName());
+
         // Finish all exchange futures.
-        for (GridDhtPartitionsExchangeFuture f : exchFuts.values())
-            f.onDone(new IgniteInterruptedCheckedException("Grid is stopping: " + cctx.gridName()));
+        ExchangeFutureSet exchFuts0 = exchFuts;
+
+        if (exchFuts0 != null) {
+            for (GridDhtPartitionsExchangeFuture f : exchFuts.values())
+                f.onDone(err);
+        }
 
         for (AffinityReadyFuture f : readyFuts.values())
-            f.onDone(new IgniteInterruptedCheckedException("Grid is stopping: " + cctx.gridName()));
+            f.onDone(err);
 
         U.cancel(exchWorker);
 
@@ -634,7 +674,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         node.id() + ", msg=" + m + ']');
             }
             catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send partitions full message [node=" + node + ']', e);
+                U.warn(log, "Failed to send partitions full message [node=" + node + ", err=" + e + ']');
             }
         }
 
@@ -1097,6 +1137,9 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 catch (IgniteInterruptedCheckedException e) {
                     throw e;
                 }
+                catch (IgniteClientDisconnectedCheckedException e) {
+                    return;
+                }
                 catch (IgniteCheckedException e) {
                     U.error(log, "Failed to wait for completion of partition map exchange " +
                         "(preloading will not start): " + exchFut, e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index e0f6181..b8bb08e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -56,6 +56,11 @@ public interface GridCachePreloader {
     public void onKernalStop();
 
     /**
+     * Client reconnected callback.
+     */
+    public void onReconnected();
+
+    /**
      * Callback by exchange manager when initial partition exchange is complete.
      *
      * @param err Error, if any happened on initial exchange.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index b4f386f..0adf510 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -87,6 +87,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
+    @Override public void onReconnected() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void preloadPredicate(IgnitePredicate<GridCacheEntryInfo> preloadPred) {
         this.preloadPred = preloadPred;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index bb87a86..bda0485 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -127,6 +127,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** Count down latch for caches. */
     private final CountDownLatch cacheStartedLatch = new CountDownLatch(1);
 
+    /** */
+    private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
+
     /**
      * @param ctx Kernal context.
      */
@@ -704,7 +707,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                 });
 
                 for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
-                    mgr.onKernalStart();
+                    mgr.onKernalStart(false);
 
                 for (GridCacheAdapter<?, ?> cache : caches.values())
                     onKernalStart(cache);
@@ -796,7 +799,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         // Must call onKernalStart on shared managers after creation of fetched caches.
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
-            mgr.onKernalStart();
+            mgr.onKernalStart(false);
 
         for (Map.Entry<String, GridCacheAdapter<?, ?>> e : caches.entrySet()) {
             GridCacheAdapter cache = e.getValue();
@@ -911,6 +914,90 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        cachesOnDisconnect = new HashMap<>(registeredCaches);
+
+        IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
+            ctx.cluster().clientReconnectFuture(),
+            "Failed to execute dynamic cache change request, client node disconnected.");
+
+        for (IgniteInternalFuture fut : pendingFuts.values())
+            ((GridFutureAdapter)fut).onDone(err);
+
+        for (IgniteInternalFuture fut : pendingTemplateFuts.values())
+            ((GridFutureAdapter)fut).onDone(err);
+
+        for (GridCacheAdapter cache : caches.values()) {
+            GridCacheContext cctx = cache.context();
+
+            cctx.gate().onDisconnected(reconnectFut);
+
+            List<GridCacheManager> mgrs = cache.context().managers();
+
+            for (ListIterator<GridCacheManager> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
+                GridCacheManager mgr = it.previous();
+
+                mgr.onDisconnected(reconnectFut);
+            }
+        }
+
+        sharedCtx.onDisconnected(reconnectFut);
+
+        registeredCaches.clear();
+
+        registeredTemplates.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        List<GridCacheAdapter> reconnected = new ArrayList<>(caches.size());
+
+        for (GridCacheAdapter cache : caches.values()) {
+            String name = cache.name();
+
+            boolean stopped;
+
+            boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+
+            if (!sysCache) {
+                DynamicCacheDescriptor oldDesc = cachesOnDisconnect.get(maskNull(name));
+
+                assert oldDesc != null : "No descriptor for cache: " + name;
+
+                DynamicCacheDescriptor newDesc = registeredCaches.get(maskNull(name));
+
+                stopped = newDesc == null || !oldDesc.deploymentId().equals(newDesc.deploymentId());
+            }
+            else
+                stopped = false;
+
+            if (stopped) {
+                cache.context().gate().reconnected(true);
+
+                sharedCtx.removeCacheContext(cache.ctx);
+
+                caches.remove(maskNull(cache.name()));
+                jCacheProxies.remove(maskNull(cache.name()));
+
+                onKernalStop(cache, true);
+                stopCache(cache, true);
+            }
+            else {
+                cache.onReconnected();
+
+                reconnected.add(cache);
+            }
+        }
+
+        sharedCtx.onReconnected();
+
+        for (GridCacheAdapter cache : reconnected)
+            cache.context().gate().reconnected(false);
+
+        cachesOnDisconnect = null;
+    }
+
     /**
      * @param cache Cache to start.
      * @throws IgniteCheckedException If failed to start cache.
@@ -1529,7 +1616,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             if (proxy != null) {
                 if (req.stop())
-                    proxy.gate().block();
+                    proxy.gate().stopped();
                 else
                     proxy.closeProxy();
             }
@@ -1673,8 +1760,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             depMgr,
             exchMgr,
             ioMgr,
-            storeSesLsnrs,
-            jta
+            jta,
+            storeSesLsnrs
         );
     }
 
@@ -1689,7 +1776,11 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         Collection<DynamicCacheChangeRequest> reqs =
             new ArrayList<>(registeredCaches.size() + registeredTemplates.size());
 
-        for (DynamicCacheDescriptor desc : registeredCaches.values()) {
+        boolean reconnect = ctx.localNodeId().equals(nodeId) && cachesOnDisconnect != null;
+
+        Map<String, DynamicCacheDescriptor> descs = reconnect ? cachesOnDisconnect : registeredCaches;
+
+        for (DynamicCacheDescriptor desc : descs.values()) {
             if (!desc.cancelled()) {
                 DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(desc.cacheConfiguration().getName(), null);
 
@@ -1717,7 +1808,25 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
         DynamicCacheChangeBatch req = new DynamicCacheChangeBatch(reqs);
 
-        req.clientNodes(ctx.discovery().clientNodesMap());
+        Map<String, Map<UUID, Boolean>> clientNodesMap = ctx.discovery().clientNodesMap();
+
+        if (reconnect) {
+            clientNodesMap = U.newHashMap(caches.size());
+
+            for (GridCacheAdapter<?, ?> cache : caches.values()) {
+                Boolean nearEnabled = cache.isNear();
+
+                Map<UUID, Boolean> map = U.newHashMap(1);
+
+                map.put(nodeId, nearEnabled);
+
+                clientNodesMap.put(cache.name(), map);
+            }
+        }
+
+        req.clientNodes(clientNodesMap);
+
+        req.clientReconnect(reconnect);
 
         return req;
     }
@@ -1727,38 +1836,86 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (data instanceof DynamicCacheChangeBatch) {
             DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)data;
 
-            for (DynamicCacheChangeRequest req : batch.requests()) {
-                if (req.template()) {
-                    CacheConfiguration ccfg = req.startCacheConfiguration();
+            if (batch.clientReconnect()) {
+                for (DynamicCacheChangeRequest req : batch.requests()) {
+                    assert !req.template() : req;
 
-                    assert ccfg != null : req;
+                    String name = req.cacheName();
 
-                    DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
+                    boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
 
-                    if (existing == null) {
-                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
-                            ctx,
-                            ccfg,
-                            req.cacheType(),
-                            true,
-                            req.deploymentId());
+                    if (!sysCache) {
+                        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
 
-                        registeredTemplates.put(maskNull(req.cacheName()), desc);
-                    }
+                        if (desc != null && !desc.cancelled() && desc.deploymentId().equals(req.deploymentId())) {
+                            Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
 
-                    continue;
+                            assert nodes != null : req;
+                            assert nodes.containsKey(joiningNodeId) : nodes;
+
+                            ctx.discovery().addClientNode(req.cacheName(), joiningNodeId, nodes.get(joiningNodeId));
+                        }
+                    }
+                    else
+                        ctx.discovery().addClientNode(req.cacheName(), joiningNodeId, false);
                 }
+            }
+            else {
+                for (DynamicCacheChangeRequest req : batch.requests()) {
+                    if (req.template()) {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
 
-                DynamicCacheDescriptor existing = registeredCaches.get(maskNull(req.cacheName()));
+                        assert ccfg != null : req;
 
-                if (req.start() && !req.clientStartOnly()) {
-                    CacheConfiguration ccfg = req.startCacheConfiguration();
+                        DynamicCacheDescriptor existing = registeredTemplates.get(maskNull(req.cacheName()));
 
-                    if (existing != null) {
-                        if (existing.locallyConfigured()) {
-                            existing.deploymentId(req.deploymentId());
+                        if (existing == null) {
+                            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                                ctx,
+                                ccfg,
+                                req.cacheType(),
+                                true,
+                                req.deploymentId());
+
+                            registeredTemplates.put(maskNull(req.cacheName()), desc);
+                        }
+
+                        continue;
+                    }
 
-                            existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration());
+                    DynamicCacheDescriptor existing = registeredCaches.get(maskNull(req.cacheName()));
+
+                    if (req.start() && !req.clientStartOnly()) {
+                        CacheConfiguration ccfg = req.startCacheConfiguration();
+
+                        if (existing != null) {
+                            if (existing.locallyConfigured()) {
+                                existing.deploymentId(req.deploymentId());
+
+                                existing.addRemoteConfiguration(rmtNodeId, req.startCacheConfiguration());
+
+                                ctx.discovery().setCacheFilter(
+                                    req.cacheName(),
+                                    ccfg.getNodeFilter(),
+                                    ccfg.getNearConfiguration() != null,
+                                    ccfg.getCacheMode() == LOCAL);
+                            }
+                        }
+                        else {
+                            assert req.cacheType() != null : req;
+
+                            DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
+                                ctx,
+                                ccfg,
+                                req.cacheType(),
+                                false,
+                                req.deploymentId());
+
+                            // Received statically configured cache.
+                            if (req.initiatingNodeId() == null)
+                                desc.staticallyConfigured(true);
+
+                            registeredCaches.put(maskNull(req.cacheName()), desc);
 
                             ctx.discovery().setCacheFilter(
                                 req.cacheName(),
@@ -1767,37 +1924,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                                 ccfg.getCacheMode() == LOCAL);
                         }
                     }
-                    else {
-                        assert req.cacheType() != null : req;
-
-                        DynamicCacheDescriptor desc = new DynamicCacheDescriptor(
-                            ctx,
-                            ccfg,
-                            req.cacheType(),
-                            false,
-                            req.deploymentId());
-
-                        // Received statically configured cache.
-                        if (req.initiatingNodeId() == null)
-                            desc.staticallyConfigured(true);
-
-                        registeredCaches.put(maskNull(req.cacheName()), desc);
-
-                        ctx.discovery().setCacheFilter(
-                            req.cacheName(),
-                            ccfg.getNodeFilter(),
-                            ccfg.getNearConfiguration() != null,
-                            ccfg.getCacheMode() == LOCAL);
-                    }
                 }
-            }
 
-            if (!F.isEmpty(batch.clientNodes())) {
-                for (Map.Entry<String, Map<UUID, Boolean>> entry : batch.clientNodes().entrySet()) {
-                    String cacheName = entry.getKey();
+                if (!F.isEmpty(batch.clientNodes())) {
+                    for (Map.Entry<String, Map<UUID, Boolean>> entry : batch.clientNodes().entrySet()) {
+                        String cacheName = entry.getKey();
 
-                    for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
-                        ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
+                        for (Map.Entry<UUID, Boolean> tup : entry.getValue().entrySet())
+                            ctx.discovery().addClientNode(cacheName, tup.getKey(), tup.getValue());
+                    }
                 }
             }
         }
@@ -2152,8 +2287,25 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         }
 
-        if (!sndReqs.isEmpty())
-            ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs));
+        Exception err = null;
+
+        if (!sndReqs.isEmpty()) {
+            try {
+                ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(sndReqs));
+
+                if (ctx.clientDisconnected())
+                    err = new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                        "Failed to execute dynamic cache change request, client node disconnected.");
+            }
+            catch (IgniteCheckedException e) {
+                err = e;
+            }
+        }
+
+        if (err != null) {
+            for (DynamicCacheStartFuture fut : res)
+                fut.onDone(err);
+        }
 
         return res;
     }
@@ -2666,8 +2818,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      *
      * @return Utility cache.
      */
-    public <K, V> GridCacheAdapter<K, V> utilityCache() {
-        return internalCache(CU.UTILITY_CACHE_NAME);
+    public <K, V> IgniteInternalCache<K, V> utilityCache() {
+        return internalCacheEx(CU.UTILITY_CACHE_NAME);
     }
 
     /**
@@ -2676,7 +2828,23 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @return Utility cache for atomic data structures.
      */
     public <K, V> IgniteInternalCache<K, V> atomicsCache() {
-        return cache(CU.ATOMICS_CACHE_NAME);
+        return internalCacheEx(CU.ATOMICS_CACHE_NAME);
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache.
+     */
+    private <K, V> IgniteInternalCache<K, V> internalCacheEx(String name) {
+        if (ctx.discovery().localNode().isClient()) {
+            IgniteCacheProxy<K, V> proxy = (IgniteCacheProxy<K, V>)jCacheProxies.get(name);
+
+            assert proxy != null;
+
+            return proxy.internalProxy();
+        }
+
+        return internalCache(name);
     }
 
     /**
@@ -2796,7 +2964,21 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         if (old != null)
             fut = old;
 
-        ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req)));
+        Exception err = null;
+
+        try {
+            ctx.discovery().sendCustomEvent(new DynamicCacheChangeBatch(Collections.singleton(req)));
+
+            if (ctx.clientDisconnected())
+                err = new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                    "Failed to execute dynamic cache change request, client node disconnected.");
+        }
+        catch (IgniteCheckedException e) {
+            err = e;
+        }
+
+        if (err != null)
+            fut.onDone(err);
 
         fut.get();
     }
@@ -2856,8 +3038,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * Cancel all user operations.
      */
     public void cancelUserOperations() {
-        for (GridCacheAdapter<?, ?> cache : caches.values())
-            cache.ctx.mvcc().cancelClientFutures();
+        sharedCtx.mvcc().cancelClientFutures();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 7f4daff..4075d79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.jetbrains.annotations.*;
 
@@ -86,9 +87,15 @@ public class GridCacheSharedContext<K, V> {
     private Collection<CacheStoreSessionListener> storeSesLsnrs;
 
     /**
+     * @param kernalCtx  Context.
      * @param txMgr Transaction manager.
      * @param verMgr Version manager.
      * @param mvccMgr MVCC manager.
+     * @param depMgr Deployment manager.
+     * @param exchMgr Exchange manager.
+     * @param ioMgr IO manager.
+     * @param jtaMgr JTA manager.
+     * @param storeSesLsnrs Store session listeners.
      */
     public GridCacheSharedContext(
         GridKernalContext kernalCtx,
@@ -98,17 +105,13 @@ public class GridCacheSharedContext<K, V> {
         GridCacheDeploymentManager<K, V> depMgr,
         GridCachePartitionExchangeManager<K, V> exchMgr,
         GridCacheIoManager ioMgr,
-        Collection<CacheStoreSessionListener> storeSesLsnrs,
-        CacheJtaManagerAdapter jtaMgr
+        CacheJtaManagerAdapter jtaMgr,
+        Collection<CacheStoreSessionListener> storeSesLsnrs
     ) {
         this.kernalCtx = kernalCtx;
-        this.mvccMgr = add(mvccMgr);
-        this.verMgr = add(verMgr);
-        this.txMgr = add(txMgr);
-        this.jtaMgr = add(jtaMgr);
-        this.depMgr = add(depMgr);
-        this.exchMgr = add(exchMgr);
-        this.ioMgr = add(ioMgr);
+
+        setManagers(mgrs, txMgr, jtaMgr, verMgr, mvccMgr, depMgr, exchMgr, ioMgr);
+
         this.storeSesLsnrs = storeSesLsnrs;
 
         txMetrics = new TransactionMetricsAdapter();
@@ -117,6 +120,89 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @param reconnectFut Reconnect future.
+     * @throws IgniteCheckedException If failed.
+     */
+    void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = mgrs.listIterator(mgrs.size());
+            it.hasPrevious();) {
+            GridCacheSharedManager<?, ?> mgr = it.previous();
+
+            mgr.onDisconnected(reconnectFut);
+
+            if (restartOnDisconnect(mgr))
+                mgr.onKernalStop(true);
+        }
+
+        for (ListIterator<? extends GridCacheSharedManager<?, ?>> it = mgrs.listIterator(mgrs.size()); it.hasPrevious();) {
+            GridCacheSharedManager<?, ?> mgr = it.previous();
+
+            if (restartOnDisconnect(mgr))
+                mgr.stop(true);
+        }
+    }
+
+    /**
+     * @throws IgniteCheckedException If failed.
+     */
+    void onReconnected() throws IgniteCheckedException {
+        List<GridCacheSharedManager<K, V>> mgrs = new LinkedList<>();
+
+        setManagers(mgrs, txMgr,
+            jtaMgr,
+            verMgr,
+            mvccMgr,
+            new GridCacheDeploymentManager<K, V>(),
+            new GridCachePartitionExchangeManager<K, V>(),
+            ioMgr);
+
+        this.mgrs = mgrs;
+
+        for (GridCacheSharedManager<K, V> mgr : mgrs) {
+            if (restartOnDisconnect(mgr))
+                mgr.start(this);
+        }
+
+        for (GridCacheSharedManager<?, ?> mgr : mgrs)
+            mgr.onKernalStart(true);
+    }
+
+    /**
+     * @param mgr Manager.
+     * @return {@code True} if manager is restarted cn reconnect.
+     */
+    private boolean restartOnDisconnect(GridCacheSharedManager<?, ?> mgr) {
+        return mgr instanceof GridCacheDeploymentManager || mgr instanceof GridCachePartitionExchangeManager;
+    }
+
+    /**
+     * @param mgrs Managers list.
+     * @param txMgr Transaction manager.
+     * @param verMgr Version manager.
+     * @param mvccMgr MVCC manager.
+     * @param depMgr Deployment manager.
+     * @param exchMgr Exchange manager.
+     * @param ioMgr IO manager.
+     * @param jtaMgr JTA manager.
+     */
+    private void setManagers(List<GridCacheSharedManager<K, V>> mgrs,
+        IgniteTxManager txMgr,
+        CacheJtaManagerAdapter jtaMgr,
+        GridCacheVersionManager verMgr,
+        GridCacheMvccManager mvccMgr,
+        GridCacheDeploymentManager<K, V> depMgr,
+        GridCachePartitionExchangeManager<K, V> exchMgr,
+        GridCacheIoManager ioMgr) {
+        this.mvccMgr = add(mgrs, mvccMgr);
+        this.verMgr = add(mgrs, verMgr);
+        this.txMgr = add(mgrs, txMgr);
+        this.jtaMgr = add(mgrs, jtaMgr);
+        this.depMgr = add(mgrs, depMgr);
+        this.exchMgr = add(mgrs, exchMgr);
+        this.ioMgr = add(mgrs, ioMgr);
+    }
+
+    /**
      * Gets all cache contexts for local node.
      *
      * @return Collection of all cache contexts.
@@ -136,6 +222,7 @@ public class GridCacheSharedContext<K, V> {
      * Adds cache context to shared cache context.
      *
      * @param cacheCtx Cache context to add.
+     * @throws IgniteCheckedException If cache ID conflict detected.
      */
     @SuppressWarnings("unchecked")
     public void addCacheContext(GridCacheContext cacheCtx) throws IgniteCheckedException {
@@ -230,7 +317,7 @@ public class GridCacheSharedContext<K, V> {
      */
     public byte dataCenterId() {
         // Data center ID is same for all caches, so grab the first one.
-        GridCacheContext<K, V> cacheCtx = F.first(cacheContexts());
+        GridCacheContext<?, ?> cacheCtx = F.first(cacheContexts());
 
         return cacheCtx.dataCenterId();
     }
@@ -242,7 +329,7 @@ public class GridCacheSharedContext<K, V> {
         if (preloadersStartFut == null) {
             GridCompoundFuture<Object, Object> compound = null;
 
-            for (GridCacheContext<K, V> cacheCtx : cacheContexts()) {
+            for (GridCacheContext<?, ?> cacheCtx : cacheContexts()) {
                 IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
 
                 if (!startFut.isDone()) {
@@ -551,10 +638,12 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @param mgrs Managers list.
      * @param mgr Manager to add.
      * @return Added manager.
      */
-    @Nullable private <T extends GridCacheSharedManager<K, V>> T add(@Nullable T mgr) {
+    @Nullable private <T extends GridCacheSharedManager<K, V>> T add(List<GridCacheSharedManager<K, V>> mgrs,
+        @Nullable T mgr) {
         if (mgr != null)
             mgrs.add(mgr);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
index d45052c..668bd00 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManager.java
@@ -18,11 +18,12 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
 
 /**
  * Cache manager shared across all caches.
  */
-public interface GridCacheSharedManager <K, V> {
+public interface GridCacheSharedManager<K, V> {
     /**
      * Starts manager.
      *
@@ -39,9 +40,10 @@ public interface GridCacheSharedManager <K, V> {
     public void stop(boolean cancel);
 
     /**
+     * @param reconnect {@code True} if manager restarted after client reconnect.
      * @throws IgniteCheckedException If failed.
      */
-    public void onKernalStart() throws IgniteCheckedException;
+    public void onKernalStart(boolean reconnect) throws IgniteCheckedException;
 
     /**
      * @param cancel Cancel flag.
@@ -49,6 +51,11 @@ public interface GridCacheSharedManager <K, V> {
     public void onKernalStop(boolean cancel);
 
     /**
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut);
+
+    /**
      * Prints memory statistics (sizes of internal data structures, etc.).
      *
      * NOTE: this method is for testing and profiling purposes only.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
index 2cf7051..6ad76ca 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedManagerAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 
 import java.util.concurrent.atomic.*;
 
@@ -35,6 +36,9 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     /** Starting flag. */
     private final AtomicBoolean starting = new AtomicBoolean(false);
 
+    /** */
+    private final AtomicBoolean stop = new AtomicBoolean(false);
+
     /** {@inheritDoc} */
     @Override public final void start(GridCacheSharedContext<K, V> cctx) throws IgniteCheckedException {
         if (!starting.compareAndSet(false, true))
@@ -75,7 +79,7 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
 
     /** {@inheritDoc} */
     @Override public final void stop(boolean cancel) {
-        if (!starting.get())
+        if (!starting.get() || !stop.compareAndSet(false, true))
             // Ignoring attempt to stop manager that has never been started.
             return;
 
@@ -93,10 +97,10 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     }
 
     /** {@inheritDoc} */
-    @Override public final void onKernalStart() throws IgniteCheckedException {
-        onKernalStart0();
+    @Override public final void onKernalStart(boolean reconnect) throws IgniteCheckedException {
+        onKernalStart0(reconnect);
 
-        if (log != null && log.isDebugEnabled())
+        if (!reconnect && log != null && log.isDebugEnabled())
             log.debug(kernalStartInfo());
     }
 
@@ -113,9 +117,10 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     }
 
     /**
+     * @param reconnect {@code True} if manager restarted after client reconnect.
      * @throws IgniteCheckedException If failed.
      */
-    protected void onKernalStart0() throws IgniteCheckedException {
+    protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
         // No-op.
     }
 
@@ -127,6 +132,11 @@ public class GridCacheSharedManagerAdapter<K, V> implements GridCacheSharedManag
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index f88e288..bd2623d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -1560,6 +1560,17 @@ public class GridCacheUtils {
      * @return CacheException runtime exception, never null.
      */
     @NotNull public static RuntimeException convertToCacheException(IgniteCheckedException e) {
+        IgniteClientDisconnectedCheckedException disconnectedErr =
+            e instanceof IgniteClientDisconnectedCheckedException ?
+            (IgniteClientDisconnectedCheckedException)e
+            : e.getCause(IgniteClientDisconnectedCheckedException.class);
+
+        if (disconnectedErr != null) {
+            assert disconnectedErr.reconnectFuture() != null : disconnectedErr;
+
+            e = disconnectedErr;
+        }
+
         if (e.hasCause(CacheWriterException.class))
             return new CacheWriterException(U.convertExceptionNoWrap(e));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
index 42e31d2..9233f24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheFutureImpl.java
@@ -43,6 +43,11 @@ public class IgniteCacheFutureImpl<V> extends IgniteFutureImpl<V> {
 
     /** {@inheritDoc} */
     @Override protected RuntimeException convertException(IgniteCheckedException e) {
+        if (e instanceof IgniteFutureCancelledCheckedException ||
+            e instanceof IgniteInterruptedCheckedException ||
+            e instanceof IgniteFutureTimeoutCheckedException)
+            return U.convertException(e);
+
         return CU.convertToCacheException(e);
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index bb7714a..0b2eba0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -1751,7 +1751,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
                 assert false;
             }
 
-            @Override public void block() {
+            @Override public void stopped() {
                 // No-op.
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
index b5c5161..78bd0eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/CacheDataStructuresManager.java
@@ -106,6 +106,41 @@ public class CacheDataStructuresManager extends GridCacheManagerAdapter {
     }
 
     /**
+     * @param set Set.
+     */
+    public void onRemoved(GridCacheSetProxy set) {
+        setsMap.remove(set.delegate().id(), set);
+    }
+
+    /**
+     * @param clusterRestarted Cluster restarted flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        for (Map.Entry<IgniteUuid, GridCacheSetProxy> e : setsMap.entrySet()) {
+            GridCacheSetProxy set = e.getValue();
+
+            if (clusterRestarted) {
+                set.blockOnRemove();
+
+                setsMap.remove(e.getKey(), set);
+            }
+            else
+                set.needCheckNotRemoved();
+        }
+
+        for (Map.Entry<IgniteUuid, GridCacheQueueProxy> e : queuesMap.entrySet()) {
+            GridCacheQueueProxy queue = e.getValue();
+
+            if (clusterRestarted) {
+                queue.delegate().onRemoved(false);
+
+                queuesMap.remove(e.getKey(), queue);
+            }
+        }
+    }
+
+    /**
      * @throws IgniteCheckedException If thread is interrupted or manager
      *     was not successfully initialized.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
index 2838838..0b351b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
@@ -22,6 +22,7 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
@@ -81,6 +82,16 @@ public class GridCacheTxFinishSync<K, V> {
     }
 
     /**
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut) {
+       for (ThreadFinishSync threadSync : threadMap.values())
+            threadSync.onDisconnected(reconnectFut);
+
+        threadMap.clear();
+    }
+
+    /**
      * Callback invoked when finish response is received from remote node.
      *
      * @param nodeId Node ID response was received from.
@@ -139,6 +150,11 @@ public class GridCacheTxFinishSync<K, V> {
 
                     nodeMap.remove(nodeId);
                 }
+                else if (cctx.kernalContext().clientDisconnected()) {
+                    sync.onDisconnected(cctx.kernalContext().cluster().clientReconnectFuture());
+
+                    nodeMap.remove(nodeId);
+                }
             }
 
             sync.onSend();
@@ -160,6 +176,16 @@ public class GridCacheTxFinishSync<K, V> {
         }
 
         /**
+         * @param reconnectFut Reconnect future.
+         */
+        public void onDisconnected(IgniteFuture<?> reconnectFut) {
+            for (TxFinishSync sync : nodeMap.values())
+                sync.onDisconnected(reconnectFut);
+
+            nodeMap.clear();
+        }
+
+        /**
          * @param nodeId Node ID response received from.
          */
         public void onReceive(UUID nodeId) {
@@ -288,5 +314,25 @@ public class GridCacheTxFinishSync<K, V> {
                 }
             }
         }
+
+        /**
+         * Client disconnected callback.
+         *
+         * @param reconnectFut Reconnect future.
+         */
+        public void onDisconnected(IgniteFuture<?> reconnectFut) {
+            synchronized (this) {
+                nodeLeft = true;
+
+                if (pendingFut != null) {
+                    IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
+                        reconnectFut,
+                        "Failed to wait for transaction synchronizer, client node disconnected: " + nodeId);
+                    pendingFut.onDone(err);
+
+                    pendingFut = null;
+                }
+            }
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index adea9e0..22a5287 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -51,7 +51,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     private static final long serialVersionUID = 0L;
 
     /** Topology. */
-    private GridDhtPartitionTopology top;
+    private GridDhtPartitionTopologyImpl top;
 
     /** Preloader. */
     protected GridCachePreloader preldr;
@@ -134,6 +134,18 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     }
 
     /** {@inheritDoc} */
+    @Override public void onReconnected() {
+        super.onReconnected();
+
+        ctx.affinity().onReconnected();
+
+        top.onReconnected();
+
+        if (preldr != null)
+            preldr.onReconnected();
+    }
+
+    /** {@inheritDoc} */
     @Override public void onKernalStart() throws IgniteCheckedException {
         super.onKernalStart();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index de7f876..facf329 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -94,6 +94,30 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /**
+     *
+     */
+    public void onReconnected() {
+        lock.writeLock().lock();
+
+        try {
+            node2part = null;
+
+            part2node = new HashMap<>();
+
+            lastExchangeId = null;
+
+            updateSeq.set(1);
+
+            topReadyFut = null;
+
+            topVer = AffinityTopologyVersion.NONE;
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
      * @return Full map string representation.
      */
     @SuppressWarnings( {"ConstantConditions"})

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 79d5e75..bb3673d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -639,10 +639,17 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                         if (timeout.finish()) {
                             cctx.kernalContext().timeout().removeTimeoutObject(timeout);
 
-                            // Remap.
-                            map(keys.keySet(), F.t(node, keys), updTopVer);
+                            try {
+                                fut.get();
 
-                            onDone(Collections.<K, V>emptyMap());
+                                // Remap.
+                                map(keys.keySet(), F.t(node, keys), updTopVer);
+
+                                onDone(Collections.<K, V>emptyMap());
+                            }
+                            catch (IgniteCheckedException e) {
+                                GridPartitionedGetFuture.this.onDone(e);
+                            }
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 0355bb3..a43ebe2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -67,7 +67,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     private GridDhtPartitionDemandPool demandPool;
 
     /** Start future. */
-    private final GridFutureAdapter<Object> startFut;
+    private GridFutureAdapter<Object> startFut;
 
     /** Busy lock to prevent activities from accessing exchanger while it's stopping. */
     private final ReadWriteLock busyLock = new ReentrantReadWriteLock();
@@ -180,13 +180,6 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
         topVer.setIfGreater(startTopVer);
 
-        // Generate dummy discovery event for local node joining.
-        DiscoveryEvent discoEvt = cctx.discovery().localJoinEvent();
-
-        assert discoEvt != null;
-
-        assert discoEvt.topologyVersion() == startTopVer;
-
         supplyPool.start();
         demandPool.start();
     }
@@ -230,7 +223,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
             final CacheConfiguration cfg = cctx.config();
 
-            if (cfg.getRebalanceDelay() >= 0) {
+            if (cfg.getRebalanceDelay() >= 0 && !cctx.kernalContext().clientNode()) {
                 U.log(log, "Starting rebalancing in " + cfg.getRebalanceMode() + " mode: " + cctx.name());
 
                 demandPool.syncFuture().listen(new CI1<Object>() {
@@ -246,6 +239,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public void onReconnected() {
+        startFut = new GridFutureAdapter<>();
+    }
+
+    /** {@inheritDoc} */
     @Override public void onExchangeFutureAdded() {
         demandPool.onExchangeFutureAdded();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 351d6cd..79b7c1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -90,6 +90,14 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     public abstract GridDhtCacheAdapter<K, V> dht();
 
     /** {@inheritDoc} */
+    @Override public void onReconnected() {
+        map = new GridCacheConcurrentMap(ctx,
+            ctx.config().getNearConfiguration().getNearStartSize(),
+            0.75F,
+            map.getEntryFactory());
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean isNear() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 58f6fe5..0691d39 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -764,10 +764,17 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                         if (timeout.finish()) {
                             cctx.kernalContext().timeout().removeTimeoutObject(timeout);
 
-                            // Remap.
-                            map(keys.keySet(), F.t(node, keys), updTopVer);
+                            try {
+                                fut.get();
 
-                            onDone(Collections.<K, V>emptyMap());
+                                // Remap.
+                                map(keys.keySet(), F.t(node, keys), updTopVer);
+
+                                onDone(Collections.<K, V>emptyMap());
+                            }
+                            catch (IgniteCheckedException e) {
+                                GridNearGetFuture.this.onDone(e);
+                            }
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
index 00ed020..7f0a568 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridOsCacheDrManager.java
@@ -20,9 +20,9 @@ package org.apache.ignite.internal.processors.cache.dr;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.cache.dr.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.dr.*;
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -103,4 +103,9 @@ public class GridOsCacheDrManager implements GridCacheDrManager {
     @Override public boolean receiveEnabled() {
         return false;
     }
+
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture reconnectFut) {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index 2b93144..316713f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -110,6 +110,20 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) {
+        IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
+            "Query was cancelled, client node disconnected.");
+
+        for (Map.Entry<Long, GridCacheDistributedQueryFuture<?, ?, ?>> e : futs.entrySet()) {
+            GridCacheDistributedQueryFuture<?, ?, ?> fut = e.getValue();
+
+            fut.onPage(null, null, err, true);
+
+            futs.remove(e.getKey(), fut);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats() {
         super.printMemoryStats();
 
@@ -125,6 +139,14 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
      */
     protected void addQueryFuture(long reqId, GridCacheDistributedQueryFuture<?, ?, ?> fut) {
         futs.put(reqId, fut);
+
+        if (cctx.kernalContext().clientDisconnected()) {
+            IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(
+                cctx.kernalContext().cluster().clientReconnectFuture(),
+                "Query was cancelled, client node disconnected.");
+
+            fut.onDone(err);
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index c2425f0..953cb9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -578,6 +578,7 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
 
         /**
          * @param nodes Nodes.
+         * @return Nodes for query execution.
          */
         private Queue<ClusterNode> fallbacks(Collection<ClusterNode> nodes) {
             Queue<ClusterNode> fallbacks = new LinkedList<>();
@@ -595,18 +596,22 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         /**
          *
          */
+        @SuppressWarnings("unchecked")
         private void init() {
             ClusterNode node = nodes.poll();
 
-            GridCacheQueryFutureAdapter<?, ?, R> fut0 =
-                (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ? qryMgr.queryLocal(bean) :
-                    qryMgr.queryDistributed(bean, Collections.singleton(node)));
+            GridCacheQueryFutureAdapter<?, ?, R> fut0 = (GridCacheQueryFutureAdapter<?, ?, R>)(node.isLocal() ?
+                qryMgr.queryLocal(bean) :
+                qryMgr.queryDistributed(bean, Collections.singleton(node)));
 
             fut0.listen(new IgniteInClosure<IgniteInternalFuture<Collection<R>>>() {
                 @Override public void apply(IgniteInternalFuture<Collection<R>> fut) {
                     try {
                         onDone(fut.get());
                     }
+                    catch (IgniteClientDisconnectedCheckedException e) {
+                        onDone(e);
+                    }
                     catch (IgniteCheckedException e) {
                         if (F.isEmpty(nodes))
                             onDone(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index a8bace0..53017c9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -163,7 +163,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
             return null;
         }
         catch (IgniteCheckedException e) {
-            throw new IgniteException(e);
+            throw CU.convertToCacheException(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
index e059760..879c30c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryHandler.java
@@ -396,6 +396,11 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
     }
 
     /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(CacheContinuousQueryHandler.class, this);
+    }
+
+    /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         U.writeString(out, cacheName);
         out.writeObject(topic);

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
index 99907e4..7d9bcf0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -140,32 +140,39 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
     @SuppressWarnings("unchecked")
     private IgniteInternalTx txStart0(TransactionConcurrency concurrency, TransactionIsolation isolation,
         long timeout, int txSize, @Nullable GridCacheContext sysCacheCtx) {
-        TransactionConfiguration cfg = cctx.gridConfig().getTransactionConfiguration();
-
-        if (!cfg.isTxSerializableEnabled() && isolation == SERIALIZABLE)
-            throw new IllegalArgumentException("SERIALIZABLE isolation level is disabled (to enable change " +
-                "'txSerializableEnabled' configuration property)");
-
-        IgniteInternalTx tx = cctx.tm().userTx(sysCacheCtx);
-
-        if (tx != null)
-            throw new IllegalStateException("Failed to start new transaction " +
-                "(current thread already has a transaction): " + tx);
-
-        tx = cctx.tm().newTx(
-            false,
-            false,
-            sysCacheCtx,
-            concurrency,
-            isolation,
-            timeout,
-            true,
-            txSize
-        );
-
-        assert tx != null;
-
-        return tx;
+        cctx.kernalContext().gateway().readLock();
+
+        try {
+            TransactionConfiguration cfg = cctx.gridConfig().getTransactionConfiguration();
+
+            if (!cfg.isTxSerializableEnabled() && isolation == SERIALIZABLE)
+                throw new IllegalArgumentException("SERIALIZABLE isolation level is disabled (to enable change " +
+                    "'txSerializableEnabled' configuration property)");
+
+            IgniteInternalTx tx = cctx.tm().userTx(sysCacheCtx);
+
+            if (tx != null)
+                throw new IllegalStateException("Failed to start new transaction " +
+                    "(current thread already has a transaction): " + tx);
+
+            tx = cctx.tm().newTx(
+                false,
+                false,
+                sysCacheCtx,
+                concurrency,
+                isolation,
+                timeout,
+                true,
+                txSize
+            );
+
+            assert tx != null;
+
+            return tx;
+        }
+        finally {
+            cctx.kernalContext().gateway().readUnlock();
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index b6c77f6..82543c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -115,7 +115,10 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         new ConcurrentHashMap8<>(5120);
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() {
+    @Override protected void onKernalStart0(boolean reconnect) {
+        if (reconnect)
+            return;
+
         cctx.gridEvents().addLocalEventListener(
             new GridLocalEventListener() {
                 @Override public void onEvent(Event evt) {
@@ -149,6 +152,14 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
         txHandler = new IgniteTxHandler(cctx);
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture reconnectFut) {
+        txFinishSync.onDisconnected(reconnectFut);
+
+        for (Map.Entry<Long, IgniteInternalTx> e : threadMap.entrySet())
+            rollbackTx(e.getValue());
+    }
+
     /**
      * @return TX handler.
      */
@@ -764,11 +775,11 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             throw new IgniteTxTimeoutCheckedException("Transaction timed out: " + this);
         }
 
-        boolean txSerializableEnabled = cctx.txConfig().isTxSerializableEnabled();
+        boolean txSerEnabled = cctx.txConfig().isTxSerializableEnabled();
 
         // Clean up committed transactions queue.
         if (tx.pessimistic() && tx.local()) {
-            if (tx.enforceSerializable() && txSerializableEnabled) {
+            if (tx.enforceSerializable() && txSerEnabled) {
                 for (Iterator<IgniteInternalTx> it = committedQ.iterator(); it.hasNext();) {
                     IgniteInternalTx committedTx = it.next();
 
@@ -784,7 +795,7 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             return;
         }
 
-        if (txSerializableEnabled && tx.optimistic() && tx.enforceSerializable()) {
+        if (txSerEnabled && tx.optimistic() && tx.enforceSerializable()) {
             Set<IgniteTxKey> readSet = tx.readSet();
             Set<IgniteTxKey> writeSet = tx.writeSet();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
index 5099b42..9346e43 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
@@ -89,7 +89,7 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza
         try {
             cctx.kernalContext().gateway().readLock();
         }
-        catch (IllegalStateException e) {
+        catch (IllegalStateException | IgniteClientDisconnectedException e) {
             throw e;
         }
         catch (RuntimeException | Error e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
index c776361..90919c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionManager.java
@@ -78,13 +78,6 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
         }
     };
 
-    /**
-     * @return Pre-generated UUID.
-     */
-    private IgniteUuid uuid() {
-        return IgniteUuid.randomUuid();
-    }
-
     /** {@inheritDoc} */
     @Override public void start0() throws IgniteCheckedException {
         txSerEnabled = cctx.gridConfig().getTransactionConfiguration().isTxSerializableEnabled();
@@ -95,7 +88,7 @@ public class GridCacheVersionManager extends GridCacheSharedManagerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws IgniteCheckedException {
+    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
         for (ClusterNode n : cctx.discovery().remoteNodes())
             onReceived(n.id(), n.metrics().getLastDataVersion());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
index 2920176..3ac44f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
@@ -57,7 +57,7 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
     /** Time coordinator thread. */
     private volatile TimeCoordinator timeCoord;
 
-    /** Time delta history. Constructed on coorinator. */
+    /** Time delta history. Constructed on coordinator. */
     private NavigableMap<GridClockDeltaVersion, GridClockDeltaSnapshot> timeSyncHist =
         new GridBoundedConcurrentOrderedMap<>(MAX_TIME_SYNC_HISTORY);
 
@@ -222,7 +222,7 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
                         minNodeOrder = node.order();
                 }
 
-                ClusterNode locNode = ctx.grid().localNode();
+                ClusterNode locNode = ctx.discovery().localNode();
 
                 if (locNode.order() == minNodeOrder) {
                     if (log.isDebugEnabled())
@@ -295,7 +295,7 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
                     ctx.io().send(n, TOPIC_TIME_SYNC, msg, SYSTEM_POOL);
                 }
                 catch (IgniteCheckedException e) {
-                    if (ctx.discovery().pingNode(n.id()))
+                    if (ctx.discovery().pingNodeNoError(n.id()))
                         U.error(log, "Failed to send time sync snapshot to remote node (did not leave grid?) " +
                             "[nodeId=" + n.id() + ", msg=" + msg + ", err=" + e.getMessage() + ']');
                     else if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
index 0ee00f1..1f5589f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/ClusterProcessor.java
@@ -20,6 +20,8 @@ package org.apache.ignite.internal.processors.cluster;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.lang.*;
 
 /**
  *
@@ -43,4 +45,13 @@ public class ClusterProcessor extends GridProcessorAdapter {
     public IgniteClusterImpl get() {
         return cluster;
     }
+
+    /**
+     * @return Client reconnect future.
+     */
+    public IgniteFuture<?> clientReconnectFuture() {
+        IgniteFuture<?> fut = cluster.clientReconnectFuture();
+
+        return fut != null ? fut : new IgniteFinishedFutureImpl<>();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
index ce9b7c0..79020da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousHandler.java
@@ -33,7 +33,14 @@ public interface GridContinuousHandler extends Externalizable, Cloneable {
      * Listener registration status.
      */
     public enum RegisterStatus {
-        REGISTERED, NOT_REGISTERED, DELAYED
+        /** */
+        REGISTERED,
+
+        /** */
+        NOT_REGISTERED,
+
+        /** */
+        DELAYED
     }
 
     /**


[27/50] [abbrv] ignite git commit: # master minor

Posted by nt...@apache.org.
# master minor


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

Branch: refs/heads/ignite-788-dev
Commit: 9e529128b909cfd8fb74710a3d97fb8eaec5362c
Parents: 57ac2b3
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jul 16 13:26:16 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jul 16 13:26:16 2015 +0300

----------------------------------------------------------------------
 modules/core/src/main/java/org/apache/ignite/IgniteCluster.java | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9e529128/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
index d3ce0e7..969d320 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCluster.java
@@ -329,6 +329,9 @@ public interface IgniteCluster extends ClusterGroup, IgniteAsyncSupport {
     public void resetMetrics();
 
     /**
+     * If local client node disconnected from cluster returns future
+     * that will be completed when client reconnected.
+     *
      * @return Future that will be completed when client reconnected.
      */
     @Nullable public IgniteFuture<?> clientReconnectFuture();


[14/50] [abbrv] ignite git commit: Merge tag '1.3.1'

Posted by nt...@apache.org.
Merge tag '1.3.1'

community version 1.3.1 release (REV c559692d6b3aa96316dc0c9b2874c67179489a87)

Conflicts:
	examples/pom.xml
	modules/aop/pom.xml
	modules/aws/pom.xml
	modules/clients/pom.xml
	modules/cloud/pom.xml
	modules/codegen/pom.xml
	modules/core/pom.xml
	modules/core/src/main/resources/ignite.properties
	modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
	modules/extdata/p2p/pom.xml
	modules/extdata/uri/modules/uri-dependency/pom.xml
	modules/extdata/uri/pom.xml
	modules/gce/pom.xml
	modules/geospatial/pom.xml
	modules/hadoop/pom.xml
	modules/hibernate/pom.xml
	modules/indexing/pom.xml
	modules/jcl/pom.xml
	modules/jta/pom.xml
	modules/kafka/pom.xml
	modules/log4j/pom.xml
	modules/mesos/pom.xml
	modules/rest-http/pom.xml
	modules/scalar-2.10/pom.xml
	modules/scalar/pom.xml
	modules/schedule/pom.xml
	modules/schema-import/pom.xml
	modules/slf4j/pom.xml
	modules/spark-2.10/pom.xml
	modules/spark/pom.xml
	modules/spring/pom.xml
	modules/ssh/pom.xml
	modules/tools/pom.xml
	modules/urideploy/pom.xml
	modules/visor-console-2.10/pom.xml
	modules/visor-console/pom.xml
	modules/visor-plugins/pom.xml
	modules/web/pom.xml
	modules/yardstick/pom.xml
	pom.xml


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

Branch: refs/heads/ignite-788-dev
Commit: 13e55b2469c3fb00bf7f1726e5ade1594d476c72
Parents: 77da728 c559692
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Thu Jul 16 10:14:58 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Thu Jul 16 10:14:58 2015 +0300

----------------------------------------------------------------------
 DEVNOTES.txt                                    |   8 +
 RELEASE_NOTES.txt                               |   2 +
 assembly/LICENSE_FABRIC                         | 317 +++++++++++
 assembly/LICENSE_HADOOP                         | 270 ++++++++++
 assembly/NOTICE_FABRIC                          |  13 +
 assembly/NOTICE_HADOOP                          |  12 +
 assembly/dependencies-fabric.xml                |  13 +
 assembly/dependencies-hadoop.xml                |  12 +
 assembly/dependencies-visor-console.xml         |  20 +-
 assembly/release-base.xml                       |  10 -
 assembly/release-fabric.xml                     |  12 +
 assembly/release-hadoop.xml                     |  12 +
 modules/aop/licenses/aspectj-epl-license.txt    |  69 ---
 modules/apache-license-gen/pom.xml              |  48 ++
 .../src/main/resources/META-INF/licenses.txt.vm |  44 ++
 .../processors/cache/IgniteCacheFutureImpl.java |   6 +
 .../processors/cache/IgniteCacheProxy.java      |  55 +-
 .../util/future/GridFutureChainListener.java    |   4 -
 .../internal/util/future/IgniteFutureImpl.java  |  12 +-
 .../plugin/security/SecuritySubjectType.java    |   3 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  32 ++
 .../geospatial/licenses/jts-lgpl-license.txt    | 165 ------
 .../licenses/hibernate-lgpl-2.1-license.txt     | 174 ------
 modules/indexing/licenses/h2-epl-license.txt    |  69 ---
 modules/mesos/licenses/jetty-epl-license.txt    |  69 ---
 modules/mesos/pom.xml                           |   1 +
 .../apache/ignite/mesos/ClusterProperties.java  |  14 +
 .../apache/ignite/mesos/IgniteScheduler.java    |  26 +-
 modules/rest-http/pom.xml                       |   6 +-
 .../scalar-2.10/licenses/scala-bsd-license.txt  |  18 -
 modules/scalar/licenses/scala-bsd-license.txt   |  18 -
 .../licenses/cron4j-lgpl-2.1-license.txt        | 174 ------
 modules/slf4j/licenses/sl4j-mit-license.txt     |  21 -
 .../spark-2.10/licenses/scala-bsd-license.txt   |  18 -
 modules/spark/licenses/scala-bsd-license.txt    |  18 -
 modules/ssh/licenses/jcraft-revised-bsd.txt     |  28 -
 modules/tools/licenses/jodd-revised-bsd.txt     |  21 -
 .../urideploy/licenses/jtidy-mit-license.txt    |  50 --
 modules/urideploy/pom.xml                       |   6 +-
 .../licenses/jline-bsd-license.txt              |  18 -
 .../licenses/scala-bsd-license.txt              |  18 -
 .../licenses/slf4j-mit-license.txt              |  21 -
 modules/web/pom.xml                             |   6 +-
 modules/yarn/README.txt                         |  28 +
 modules/yarn/licenses/apache-2.0.txt            | 202 +++++++
 modules/yarn/pom.xml                            | 101 ++++
 .../apache/ignite/yarn/ApplicationMaster.java   | 400 ++++++++++++++
 .../apache/ignite/yarn/ClusterProperties.java   | 432 +++++++++++++++
 .../org/apache/ignite/yarn/IgniteContainer.java |  84 +++
 .../org/apache/ignite/yarn/IgniteProvider.java  | 339 ++++++++++++
 .../apache/ignite/yarn/IgniteYarnClient.java    | 178 +++++++
 .../org/apache/ignite/yarn/package-info.java    |  22 +
 .../ignite/yarn/utils/IgniteYarnUtils.java      |  81 +++
 .../main/resources/ignite-default-config.xml    |  35 ++
 .../org/apache/ignite/IgniteYarnTestSuite.java  |  38 ++
 .../yarn/IgniteApplicationMasterSelfTest.java   | 532 +++++++++++++++++++
 parent/pom.xml                                  |  55 ++
 pom.xml                                         |   2 +
 58 files changed, 3420 insertions(+), 1042 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/13e55b24/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index f8d1ce3,0a8f87c..fb004bf
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@@ -61,8 -61,15 +61,16 @@@ import static org.apache.ignite.transac
  /**
   * Full API cache test.
   */
 +@SuppressWarnings("TransientFieldInNonSerializableClass")
  public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstractSelfTest {
+     /** */
+     public static final CacheEntryProcessor<String, Integer, String> ERR_PROCESSOR =
+         new CacheEntryProcessor<String, Integer, String>() {
+             @Override public String process(MutableEntry<String, Integer> e, Object... args) {
+                 throw new RuntimeException("Failed!");
+             }
+         };
+ 
      /** Increment processor for invoke operations. */
      public static final EntryProcessor<String, Integer, String> INCR_PROCESSOR = new EntryProcessor<String, Integer, String>() {
          @Override public String process(MutableEntry<String, Integer> e, Object... args) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/13e55b24/modules/mesos/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/13e55b24/modules/rest-http/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/13e55b24/modules/urideploy/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/13e55b24/modules/web/pom.xml
----------------------------------------------------------------------

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

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


[16/50] [abbrv] ignite git commit: ignite-yarn pom.xml version fixed

Posted by nt...@apache.org.
ignite-yarn pom.xml version fixed


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

Branch: refs/heads/ignite-788-dev
Commit: aef4063a7ce743dfc5326cbccdc46312f50eaea4
Parents: 69e0ad2
Author: agura <ag...@gridgain.com>
Authored: Thu Jul 16 12:35:45 2015 +0300
Committer: agura <ag...@gridgain.com>
Committed: Thu Jul 16 12:36:44 2015 +0300

----------------------------------------------------------------------
 modules/yarn/pom.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aef4063a/modules/yarn/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yarn/pom.xml b/modules/yarn/pom.xml
index 63aa3cd..ad17f6b 100644
--- a/modules/yarn/pom.xml
+++ b/modules/yarn/pom.xml
@@ -98,4 +98,4 @@
         </plugins>
     </build>
 
-</project>
\ No newline at end of file
+</project>


[26/50] [abbrv] 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/dd246e54
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/dd246e54
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/dd246e54

Branch: refs/heads/ignite-788-dev
Commit: dd246e5489834b6e6d012bfdf6dc0577aae0af30
Parents: 700600d 57ac2b3
Author: Anton Vinogradov <av...@gridgain.com>
Authored: Thu Jul 16 13:17:06 2015 +0300
Committer: Anton Vinogradov <av...@gridgain.com>
Committed: Thu Jul 16 13:17:06 2015 +0300

----------------------------------------------------------------------
 .../IgniteClientDisconnectedException.java      |   61 +
 .../java/org/apache/ignite/IgniteCluster.java   |    5 +
 .../apache/ignite/internal/GridComponent.java   |   18 +
 .../ignite/internal/GridJobSiblingImpl.java     |    2 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |   31 +-
 .../ignite/internal/GridKernalGateway.java      |   46 +-
 .../ignite/internal/GridKernalGatewayImpl.java  |   85 +-
 .../apache/ignite/internal/GridKernalState.java |    3 +
 .../ignite/internal/GridPluginComponent.java    |   11 +
 ...gniteClientDisconnectedCheckedException.java |   49 +
 .../apache/ignite/internal/IgniteKernal.java    |  222 +++-
 .../cluster/IgniteClusterAsyncImpl.java         |    5 +
 .../internal/cluster/IgniteClusterImpl.java     |   18 +
 .../internal/managers/GridManagerAdapter.java   |   19 +-
 .../deployment/GridDeploymentCommunication.java |    2 +-
 .../deployment/GridDeploymentManager.java       |   95 +-
 .../discovery/GridDiscoveryManager.java         |  163 ++-
 .../processors/GridProcessorAdapter.java        |   11 +
 .../affinity/GridAffinityAssignmentCache.java   |   26 +-
 .../cache/CacheOsConflictResolutionManager.java |    6 +
 .../cache/DynamicCacheChangeBatch.java          |   17 +
 .../processors/cache/GridCacheAdapter.java      |   25 +-
 .../cache/GridCacheAffinityManager.java         |   21 +-
 .../cache/GridCacheConcurrentMap.java           |   15 +-
 .../processors/cache/GridCacheGateway.java      |  116 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +
 .../processors/cache/GridCacheManager.java      |    6 +
 .../cache/GridCacheManagerAdapter.java          |    6 +
 .../processors/cache/GridCacheMvccManager.java  |   41 +-
 .../GridCachePartitionExchangeManager.java      |   81 +-
 .../processors/cache/GridCachePreloader.java    |    5 +
 .../cache/GridCachePreloaderAdapter.java        |    5 +
 .../processors/cache/GridCacheProcessor.java    |  311 ++++-
 .../cache/GridCacheSharedContext.java           |  113 +-
 .../cache/GridCacheSharedManager.java           |   11 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   20 +-
 .../processors/cache/GridCacheUtils.java        |   11 +
 .../processors/cache/IgniteCacheFutureImpl.java |    5 +
 .../processors/cache/IgniteCacheProxy.java      |    2 +-
 .../CacheDataStructuresManager.java             |   35 +
 .../distributed/GridCacheTxFinishSync.java      |   46 +
 .../distributed/dht/GridDhtCacheAdapter.java    |   14 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   24 +
 .../dht/GridPartitionedGetFuture.java           |   13 +-
 .../dht/preloader/GridDhtPreloader.java         |   16 +-
 .../distributed/near/GridNearCacheAdapter.java  |    8 +
 .../distributed/near/GridNearGetFuture.java     |   13 +-
 .../cache/dr/GridOsCacheDrManager.java          |    7 +-
 .../query/GridCacheDistributedQueryManager.java |   22 +
 .../cache/query/GridCacheQueryAdapter.java      |   11 +-
 .../query/GridCacheQueryFutureAdapter.java      |    2 +-
 .../continuous/CacheContinuousQueryHandler.java |    5 +
 .../transactions/IgniteTransactionsImpl.java    |   59 +-
 .../cache/transactions/IgniteTxManager.java     |   19 +-
 .../transactions/TransactionProxyImpl.java      |    2 +-
 .../cache/version/GridCacheVersionManager.java  |    9 +-
 .../clock/GridClockSyncProcessor.java           |    6 +-
 .../processors/cluster/ClusterProcessor.java    |   11 +
 .../continuous/GridContinuousHandler.java       |    9 +-
 .../continuous/GridContinuousProcessor.java     |  127 +-
 .../datastreamer/DataStreamProcessor.java       |   24 +-
 .../datastreamer/DataStreamerImpl.java          |   90 +-
 .../datastructures/DataStructuresProcessor.java |   33 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   33 +-
 .../GridCacheAtomicReferenceImpl.java           |   34 +-
 .../GridCacheAtomicSequenceImpl.java            |   33 +-
 .../GridCacheAtomicStampedImpl.java             |   33 +-
 .../GridCacheCountDownLatchImpl.java            |   51 +-
 .../datastructures/GridCacheRemovable.java      |    6 +-
 .../datastructures/GridCacheSetImpl.java        |   15 +-
 .../datastructures/GridCacheSetProxy.java       |   47 +-
 .../processors/job/GridJobProcessor.java        |    2 +-
 .../internal/processors/job/GridJobWorker.java  |    2 +-
 .../processors/query/GridQueryIndexing.java     |    7 +
 .../processors/query/GridQueryProcessor.java    |    6 +
 .../service/GridServiceProcessor.java           |   45 +-
 .../processors/service/GridServiceProxy.java    |   13 +-
 .../processors/task/GridTaskProcessor.java      |   55 +-
 .../processors/task/GridTaskWorker.java         |   59 +-
 .../ignite/internal/util/IgniteUtils.java       |   28 +
 .../shmem/IpcSharedMemoryClientEndpoint.java    |    5 +-
 .../ignite/internal/util/lang/GridFunc.java     |    2 +
 .../java/org/apache/ignite/spi/IgniteSpi.java   |   15 +
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   37 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  354 ++++--
 .../spi/discovery/DiscoverySpiDataExchange.java |    3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  408 ++++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  134 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |    9 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   33 +-
 .../tcp/internal/TcpDiscoveryNode.java          |   19 +
 .../messages/TcpDiscoveryAbstractMessage.java   |    3 +
 .../messages/TcpDiscoveryClientAckResponse.java |   64 +
 .../messages/TcpDiscoveryHandshakeResponse.java |   14 +
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |   15 +-
 .../IgniteClientReconnectAbstractTest.java      |  363 ++++++
 .../IgniteClientReconnectApiExceptionTest.java  |  846 ++++++++++++
 .../IgniteClientReconnectAtomicsTest.java       |  672 ++++++++++
 .../IgniteClientReconnectCacheTest.java         | 1202 ++++++++++++++++++
 .../IgniteClientReconnectCollectionsTest.java   |  443 +++++++
 .../IgniteClientReconnectComputeTest.java       |  192 +++
 ...eClientReconnectContinuousProcessorTest.java |  372 ++++++
 ...IgniteClientReconnectDiscoveryStateTest.java |  123 ++
 ...niteClientReconnectFailoverAbstractTest.java |  231 ++++
 .../IgniteClientReconnectFailoverTest.java      |  212 +++
 .../IgniteClientReconnectServicesTest.java      |  260 ++++
 .../internal/IgniteClientReconnectStopTest.java |  106 ++
 .../IgniteClientReconnectStreamerTest.java      |  233 ++++
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 .../GridDeploymentManagerStopSelfTest.java      |    7 +
 .../IgniteCacheAbstractStopBusySelfTest.java    |    2 +-
 .../cache/IgniteCacheDynamicStopSelfTest.java   |    6 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    1 +
 .../IgniteCacheSystemTransactionsSelfTest.java  |    2 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |    3 +-
 .../loadtests/hashmap/GridCacheTestContext.java |    4 +-
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  562 +++++++-
 .../multijvm/IgniteClusterProcessProxy.java     |    5 +
 .../IgniteClientReconnectTestSuite.java         |   48 +
 .../processors/query/h2/IgniteH2Indexing.java   |    5 +
 .../query/h2/twostep/GridMergeIndex.java        |   45 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   70 +-
 ...ClientReconnectCacheQueriesFailoverTest.java |  225 ++++
 .../cache/IgniteClientReconnectQueriesTest.java |  427 +++++++
 ...dCacheAbstractReduceFieldsQuerySelfTest.java |    4 +
 .../IgniteCacheWithIndexingTestSuite.java       |    1 +
 128 files changed, 9751 insertions(+), 815 deletions(-)
----------------------------------------------------------------------



[50/50] [abbrv] ignite git commit: Stopping GridUpdateNotifier properly on Ignite stopping.

Posted by nt...@apache.org.
Stopping GridUpdateNotifier properly on Ignite stopping.


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

Branch: refs/heads/ignite-1.4
Commit: 35bc5ecafd669ace8321774e39e49d6c3b88efe7
Parents: b1f119f
Author: ashutak <as...@gridgain.com>
Authored: Mon Sep 7 23:45:01 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Sep 7 23:45:01 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/GridUpdateNotifier.java     | 82 ++++++++++++++------
 .../apache/ignite/internal/IgniteKernal.java    |  5 +-
 .../internal/GridUpdateNotifierSelfTest.java    | 23 +++---
 3 files changed, 70 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/35bc5eca/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
index 448f87b..b6162ed 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
@@ -26,12 +26,12 @@ import java.io.OutputStream;
 import java.io.PrintWriter;
 import java.io.StringReader;
 import java.io.StringWriter;
+import java.io.UnsupportedEncodingException;
 import java.net.URL;
 import java.net.URLConnection;
 import java.util.Collection;
-import java.util.Map;
-import java.util.concurrent.Executor;
 import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicReference;
 import javax.xml.parsers.DocumentBuilder;
 import javax.xml.parsers.DocumentBuilderFactory;
 import javax.xml.parsers.ParserConfigurationException;
@@ -59,21 +59,24 @@ import static java.net.URLEncoder.encode;
  * gracefully ignore any errors occurred during notification and verification process.
  */
 class GridUpdateNotifier {
+    /** Default encoding. */
+    private static final String CHARSET = "UTF-8";
+
     /** Access URL to be used to access latest version data. */
     private static final String UPD_STATUS_PARAMS = IgniteProperties.get("ignite.update.status.params");
 
     /** Throttling for logging out. */
     private static final long THROTTLE_PERIOD = 24 * 60 * 60 * 1000; // 1 day.
 
+    /** Sleep milliseconds time for worker thread. */
+    public static final int WORKER_THREAD_SLEEP_TIME = 5000;
+
     /** Grid version. */
     private final String ver;
 
     /** Site. */
     private final String url;
 
-    /** Asynchronous checked. */
-    private GridWorker checker;
-
     /** Latest version. */
     private volatile String latestVer;
 
@@ -95,7 +98,8 @@ class GridUpdateNotifier {
     /** System properties */
     private final String vmProps;
 
-    private final Map<String, String> pluginVers;
+    /** Plugins information for request */
+    private final String pluginsVers;
 
     /** Kernal gateway */
     private final GridKernalGateway gw;
@@ -103,6 +107,12 @@ class GridUpdateNotifier {
     /** */
     private long lastLog = -1;
 
+    /** Command for worker thread. */
+    private final AtomicReference<Runnable> cmd = new AtomicReference<>();
+
+    /** Worker thread to process http request. */
+    private final Thread workerThread;
+
     /**
      * Creates new notifier with default values.
      *
@@ -136,18 +146,46 @@ class GridUpdateNotifier {
             this.gridName = gridName == null ? "null" : gridName;
             this.gw = gw;
 
-            pluginVers = U.newHashMap(pluginProviders.size());
+            SB pluginsBuilder = new SB();
 
             for (PluginProvider provider : pluginProviders)
-                pluginVers.put(provider.name() + "-plugin-version", provider.version());
+                pluginsBuilder.a("&").a(provider.name() + "-plugin-version").a("=").
+                    a(encode(provider.version(), CHARSET));
+
+            pluginsVers = pluginsBuilder.toString();
 
             this.reportOnlyNew = reportOnlyNew;
 
             vmProps = getSystemProperties();
+
+            workerThread = new Thread(new Runnable() {
+                @Override public void run() {
+                    try {
+                        while(!Thread.currentThread().isInterrupted()) {
+                            Runnable cmd0 = cmd.getAndSet(null);
+
+                            if (cmd0 != null)
+                                cmd0.run();
+                            else
+                                Thread.sleep(WORKER_THREAD_SLEEP_TIME);
+                        }
+                    }
+                    catch (InterruptedException ignore) {
+                        // No-op.
+                    }
+                }
+            }, "upd-ver-checker");
+
+            workerThread.setDaemon(true);
+
+            workerThread.start();
         }
         catch (ParserConfigurationException e) {
             throw new IgniteCheckedException("Failed to create xml parser.", e);
         }
+        catch (UnsupportedEncodingException e) {
+            throw new IgniteCheckedException("Failed to encode.", e);
+        }
     }
 
     /**
@@ -197,16 +235,15 @@ class GridUpdateNotifier {
     /**
      * Starts asynchronous process for retrieving latest version data.
      *
-     * @param exec Executor service.
      * @param log Logger.
      */
-    void checkForNewVersion(Executor exec, IgniteLogger log) {
+    void checkForNewVersion(IgniteLogger log) {
         assert log != null;
 
         log = log.getLogger(getClass());
 
         try {
-            exec.execute(checker = new UpdateChecker(log));
+            cmd.set(new UpdateChecker(log));
         }
         catch (RejectedExecutionException e) {
             U.error(log, "Failed to schedule a thread due to execution rejection (safely ignoring): " +
@@ -224,10 +261,6 @@ class GridUpdateNotifier {
 
         log = log.getLogger(getClass());
 
-        // Don't join it to avoid any delays on update checker.
-        // Checker thread will eventually exit.
-        U.cancel(checker);
-
         String latestVer = this.latestVer;
         String downloadUrl = this.downloadUrl;
 
@@ -272,12 +305,16 @@ class GridUpdateNotifier {
     }
 
     /**
+     * Stops update notifier.
+     */
+    public void stop() {
+        workerThread.interrupt();
+    }
+
+    /**
      * Asynchronous checker of the latest version available.
      */
     private class UpdateChecker extends GridWorker {
-        /** Default encoding. */
-        private static final String CHARSET = "UTF-8";
-
         /** Logger. */
         private final IgniteLogger log;
 
@@ -297,18 +334,13 @@ class GridUpdateNotifier {
             try {
                 String stackTrace = gw != null ? gw.userStackTrace() : null;
 
-                SB plugins = new SB();
-
-                for (Map.Entry<String, String> p : pluginVers.entrySet())
-                    plugins.a("&").a(p.getKey()).a("=").a(encode(p.getValue(), CHARSET));
-
                 String postParams =
                     "gridName=" + encode(gridName, CHARSET) +
                     (!F.isEmpty(UPD_STATUS_PARAMS) ? "&" + UPD_STATUS_PARAMS : "") +
                     (topSize > 0 ? "&topSize=" + topSize : "") +
                     (!F.isEmpty(stackTrace) ? "&stackTrace=" + encode(stackTrace, CHARSET) : "") +
                     (!F.isEmpty(vmProps) ? "&vmProps=" + encode(vmProps, CHARSET) : "") +
-                    plugins.toString();
+                        pluginsVers;
 
                 URLConnection conn = new URL(url).openConnection();
 
@@ -419,4 +451,4 @@ class GridUpdateNotifier {
             return obtainMeta("downloadUrl", node);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/35bc5eca/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 14d7c14..f160d46 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1839,6 +1839,9 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             if (updateNtfTimer != null)
                 updateNtfTimer.cancel();
 
+            if (verChecker != null)
+                verChecker.stop();
+
             if (starveTask != null)
                 starveTask.close();
 
@@ -3220,7 +3223,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     verChecker.topologySize(kernal.cluster().nodes().size());
             }
 
-            verChecker.checkForNewVersion(execSvc, log);
+            verChecker.checkForNewVersion(log);
 
             // Just wait for 10 secs.
             Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);

http://git-wip-us.apache.org/repos/asf/ignite/blob/35bc5eca/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
index b6109f7..61c2085 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal;
 
 import java.util.Collections;
 import java.util.Properties;
-import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -27,7 +26,6 @@ import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.plugin.PluginProvider;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.common.GridCommonTest;
-import org.jetbrains.annotations.NotNull;
 
 /**
  * Update notifier test.
@@ -74,10 +72,17 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
         GridUpdateNotifier ntf = new GridUpdateNotifier(null, nodeVer,
             TEST_GATEWAY, Collections.<PluginProvider>emptyList(), false);
 
-        ntf.checkForNewVersion(new SelfExecutor(), log);
+        ntf.checkForNewVersion(log);
 
         String ver = ntf.latestVersion();
 
+        // Wait 60 sec for response.
+        for (int i = 0; ver == null && i < 600; i++) {
+            Thread.sleep(100);
+
+            ver = ntf.latestVersion();
+        }
+
         info("Latest version: " + ver);
 
         assertNotNull("Ignite latest version has not been detected.", ver);
@@ -93,16 +98,6 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     * Executor that runs task in current thread.
-     */
-    private static class SelfExecutor implements Executor {
-        /** {@inheritDoc} */
-        @Override public void execute(@NotNull Runnable r) {
-            r.run();
-        }
-    }
-
-    /**
      * Test kernal gateway that always return uninitialized user stack trace.
      */
     private static final GridKernalGateway TEST_GATEWAY = new GridKernalGateway() {
@@ -138,4 +133,4 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
             // No-op.
         }
     };
-}
\ No newline at end of file
+}


[07/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-1085

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


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

Branch: refs/heads/ignite-788-dev
Commit: d11fc9b0811f33e8d1e3f1d2eeabac0504952833
Parents: f0ace1f a6d33b1
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 14:18:16 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 14:18:16 2015 +0300

----------------------------------------------------------------------
 .../IgniteCacheAtomicReplicatedNodeRestartSelfTest.java         | 5 +++++
 1 file changed, 5 insertions(+)
----------------------------------------------------------------------



[28/50] [abbrv] 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/17f8c85d
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/17f8c85d
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/17f8c85d

Branch: refs/heads/ignite-788-dev
Commit: 17f8c85de3cc62f4556190805354f6dcc75b7ccf
Parents: 9e52912 dd246e5
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jul 16 13:26:44 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jul 16 13:26:44 2015 +0300

----------------------------------------------------------------------
 pom.xml | 14 ++++++--------
 1 file changed, 6 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[09/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-890

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


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

Branch: refs/heads/ignite-788-dev
Commit: b601a62a3ce1935afea803814bef830bd7f0230f
Parents: 10f3fad bdd1762
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 17:13:25 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 17:13:25 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAbstractNodeRestartSelfTest.java           | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[06/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-890

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


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

Branch: refs/heads/ignite-788-dev
Commit: 10f3fad26fa97face0bb7f1c83047c9f8fd4e7bf
Parents: e402e98 a6d33b1
Author: Denis Magda <dm...@gridgain.com>
Authored: Wed Jul 15 14:14:34 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Wed Jul 15 14:14:34 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/query/GridQueryProcessor.java    | 4 +++-
 .../IgniteCacheAtomicReplicatedNodeRestartSelfTest.java         | 5 +++++
 2 files changed, 8 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[46/50] [abbrv] ignite git commit: 1.4.1-SNAPSHOT

Posted by nt...@apache.org.
1.4.1-SNAPSHOT


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

Branch: refs/heads/ignite-788-dev
Commit: 0ac5fe7565a709b666c9d225574cca5bfbb38a95
Parents: cf08181
Author: Ignite Teamcity <ig...@apache.org>
Authored: Mon Jul 20 13:59:53 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Mon Jul 20 13:59:53 2015 +0300

----------------------------------------------------------------------
 pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0ac5fe75/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a6ca9f5..2ae259c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -607,12 +607,12 @@
                                         </copy>
 
                                         <!-- appending filename to md5 and sha1 files. to be improved. -->
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message="  ${project.artifactId}-fabric-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message="  ${project.artifactId}-fabric-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message="  ${project.artifactId}-hadoop-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message="  ${project.artifactId}-hadoop-${project.version}-bin.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message="  ${project.artifactId}-${project.version}-src.zip" />
-                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message="  ${project.artifactId}-${project.version}-src.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-fabric-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-fabric-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.md5" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-hadoop-${project.version}-bin.zip.sha1" append="true" message=" ${project.artifactId}-hadoop-${project.version}-bin.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.md5" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
+                                        <echo file="${basedir}/target/site/${project.artifactId}-${project.version}-src.zip.sha1" append="true" message=" ${project.artifactId}-${project.version}-src.zip" />
                                     </target>
                                 </configuration>
                             </execution>


[22/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index dd04bf4..daa9494 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -153,21 +153,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         ctx.event().addLocalEventListener(new GridLocalEventListener() {
             @Override public void onEvent(Event evt) {
-                for (Iterator<StartFuture> itr = startFuts.values().iterator(); itr.hasNext(); ) {
-                    StartFuture fut = itr.next();
-
-                    itr.remove();
-
-                    fut.onDone(new IgniteException("Topology segmented"));
-                }
-
-                for (Iterator<StopFuture> itr = stopFuts.values().iterator(); itr.hasNext(); ) {
-                    StopFuture fut = itr.next();
-
-                    itr.remove();
-
-                    fut.onDone(new IgniteException("Topology segmented"));
-                }
+                cancelFutures(new IgniteCheckedException("Topology segmented"));
             }
         }, EVT_NODE_SEGMENTED);
 
@@ -263,6 +249,27 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param e Error.
+     */
+    private void cancelFutures(IgniteCheckedException e) {
+        for (Iterator<StartFuture> itr = startFuts.values().iterator(); itr.hasNext(); ) {
+            StartFuture fut = itr.next();
+
+            itr.remove();
+
+            fut.onDone(e);
+        }
+
+        for (Iterator<StopFuture> itr = stopFuts.values().iterator(); itr.hasNext(); ) {
+            StopFuture fut = itr.next();
+
+            itr.remove();
+
+            fut.onDone(e);
+        }
+    }
+
+    /**
      * @return {@code true} if lock successful, {@code false} if processor already stopped.
      */
     @SuppressWarnings("LockAcquiredButNotSafelyReleased")
@@ -318,27 +325,30 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
     /** {@inheritDoc} */
     @Override @Nullable public Serializable collectDiscoveryData(UUID nodeId) {
-        if (!nodeId.equals(ctx.localNodeId())) {
+        if (!nodeId.equals(ctx.localNodeId()) || !locInfos.isEmpty()) {
             DiscoveryData data = new DiscoveryData(ctx.localNodeId(), clientInfos);
 
-            // Collect listeners information (will be sent to
-            // joining node during discovery process).
+            // Collect listeners information (will be sent to joining node during discovery process).
             for (Map.Entry<UUID, LocalRoutineInfo> e : locInfos.entrySet()) {
                 UUID routineId = e.getKey();
                 LocalRoutineInfo info = e.getValue();
 
-                data.addItem(new DiscoveryDataItem(routineId, info.prjPred,
-                    info.hnd, info.bufSize, info.interval));
+                data.addItem(new DiscoveryDataItem(routineId,
+                    info.prjPred,
+                    info.hnd,
+                    info.bufSize,
+                    info.interval,
+                    info.autoUnsubscribe));
             }
 
             return data;
         }
-        else
-            return null;
+
+        return null;
     }
 
     /** {@inheritDoc} */
-    @Override public void onDiscoveryDataReceived(UUID nodeId, UUID rmtNodeId, Serializable obj) {
+    @Override public void onDiscoveryDataReceived(UUID joiningNodeId, UUID rmtNodeId, Serializable obj) {
         DiscoveryData data = (DiscoveryData)obj;
 
         if (!ctx.isDaemon() && data != null) {
@@ -377,6 +387,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * Callback invoked when cache is started.
      *
      * @param ctx Cache context.
+     * @throws IgniteCheckedException If failed.
      */
     public void onCacheStart(GridCacheContext ctx) throws IgniteCheckedException {
         for (Map.Entry<UUID, RemoteRoutineInfo> entry : rmtInfos.entrySet()) {
@@ -491,7 +502,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         }
 
         // Register routine locally.
-        locInfos.put(routineId, new LocalRoutineInfo(prjPred, hnd, bufSize, interval));
+        locInfos.put(routineId, new LocalRoutineInfo(prjPred, hnd, bufSize, interval, autoUnsubscribe));
 
         StartFuture fut = new StartFuture(ctx, routineId);
 
@@ -500,7 +511,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         try {
             ctx.discovery().sendCustomEvent(new StartRoutineDiscoveryMessage(routineId, reqData));
         }
-        catch (IgniteException e) { // Marshaller exception may occurs if user pass unmarshallable filter.
+        catch (IgniteCheckedException e) { // Marshaller exception may occurs if user pass unmarshallable filter.
             startFuts.remove(routineId);
 
             locInfos.remove(routineId);
@@ -565,7 +576,12 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             // Unregister handler locally.
             unregisterHandler(routineId, routine.hnd, true);
 
-            ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
+            try {
+                ctx.discovery().sendCustomEvent(new StopRoutineDiscoveryMessage(routineId));
+            }
+            catch (IgniteCheckedException e) {
+                fut.onDone(e);
+            }
 
             if (ctx.isStopping())
                 fut.onDone();
@@ -580,6 +596,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * @param obj Notification object.
      * @param orderedTopic Topic for ordered notifications. If {@code null}, non-ordered message will be sent.
      * @param sync If {@code true} then waits for event acknowledgment.
+     * @param msg If {@code true} then sent data is message.
      * @throws IgniteCheckedException In case of error.
      */
     public void addNotification(UUID nodeId,
@@ -630,6 +647,18 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        cancelFutures(new IgniteClientDisconnectedCheckedException(reconnectFut, "Client node disconnected."));
+
+        for (UUID rmtId : rmtInfos.keySet())
+            unregisterRemote(rmtId);
+
+        rmtInfos.clear();
+
+        clientInfos.clear();
+    }
+
     /**
      * @param nodeId Node ID.
      * @param routineId Routine ID.
@@ -637,6 +666,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * @param toSnd Notification object to send.
      * @param orderedTopic Topic for ordered notifications.
      *      If {@code null}, non-ordered message will be sent.
+     * @param msg If {@code true} then sent data is collection of messages.
      * @throws IgniteCheckedException In case of error.
      */
     private void sendNotification(UUID nodeId,
@@ -703,8 +733,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                 assert old == null;
             }
 
-            clientRouteMap.put(routineId, new LocalRoutineInfo(data.projectionPredicate(), hnd, data.bufferSize(),
-                data.interval()));
+            clientRouteMap.put(routineId, new LocalRoutineInfo(data.projectionPredicate(),
+                hnd,
+                data.bufferSize(),
+                data.interval(),
+                data.autoUnsubscribe()));
         }
 
         boolean registered = false;
@@ -1022,14 +1055,22 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         /** Time interval. */
         private final long interval;
 
+        /** Automatic unsubscribe flag. */
+        private boolean autoUnsubscribe;
+
         /**
          * @param prjPred Projection predicate.
          * @param hnd Continuous routine handler.
          * @param bufSize Buffer size.
          * @param interval Interval.
+         * @param autoUnsubscribe Automatic unsubscribe flag.
          */
-        LocalRoutineInfo(@Nullable IgnitePredicate<ClusterNode> prjPred, GridContinuousHandler hnd, int bufSize,
-            long interval) {
+        LocalRoutineInfo(@Nullable IgnitePredicate<ClusterNode> prjPred,
+            GridContinuousHandler hnd,
+            int bufSize,
+            long interval,
+            boolean autoUnsubscribe)
+        {
             assert hnd != null;
             assert bufSize > 0;
             assert interval >= 0;
@@ -1038,6 +1079,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             this.hnd = hnd;
             this.bufSize = bufSize;
             this.interval = interval;
+            this.autoUnsubscribe = autoUnsubscribe;
         }
 
         /**
@@ -1046,6 +1088,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         GridContinuousHandler handler() {
             return hnd;
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(LocalRoutineInfo.class, this);
+        }
     }
 
     /**
@@ -1053,7 +1100,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      */
     private static class RemoteRoutineInfo {
         /** Master node ID. */
-        private final UUID nodeId;
+        private UUID nodeId;
 
         /** Continuous routine handler. */
         private final GridContinuousHandler hnd;
@@ -1205,6 +1252,11 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
             return F.t(toSnd, diff < interval ? interval - diff : interval);
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(RemoteRoutineInfo.class, this);
+        }
     }
 
     /**
@@ -1221,6 +1273,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
         @GridToStringInclude
         private Collection<DiscoveryDataItem> items;
 
+        /** */
         private Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos;
 
         /**
@@ -1232,6 +1285,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
 
         /**
          * @param nodeId Node ID.
+         * @param clientInfos Client information.
          */
         DiscoveryData(UUID nodeId, Map<UUID, Map<UUID, LocalRoutineInfo>> clientInfos) {
             assert nodeId != null;
@@ -1308,9 +1362,15 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
          * @param hnd Handler.
          * @param bufSize Buffer size.
          * @param interval Time interval.
+         * @param autoUnsubscribe Automatic unsubscribe flag.
          */
-        DiscoveryDataItem(UUID routineId, @Nullable IgnitePredicate<ClusterNode> prjPred,
-            GridContinuousHandler hnd, int bufSize, long interval) {
+        DiscoveryDataItem(UUID routineId,
+            @Nullable IgnitePredicate<ClusterNode> prjPred,
+            GridContinuousHandler hnd,
+            int bufSize,
+            long interval,
+            boolean autoUnsubscribe)
+        {
             assert routineId != null;
             assert hnd != null;
             assert bufSize > 0;
@@ -1321,6 +1381,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
             this.hnd = hnd;
             this.bufSize = bufSize;
             this.interval = interval;
+            this.autoUnsubscribe = autoUnsubscribe;
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index 54478f8..4f75e0c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.util.*;
 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.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.stream.*;
 import org.apache.ignite.thread.*;
@@ -63,13 +64,15 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
     public DataStreamProcessor(GridKernalContext ctx) {
         super(ctx);
 
-        ctx.io().addMessageListener(TOPIC_DATASTREAM, new GridMessageListener() {
-            @Override public void onMessage(UUID nodeId, Object msg) {
-                assert msg instanceof DataStreamerRequest;
+        if (!ctx.clientNode()) {
+            ctx.io().addMessageListener(TOPIC_DATASTREAM, new GridMessageListener() {
+                @Override public void onMessage(UUID nodeId, Object msg) {
+                    assert msg instanceof DataStreamerRequest;
 
-                processRequest(nodeId, (DataStreamerRequest)msg);
-            }
-        });
+                    processRequest(nodeId, (DataStreamerRequest)msg);
+                }
+            });
+        }
 
         marsh = ctx.config().getMarshaller();
     }
@@ -113,7 +116,8 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
         if (ctx.config().isDaemon())
             return;
 
-        ctx.io().removeMessageListener(TOPIC_DATASTREAM);
+        if (!ctx.clientNode())
+            ctx.io().removeMessageListener(TOPIC_DATASTREAM);
 
         busyLock.block();
 
@@ -139,6 +143,12 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
             log.debug("Stopped data streamer processor.");
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        for (DataStreamerImpl<?, ?> ldr : ldrs)
+            ldr.onDisconnected(reconnectFut);
+    }
+
     /**
      * @param cacheName Cache name ({@code null} for default cache).
      * @return Data loader.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index 26b0568..605f478 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -145,6 +145,9 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
     /** Busy lock. */
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
+    /** */
+    private CacheException disconnectErr;
+
     /** Closed flag. */
     private final AtomicBoolean closed = new AtomicBoolean();
 
@@ -245,7 +248,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
         fut = new DataStreamerFuture(this);
 
-        publicFut = new IgniteFutureImpl<>(fut);
+        publicFut = new IgniteCacheFutureImpl<>(fut);
     }
 
     /**
@@ -284,8 +287,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
      * Enters busy lock.
      */
     private void enterBusy() {
-        if (!busyLock.enterBusy())
+        if (!busyLock.enterBusy()) {
+            if (disconnectErr != null)
+                throw disconnectErr;
+
             throw new IllegalStateException("Data streamer has been closed.");
+        }
     }
 
     /**
@@ -435,7 +442,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             load0(entries0, resFut, keys, 0);
 
-            return new IgniteFutureImpl<>(resFut);
+            return new IgniteCacheFutureImpl<>(resFut);
         }
         catch (IgniteException e) {
             return new IgniteFinishedFutureImpl<>(e);
@@ -487,7 +494,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             load0(entries, resFut, keys, 0);
 
-            return new IgniteFutureImpl<>(resFut);
+            return new IgniteCacheFutureImpl<>(resFut);
         }
         catch (Throwable e) {
             resFut.onDone(e);
@@ -631,6 +638,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                             resFut.onDone();
                         }
                     }
+                    catch (IgniteClientDisconnectedCheckedException e1) {
+                        if (log.isDebugEnabled())
+                            log.debug("Future finished with disconnect error [nodeId=" + nodeId + ", err=" + e1 + ']');
+
+                        resFut.onDone(e1);
+                    }
                     catch (IgniteCheckedException e1) {
                         if (log.isDebugEnabled())
                             log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']');
@@ -757,6 +770,12 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                     try {
                         fut.get();
                     }
+                    catch (IgniteClientDisconnectedCheckedException e) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to flush buffer: " + e);
+
+                        throw CU.convertToCacheException(e);
+                    }
                     catch (IgniteCheckedException e) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to flush buffer: " + e);
@@ -802,7 +821,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             doFlush();
         }
         catch (IgniteCheckedException e) {
-            throw GridCacheUtils.convertToCacheException(e);
+            throw CU.convertToCacheException(e);
         }
         finally {
             leaveBusy();
@@ -843,7 +862,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             closeEx(cancel);
         }
         catch (IgniteCheckedException e) {
-            throw GridCacheUtils.convertToCacheException(e);
+            throw CU.convertToCacheException(e);
         }
     }
 
@@ -852,6 +871,15 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
      * @throws IgniteCheckedException If failed.
      */
     public void closeEx(boolean cancel) throws IgniteCheckedException {
+        closeEx(cancel, null);
+    }
+
+    /**
+     * @param cancel {@code True} to close with cancellation.
+     * @param err Error.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void closeEx(boolean cancel, IgniteCheckedException err) throws IgniteCheckedException {
         if (!closed.compareAndSet(false, true))
             return;
 
@@ -868,7 +896,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 cancelled = true;
 
                 for (Buffer buf : bufMappings.values())
-                    buf.cancelAll();
+                    buf.cancelAll(err);
             }
             else
                 doFlush();
@@ -881,13 +909,29 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
             e = e0;
         }
 
-        fut.onDone(null, e);
+        fut.onDone(null, e != null ? e : err);
 
         if (e != null)
             throw e;
     }
 
     /**
+     * @param reconnectFut Reconnect future.
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
+            "Data streamer has been closed, client node disconnected.");
+
+        disconnectErr = (CacheException)CU.convertToCacheException(err);
+
+        for (Buffer buf : bufMappings.values())
+            buf.cancelAll(err);
+
+        closeEx(true, err);
+    }
+
+    /**
      * @return {@code true} If the loader is closed.
      */
     boolean isClosed() {
@@ -1027,7 +1071,11 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                 submit(entries0, topVer, curFut0);
 
                 if (cancelled)
-                    curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " + DataStreamerImpl.this));
+                    curFut0.onDone(new IgniteCheckedException("Data streamer has been cancelled: " +
+                        DataStreamerImpl.this));
+                else if (ctx.clientDisconnected())
+                    curFut0.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                        "Client node disconnected."));
             }
 
             return curFut0;
@@ -1227,11 +1275,18 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
                         log.debug("Sent request to node [nodeId=" + node.id() + ", req=" + req + ']');
                 }
                 catch (IgniteCheckedException e) {
-                    if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
-                        ((GridFutureAdapter<Object>)fut).onDone(e);
-                    else
-                        ((GridFutureAdapter<Object>)fut).onDone(new ClusterTopologyCheckedException("Failed to send " +
-                            "request (node has left): " + node.id()));
+                    GridFutureAdapter<Object> fut0 = ((GridFutureAdapter<Object>)fut);
+
+                    try {
+                        if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
+                            fut0.onDone(e);
+                        else
+                            fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
+                                + node.id()));
+                    }
+                    catch (IgniteClientDisconnectedCheckedException e0) {
+                        fut0.onDone(e0);
+                    }
                 }
             }
         }
@@ -1304,10 +1359,11 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
         }
 
         /**
-         *
+         * @param err Error.
          */
-        void cancelAll() {
-            IgniteCheckedException err = new IgniteCheckedException("Data streamer has been cancelled: " + DataStreamerImpl.this);
+        void cancelAll(@Nullable IgniteCheckedException err) {
+            if (err == null)
+                err = new IgniteCheckedException("Data streamer has been cancelled: " + DataStreamerImpl.this);
 
             for (IgniteInternalFuture<?> f : locFuts) {
                 try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 5c171e8..57b16f0 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
@@ -185,6 +185,32 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param key Key.
+     * @param obj Object.
+     */
+    void onRemoved(GridCacheInternal key, GridCacheRemovable obj) {
+        dsMap.remove(key, obj);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onReconnected(boolean clusterRestarted) throws IgniteCheckedException {
+        for (Map.Entry<GridCacheInternal, GridCacheRemovable> e : dsMap.entrySet()) {
+            GridCacheRemovable obj = e.getValue();
+
+            if (clusterRestarted) {
+                obj.onRemoved();
+
+                dsMap.remove(e.getKey(), obj);
+            }
+            else
+                obj.needCheckNotRemoved();
+        }
+
+        for (GridCacheContext cctx : ctx.cache().context().cacheContexts())
+            cctx.dataStructures().onReconnected(clusterRestarted);
+    }
+
+    /**
      * Gets a sequence from cache or creates one if it's not cached.
      *
      * @param name Sequence name.
@@ -1001,8 +1027,11 @@ public final class DataStructuresProcessor extends GridProcessorAdapter {
                         dsView.put(key, val);
                     }
 
-                    latch = new GridCacheCountDownLatchImpl(name, val.get(), val.initialCount(),
-                        val.autoDelete(), key, cntDownLatchView, dsCacheCtx);
+                    latch = new GridCacheCountDownLatchImpl(name, val.initialCount(),
+                        val.autoDelete(),
+                        key,
+                        cntDownLatchView,
+                        dsCacheCtx);
 
                     dsMap.put(key, latch);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
index 5e9245d..1d6e735 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicLongImpl.java
@@ -57,6 +57,9 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     /** Removed flag.*/
     private volatile boolean rmvd;
 
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
     /** Atomic long key. */
     private GridCacheInternalKey key;
 
@@ -336,7 +339,31 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
      */
     private void checkRemoved() throws IllegalStateException {
         if (rmvd)
-            throw new IllegalStateException("Atomic long was removed from cache: " + name);
+            throw removedError();
+
+        if (rmvCheck) {
+            try {
+                rmvd = atomicView.get(key) == null;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                ctx.kernalContext().dataStructures().onRemoved(key, this);
+
+                throw removedError();
+            }
+        }
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Atomic long was removed from cache: " + name);
     }
 
     /** {@inheritDoc} */
@@ -345,8 +372,8 @@ public final class GridCacheAtomicLongImpl implements GridCacheAtomicLongEx, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public void onInvalid(@Nullable Exception err) {
-        // No-op.
+    @Override public void needCheckNotRemoved() {
+        rmvCheck = true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
index 0c4e5e6..f740c4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicReferenceImpl.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
-import org.jetbrains.annotations.*;
 
 import java.io.*;
 import java.util.concurrent.*;
@@ -56,6 +55,9 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     /** Status.*/
     private volatile boolean rmvd;
 
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
     /** Atomic reference key. */
     private GridCacheInternalKey key;
 
@@ -156,8 +158,8 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
     }
 
     /** {@inheritDoc} */
-    @Override public void onInvalid(@Nullable Exception err) {
-        // No-op.
+    @Override public void needCheckNotRemoved() {
+        rmvCheck = true;
     }
 
     /** {@inheritDoc} */
@@ -293,7 +295,31 @@ public final class GridCacheAtomicReferenceImpl<T> implements GridCacheAtomicRef
      */
     private void checkRemoved() throws IllegalStateException {
         if (rmvd)
-            throw new IllegalStateException("Atomic reference was removed from cache: " + name);
+            throw removedError();
+
+        if (rmvCheck) {
+            try {
+                rmvd = atomicView.get(key) == null;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                ctx.kernalContext().dataStructures().onRemoved(key, this);
+
+                throw removedError();
+            }
+        }
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Atomic reference was removed from cache: " + name);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 2400a7e..31f4f24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -61,6 +61,9 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
     /** Removed flag. */
     private volatile boolean rmvd;
 
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
     /** Sequence key. */
     private GridCacheInternalKey key;
 
@@ -391,7 +394,31 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
      */
     private void checkRemoved() throws IllegalStateException {
         if (rmvd)
-            throw new IllegalStateException("Sequence was removed from cache: " + name);
+            throw removedError();
+
+        if (rmvCheck) {
+            try {
+                rmvd = seqView.get(key) == null;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                ctx.kernalContext().dataStructures().onRemoved(key, this);
+
+                throw removedError();
+            }
+        }
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Sequence was removed from cache: " + name);
     }
 
     /** {@inheritDoc} */
@@ -400,8 +427,8 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
     }
 
     /** {@inheritDoc} */
-    @Override public void onInvalid(@Nullable Exception err) {
-        // No-op.
+    @Override public void needCheckNotRemoved() {
+        rmvCheck = true;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
index 76ea7ca..d2dedeb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicStampedImpl.java
@@ -59,6 +59,9 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     /** Removed flag.*/
     private volatile boolean rmvd;
 
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
     /** Atomic stamped key. */
     private GridCacheInternalKey key;
 
@@ -206,8 +209,8 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
     }
 
     /** {@inheritDoc} */
-    @Override public void onInvalid(@Nullable Exception err) {
-        // No-op.
+    @Override public void needCheckNotRemoved() {
+        rmvCheck = true;
     }
 
     /** {@inheritDoc} */
@@ -369,7 +372,31 @@ public final class GridCacheAtomicStampedImpl<T, S> implements GridCacheAtomicSt
      */
     private void checkRemoved() throws IllegalStateException {
         if (rmvd)
-            throw new IllegalStateException("Atomic stamped was removed from cache: " + name);
+            throw removedError();
+
+        if (rmvCheck) {
+            try {
+                rmvd = atomicView.get(key) == null;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                ctx.kernalContext().dataStructures().onRemoved(key, this);
+
+                throw removedError();
+            }
+        }
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Atomic stamped was removed from cache: " + name);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
index 85b6cfd..95b970a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheCountDownLatchImpl.java
@@ -67,9 +67,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     /** Cache context. */
     private GridCacheContext ctx;
 
-    /** Current count. */
-    private int cnt;
-
     /** Initial count. */
     private int initCnt;
 
@@ -96,7 +93,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
      * Constructor.
      *
      * @param name Latch name.
-     * @param cnt Current count.
      * @param initCnt Initial count.
      * @param autoDel Auto delete flag.
      * @param key Latch key.
@@ -104,7 +100,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
      * @param ctx Cache context.
      */
     public GridCacheCountDownLatchImpl(String name,
-        int cnt,
         int initCnt,
         boolean autoDel,
         GridCacheInternalKey key,
@@ -112,14 +107,12 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
         GridCacheContext ctx)
     {
         assert name != null;
-        assert cnt >= 0;
         assert initCnt >= 0;
         assert key != null;
         assert latchView != null;
         assert ctx != null;
 
         this.name = name;
-        this.cnt = cnt;
         this.initCnt = initCnt;
         this.autoDel = autoDel;
         this.key = key;
@@ -136,7 +129,12 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public int count() {
-        return cnt;
+        try {
+            return CU.outTx(new GetCountCallable(), ctx);
+        }
+        catch (IgniteCheckedException e) {
+            throw U.convertException(e);
+        }
     }
 
     /** {@inheritDoc} */
@@ -207,13 +205,11 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
 
     /** {@inheritDoc} */
     @Override public boolean onRemoved() {
-        assert cnt == 0;
-
         return rmvd = true;
     }
 
     /** {@inheritDoc} */
-    @Override public void onInvalid(@Nullable Exception err) {
+    @Override public void needCheckNotRemoved() {
         // No-op.
     }
 
@@ -231,8 +227,6 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     @Override public void onUpdate(int cnt) {
         assert cnt >= 0;
 
-        this.cnt = cnt;
-
         while (internalLatch != null && internalLatch.getCount() > cnt)
             internalLatch.countDown();
     }
@@ -253,9 +247,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
                                     if (log.isDebugEnabled())
                                         log.debug("Failed to find count down latch with given name: " + name);
 
-                                    assert cnt == 0;
-
-                                    return new CountDownLatch(cnt);
+                                    return new CountDownLatch(0);
                                 }
 
                                 tx.commit();
@@ -337,6 +329,29 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
     /**
      *
      */
+    private class GetCountCallable implements Callable<Integer> {
+        /** {@inheritDoc} */
+        @Override public Integer call() throws Exception {
+            Integer val;
+
+            try (IgniteInternalTx tx = CU.txStartInternal(ctx, latchView, PESSIMISTIC, REPEATABLE_READ)) {
+                GridCacheCountDownLatchValue latchVal = latchView.get(key);
+
+                if (latchVal == null)
+                    return 0;
+
+                val = latchVal.get();
+
+                tx.rollback();
+            }
+
+            return val;
+        }
+    }
+
+    /**
+     *
+     */
     private class CountDownCallable implements Callable<Integer> {
         /** Value to count down on (if 0 then latch is counted down to 0). */
         private final int val;
@@ -359,9 +374,7 @@ public final class GridCacheCountDownLatchImpl implements GridCacheCountDownLatc
                     if (log.isDebugEnabled())
                         log.debug("Failed to find count down latch with given name: " + name);
 
-                    assert cnt == 0;
-
-                    return cnt;
+                    return 0;
                 }
 
                 int retVal;

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
index 48d8644..dd4f2cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheRemovable.java
@@ -17,8 +17,6 @@
 
 package org.apache.ignite.internal.processors.datastructures;
 
-import org.jetbrains.annotations.*;
-
 /**
  * Provides callback for marking object as removed.
  */
@@ -31,7 +29,7 @@ public interface GridCacheRemovable {
     public boolean onRemoved();
 
     /**
-     * @param err Error which cause data structure to become invalid.
+     *
      */
-    public void onInvalid(@Nullable Exception err);
+    public void needCheckNotRemoved();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
index f74fe95..6d920fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetImpl.java
@@ -101,6 +101,19 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
         return rmvd;
     }
 
+    /**
+     * @return {@code True} if set header found in cache.
+     * @throws IgniteCheckedException If failed.
+     */
+    @SuppressWarnings("unchecked")
+    public boolean checkHeader() throws IgniteCheckedException {
+        IgniteInternalCache<GridCacheSetHeaderKey, GridCacheSetHeader> cache0 = ctx.cache();
+
+        GridCacheSetHeader hdr = cache0.get(new GridCacheSetHeaderKey(name));
+
+        return hdr != null && hdr.id().equals(id);
+    }
+
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public int size() {
@@ -476,7 +489,7 @@ public class GridCacheSetImpl<T> extends AbstractCollection<T> implements Ignite
     /**
      * @return Set ID.
      */
-    IgniteUuid id() {
+    public IgniteUuid id() {
         return id;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
index ba43da7..90c26f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheSetProxy.java
@@ -57,6 +57,9 @@ public class GridCacheSetProxy<T> implements IgniteSet<T>, Externalizable {
     /** Busy lock. */
     private GridSpinBusyLock busyLock;
 
+    /** Check removed flag. */
+    private boolean rmvCheck;
+
     /**
      * Required by {@link Externalizable}.
      */
@@ -78,6 +81,13 @@ public class GridCacheSetProxy<T> implements IgniteSet<T>, Externalizable {
     }
 
     /**
+     * @return Set delegate.
+     */
+    public GridCacheSetImpl delegate() {
+        return delegate;
+    }
+
+    /**
      * Remove callback.
      */
     public void blockOnRemove() {
@@ -510,8 +520,43 @@ public class GridCacheSetProxy<T> implements IgniteSet<T>, Externalizable {
      * Enters busy state.
      */
     private void enterBusy() {
+        boolean rmvd;
+
+        if (rmvCheck) {
+            try {
+                rmvd = !delegate().checkHeader();
+            }
+            catch (IgniteCheckedException e) {
+                throw U.convertException(e);
+            }
+
+            rmvCheck = false;
+
+            if (rmvd) {
+                delegate.removed(true);
+
+                cctx.dataStructures().onRemoved(this);
+
+                throw removedError();
+            }
+        }
+
         if (!busyLock.enterBusy())
-            throw new IllegalStateException("Set has been removed from cache: " + delegate);
+            throw removedError();
+    }
+
+    /**
+     *
+     */
+    public void needCheckNotRemoved() {
+        rmvCheck = true;
+    }
+
+    /**
+     * @return Error.
+     */
+    private IllegalStateException removedError() {
+        return new IllegalStateException("Set has been removed from cache: " + delegate);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
index 48e9686..350068a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobProcessor.java
@@ -1413,7 +1413,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
      * @return {@code true} if node is dead, {@code false} is node is alive.
      */
     private boolean isDeadNode(UUID uid) {
-        return ctx.discovery().node(uid) == null || !ctx.discovery().pingNode(uid);
+        return ctx.discovery().node(uid) == null || !ctx.discovery().pingNodeNoError(uid);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
index d1ee5ad..3a309f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/job/GridJobWorker.java
@@ -863,7 +863,7 @@ public class GridJobWorker extends GridWorker implements GridTimeoutObject {
      * @return {@code true} if node is dead, {@code false} is node is alive.
      */
     private boolean isDeadNode(UUID uid) {
-        return ctx.discovery().node(uid) == null || !ctx.discovery().pingNode(uid);
+        return ctx.discovery().node(uid) == null || !ctx.discovery().pingNodeNoError(uid);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 0cbb77a..8639bc8 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
@@ -246,4 +246,11 @@ public interface GridQueryIndexing {
      * @return Backup filter.
      */
     public IndexingQueryFilter backupFilter(List<String> caches, AffinityTopologyVersion topVer, int[] parts);
+
+    /**
+     * Client disconnected callback.
+     *
+     * @param reconnectFut Reconnect future.
+     */
+    public void onDisconnected(IgniteFuture<?> reconnectFut);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 1ba1fae..f3ad4b2 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
@@ -234,6 +234,12 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             idx.stop();
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        if (idx != null)
+            idx.onDisconnected(reconnectFut);
+    }
+
     /**
      * @param cctx Cache context.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index bb451c7..78b09e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -124,7 +124,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
         cache = ctx.cache().utilityCache();
 
-        ctx.event().addLocalEventListener(topLsnr, EVTS_DISCOVERY);
+        if (!ctx.clientNode())
+            ctx.event().addLocalEventListener(topLsnr, EVTS_DISCOVERY);
 
         try {
             if (ctx.deploy().enabled())
@@ -165,7 +166,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
         busyLock.block();
 
-        ctx.event().removeLocalEventListener(topLsnr);
+        if (!ctx.clientNode())
+            ctx.event().removeLocalEventListener(topLsnr);
 
         if (cfgQryId != null)
             cache.context().continuousQueries().cancelInternalQuery(cfgQryId);
@@ -209,6 +211,27 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             log.debug("Stopped service processor.");
     }
 
+    /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        for (Map.Entry<String, GridServiceDeploymentFuture> e : depFuts.entrySet()) {
+            GridServiceDeploymentFuture fut = e.getValue();
+
+            fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                "Failed to deploy service, client node disconnected."));
+
+            depFuts.remove(e.getKey(), fut);
+        }
+
+        for (Map.Entry<String, GridFutureAdapter<?>> e : undepFuts.entrySet()) {
+            GridFutureAdapter fut = e.getValue();
+
+            fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                "Failed to undeploy service, client node disconnected."));
+
+            undepFuts.remove(e.getKey(), fut);
+        }
+    }
+
     /**
      * Validates service configuration.
      *
@@ -328,6 +351,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             return old;
         }
 
+        if (ctx.clientDisconnected()) {
+            fut.onDone(new IgniteClientDisconnectedCheckedException(ctx.cluster().clientReconnectFuture(),
+                "Failed to deploy service, client node disconnected."));
+
+            depFuts.remove(cfg.getName(), fut);
+        }
+
         while (true) {
             try {
                 GridServiceDeploymentKey key = new GridServiceDeploymentKey(cfg.getName());
@@ -646,10 +676,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                     }
                 }
                 else {
-                    Collection<ClusterNode> nodes =
-                        assigns.nodeFilter() == null ?
-                            ctx.discovery().nodes(topVer) :
-                            F.view(ctx.discovery().nodes(topVer), assigns.nodeFilter());
+                    Collection<ClusterNode> nodes = assigns.nodeFilter() == null ?
+                        ctx.discovery().nodes(topVer) :
+                        F.view(ctx.discovery().nodes(topVer), assigns.nodeFilter());
 
                     if (!nodes.isEmpty()) {
                         int size = nodes.size();
@@ -1019,7 +1048,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                     cache.getAndRemove(key);
                                 }
                                 catch (IgniteCheckedException ex) {
-                                    log.error("Failed to remove assignments for undeployed service: " + name, ex);
+                                    U.error(log, "Failed to remove assignments for undeployed service: " + name, ex);
                                 }
                             }
                         }
@@ -1164,7 +1193,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 }
                             }
                             catch (IgniteCheckedException ex) {
-                                log.error("Failed to clean up zombie assignments for service: " + name, ex);
+                                U.error(log, "Failed to clean up zombie assignments for service: " + name, ex);
                             }
                         }
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
index 8e13bc4..556beea 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
@@ -68,9 +68,15 @@ class GridServiceProxy<T> implements Serializable {
      * @param name Service name.
      * @param svc Service type class.
      * @param sticky Whether multi-node request should be done.
+     * @param ctx Context.
      */
-    @SuppressWarnings("unchecked") GridServiceProxy(ClusterGroup prj, String name, Class<? super T> svc,
-        boolean sticky, GridKernalContext ctx) {
+    @SuppressWarnings("unchecked")
+    GridServiceProxy(ClusterGroup prj,
+        String name,
+        Class<? super T> svc,
+        boolean sticky,
+        GridKernalContext ctx)
+    {
         this.prj = prj;
         this.ctx = ctx;
         hasLocNode = hasLocalNode(prj);
@@ -159,6 +165,9 @@ class GridServiceProxy<T> implements Serializable {
                 catch (RuntimeException | Error e) {
                     throw e;
                 }
+                catch (IgniteCheckedException e) {
+                    throw U.convertException(e);
+                }
                 catch (Exception e) {
                     throw new IgniteException(e);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
index d59a51d..d3caf5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskProcessor.java
@@ -76,8 +76,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
     private final LongAdder8 execTasks = new LongAdder8();
 
     /** */
-    private final ThreadLocal<Map<GridTaskThreadContextKey, Object>> thCtx =
-        new ThreadLocal<>();
+    private final ThreadLocal<Map<GridTaskThreadContextKey, Object>> thCtx = new ThreadLocal<>();
 
     /** */
     private final GridSpinReadWriteLock lock = new GridSpinReadWriteLock();
@@ -119,6 +118,24 @@ public class GridTaskProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
+    @Override public void onDisconnected(IgniteFuture<?> reconnectFut) throws IgniteCheckedException {
+        IgniteClientDisconnectedCheckedException err = disconnectedError(reconnectFut);
+
+        for (GridTaskWorker<?, ?> worker : tasks.values())
+            worker.finishTask(null, err);
+    }
+
+    /**
+     * @param reconnectFut Reconnect future.
+     * @return Client disconnected exception.
+     */
+    private IgniteClientDisconnectedCheckedException disconnectedError(@Nullable IgniteFuture<?> reconnectFut) {
+        return new IgniteClientDisconnectedCheckedException(
+            reconnectFut != null ? reconnectFut : ctx.cluster().clientReconnectFuture(),
+            "Failed to execute task, client node disconnected.");
+    }
+
+    /** {@inheritDoc} */
     @SuppressWarnings("TooBroadScope")
     @Override public void onKernalStop(boolean cancel) {
         lock.writeLock();
@@ -552,7 +569,7 @@ public class GridTaskProcessor extends GridProcessorAdapter {
         // Creates task session with task name and task version.
         GridTaskSessionImpl ses = ctx.session().createTaskSession(
             sesId,
-            ctx.config().getNodeId(),
+            ctx.localNodeId(),
             taskName,
             dep,
             taskCls == null ? null : taskCls.getName(),
@@ -597,25 +614,29 @@ public class GridTaskProcessor extends GridProcessorAdapter {
 
                 assert taskWorker0 == null : "Session ID is not unique: " + sesId;
 
-                if (dep.annotation(taskCls, ComputeTaskMapAsync.class) != null) {
-                    try {
-                        // Start task execution in another thread.
-                        if (sys)
-                            ctx.getSystemExecutorService().execute(taskWorker);
-                        else
-                            ctx.getExecutorService().execute(taskWorker);
-                    }
-                    catch (RejectedExecutionException e) {
-                        tasks.remove(sesId);
+                if (!ctx.clientDisconnected()) {
+                    if (dep.annotation(taskCls, ComputeTaskMapAsync.class) != null) {
+                        try {
+                            // Start task execution in another thread.
+                            if (sys)
+                                ctx.getSystemExecutorService().execute(taskWorker);
+                            else
+                                ctx.getExecutorService().execute(taskWorker);
+                        }
+                        catch (RejectedExecutionException e) {
+                            tasks.remove(sesId);
 
-                        release(dep);
+                            release(dep);
 
-                        handleException(new ComputeExecutionRejectedException("Failed to execute task " +
-                            "due to thread pool execution rejection: " + taskName, e), fut);
+                            handleException(new ComputeExecutionRejectedException("Failed to execute task " +
+                                "due to thread pool execution rejection: " + taskName, e), fut);
+                        }
                     }
+                    else
+                        taskWorker.run();
                 }
                 else
-                    taskWorker.run();
+                    taskWorker.finishTask(null, disconnectedError(null));
             }
         }
         else {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/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 eb5fa77..133a31f 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
@@ -1070,10 +1070,17 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                             PUBLIC_POOL);
                 }
                 catch (IgniteCheckedException e) {
-                    if (!isDeadNode(nodeId))
-                        U.error(log, "Failed to send cancel request to node (will ignore) [nodeId=" +
-                            nodeId + ", taskName=" + ses.getTaskName() +
-                            ", taskSesId=" + ses.getId() + ", jobSesId=" + res.getJobContext().getJobId() + ']', e);
+                    try {
+                        if (!isDeadNode(nodeId))
+                            U.error(log, "Failed to send cancel request to node (will ignore) [nodeId=" +
+                                nodeId + ", taskName=" + ses.getTaskName() +
+                                ", taskSesId=" + ses.getId() + ", jobSesId=" + res.getJobContext().getJobId() + ']', e);
+                    }
+                    catch (IgniteClientDisconnectedCheckedException e0) {
+                        if (log.isDebugEnabled())
+                            log.debug("Failed to send cancel request to node, client disconnected [nodeId=" +
+                                nodeId + ", taskName=" + ses.getTaskName() + ']');
+                    }
                 }
             }
         }
@@ -1169,24 +1176,39 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
             }
         }
         catch (IgniteCheckedException e) {
-            boolean deadNode = isDeadNode(res.getNode().id());
+            IgniteException fakeErr = null;
 
-            // Avoid stack trace if node has left grid.
-            if (deadNode)
-                U.warn(log, "Failed to send job request because remote node left grid (if failover is enabled, " +
-                    "will attempt fail-over to another node) [node=" + node + ", taskName=" + ses.getTaskName() +
-                    ", taskSesId=" + ses.getId() + ", jobSesId=" + res.getJobContext().getJobId() + ']');
-            else
-                U.error(log, "Failed to send job request: " + req, e);
+            try {
+                boolean deadNode = isDeadNode(res.getNode().id());
+
+                // Avoid stack trace if node has left grid.
+                if (deadNode) {
+                    U.warn(log, "Failed to send job request because remote node left grid (if failover is enabled, " +
+                        "will attempt fail-over to another node) [node=" + node + ", taskName=" + ses.getTaskName() +
+                        ", taskSesId=" + ses.getId() + ", jobSesId=" + res.getJobContext().getJobId() + ']');
+
+                    fakeErr = new ClusterTopologyException("Failed to send job due to node failure: " + node, e);
+                }
+                else
+                    U.error(log, "Failed to send job request: " + req, e);
+
+            }
+            catch (IgniteClientDisconnectedCheckedException e0) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send job request, client disconnected [node=" + node +
+                        ", taskName=" + ses.getTaskName() + ", taskSesId=" + ses.getId() + ", jobSesId=" +
+                        res.getJobContext().getJobId() + ']');
+
+                fakeErr = U.convertException(e0);
+            }
 
             GridJobExecuteResponse fakeRes = new GridJobExecuteResponse(node.id(), ses.getId(),
                 res.getJobContext().getJobId(), null, null, null, null, null, null, false);
 
-            if (deadNode)
-                fakeRes.setFakeException(new ClusterTopologyException("Failed to send job due to node failure: " +
-                    node, e));
-            else
-                fakeRes.setFakeException(U.convertException(e));
+            if (fakeErr == null)
+                fakeErr = U.convertException(e);
+
+            fakeRes.setFakeException(fakeErr);
 
             onResponse(fakeRes);
         }
@@ -1345,8 +1367,9 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
      *
      * @param uid UID of node to check.
      * @return {@code true} if node is dead, {@code false} is node is alive.
+     * @throws IgniteClientDisconnectedCheckedException if ping failed when client disconnected.
      */
-    private boolean isDeadNode(UUID uid) {
+    private boolean isDeadNode(UUID uid) throws IgniteClientDisconnectedCheckedException {
         return ctx.discovery().node(uid) == null || !ctx.discovery().pingNode(uid);
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index f457d6c..66eb596 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -626,6 +626,15 @@ public abstract class IgniteUtils {
             }
         });
 
+        m.put(IgniteClientDisconnectedCheckedException.class, new C1<IgniteCheckedException, IgniteException>() {
+            @Override public IgniteException apply(IgniteCheckedException e) {
+                return new IgniteClientDisconnectedException(
+                    ((IgniteClientDisconnectedCheckedException)e).reconnectFuture(),
+                    e.getMessage(),
+                    e);
+            }
+        });
+
         return m;
     }
 
@@ -673,6 +682,25 @@ public abstract class IgniteUtils {
      * @return Ignite runtime exception.
      */
     public static IgniteException convertException(IgniteCheckedException e) {
+        IgniteClientDisconnectedException e0 = e.getCause(IgniteClientDisconnectedException.class);
+
+        if (e0 != null) {
+            assert e0.reconnectFuture() != null : e0;
+
+            throw e0;
+        }
+
+        IgniteClientDisconnectedCheckedException disconnectedErr =
+            e instanceof IgniteClientDisconnectedCheckedException ?
+            (IgniteClientDisconnectedCheckedException)e
+            : e.getCause(IgniteClientDisconnectedCheckedException.class);
+
+        if (disconnectedErr != null) {
+            assert disconnectedErr.reconnectFuture() != null : disconnectedErr;
+
+            e = disconnectedErr;
+        }
+
         C1<IgniteCheckedException, IgniteException> converter = exceptionConverters.get(e.getClass());
 
         if (converter != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
index c935c4a..a4f7804 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryClientEndpoint.java
@@ -59,8 +59,9 @@ public class IpcSharedMemoryClientEndpoint implements IpcEndpoint {
      * @param outSpace Out space.
      * @param parent Parent logger.
      */
-    public IpcSharedMemoryClientEndpoint(IpcSharedMemorySpace inSpace, IpcSharedMemorySpace outSpace,
-                                         IgniteLogger parent) {
+    public IpcSharedMemoryClientEndpoint(IpcSharedMemorySpace inSpace,
+        IpcSharedMemorySpace outSpace,
+        IgniteLogger parent) {
         assert inSpace != null;
         assert outSpace != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index 6f544e0..f3bcab0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -1570,6 +1570,7 @@ public class GridFunc {
      * @param <T> Type of the collection.
      * @return Light-weight view on given collection with provided predicate.
      */
+    @SafeVarargs
     public static <T> Collection<T> view(@Nullable final Collection<T> c,
         @Nullable final IgnitePredicate<? super T>... p) {
         if (isEmpty(c) || isAlwaysFalse(p))
@@ -2706,6 +2707,7 @@ public class GridFunc {
      * @param <T> Type of the free variable, i.e. the element the predicate is called on.
      * @return Negated predicate.
      */
+    @SafeVarargs
     public static <T> IgnitePredicate<T> not(@Nullable final IgnitePredicate<? super T>... p) {
         return isAlwaysFalse(p) ? F.<T>alwaysTrue() : isAlwaysTrue(p) ? F.<T>alwaysFalse() : new P1<T>() {
             @Override public boolean apply(T t) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpi.java
index 968d88d..0f6ed5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpi.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi;
 
+import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -106,4 +107,18 @@ public interface IgniteSpi {
      * @throws IgniteSpiException Thrown in case of any error during SPI stop.
      */
     public void spiStop() throws IgniteSpiException;
+
+    /**
+     * Client node disconnected callback.
+     *
+     * @param reconnectFut Future that will be completed when client reconnected.
+     */
+    public void onClientDisconnected(IgniteFuture<?> reconnectFut);
+
+    /**
+     * Client node reconnected callback.
+     *
+     * @param clusterRestarted {@code True} if all cluster nodes restarted while client was disconnected.
+     */
+    public void onClientReconnected(boolean clusterRestarted);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
index 5e557bd..07b39bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.resources.*;
@@ -58,9 +59,6 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
     /** Ignite instance. */
     protected Ignite ignite;
 
-    /** Local node id. */
-    protected UUID nodeId;
-
     /** Grid instance name. */
     protected String gridName;
 
@@ -73,6 +71,9 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
     /** Discovery listener. */
     private GridLocalEventListener paramsLsnr;
 
+    /** Local node. */
+    private ClusterNode locNode;
+
     /**
      * Creates new adapter and initializes it from the current (this) class.
      * SPI name will be initialized to the simple name of the class
@@ -111,7 +112,19 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
 
     /** {@inheritDoc} */
     @Override public UUID getLocalNodeId() {
-        return nodeId;
+        return ignite.cluster().localNode().id();
+    }
+
+    /**
+     * @return Local node.
+     */
+    protected ClusterNode getLocalNode() {
+        if (locNode != null)
+            return locNode;
+
+        locNode = getSpiContext().localNode();
+
+        return locNode;
     }
 
     /** {@inheritDoc} */
@@ -194,17 +207,27 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         spiCtx = new GridDummySpiContext(locNode, true, spiCtx);
     }
 
+    /** {@inheritDoc} */
+    @Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onClientReconnected(boolean clusterRestarted) {
+        // No-op.
+    }
+
     /**
      * Inject ignite instance.
+     *
+     * @param ignite Ignite instance.
      */
     @IgniteInstanceResource
     protected void injectResources(Ignite ignite) {
         this.ignite = ignite;
 
-        if (ignite != null) {
-            nodeId = ignite.configuration().getNodeId();
+        if (ignite != null)
             gridName = ignite.name();
-        }
     }
 
     /**


[19/50] [abbrv] ignite git commit: # ignite-901 client reconnect support

Posted by nt...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
new file mode 100644
index 0000000..be3234d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -0,0 +1,1202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import junit.framework.*;
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.transactions.*;
+
+import javax.cache.*;
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.transactions.TransactionConcurrency.*;
+import static org.apache.ignite.transactions.TransactionIsolation.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    private static final int SRV_CNT = 3;
+
+    /** */
+    private static final String STATIC_CACHE = "static-cache";
+
+    /** */
+    private UUID nodeId;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TestCommunicationSpi commSpi = new TestCommunicationSpi();
+
+        commSpi.setSharedMemoryPort(-1);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setNetworkTimeout(5000);
+
+        if (nodeId != null) {
+            cfg.setNodeId(nodeId);
+
+            nodeId = null;
+        }
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setName(STATIC_CACHE);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        startGrids(SRV_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect() throws Exception {
+        clientMode = true;
+
+        IgniteEx client = startGrid(SRV_CNT);
+
+        final TestTcpDiscoverySpi clientSpi = spi(client);
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final IgniteCache<Object, Object> cache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        final IgniteCache<Object, Object> staticCache = client.cache(STATIC_CACHE);
+
+        staticCache.put(1, 1);
+
+        assertEquals(1, staticCache.get(1));
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setName("nearCache");
+
+        final IgniteCache<Object, Object> nearCache = client.getOrCreateCache(ccfg, new NearCacheConfiguration<>());
+
+        nearCache.put(1, 1);
+
+        assertEquals(1, nearCache.localPeek(1));
+
+        cache.put(1, 1);
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        log.info("Block reconnect.");
+
+        clientSpi.writeLatch = new CountDownLatch(1);
+
+        final AtomicReference<IgniteInternalFuture> blockPutRef = new AtomicReference<>();
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    assertEquals(1, reconnectLatch.getCount());
+
+                    blockPutRef.set(GridTestUtils.runAsync(new Callable() {
+                        @Override public Object call() throws Exception {
+                            log.info("Start put.");
+
+                            try {
+                                cache.put(2, 2);
+
+                                fail();
+                            }
+                            catch (CacheException e) {
+                                log.info("Expected exception: " + e);
+
+                                IgniteClientDisconnectedException e0 = (IgniteClientDisconnectedException) e.getCause();
+
+                                e0.reconnectFuture().get();
+                            }
+
+                            cache.put(2, 2);
+
+                            log.info("Finish put.");
+
+                            return null;
+                        }
+                    }));
+
+                    disconnectLatch.countDown();
+                } else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    assertEquals(0, disconnectLatch.getCount());
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        log.info("Fail client.");
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(disconnectLatch);
+
+        IgniteInternalFuture putFut = blockPutRef.get();
+
+        assertNotDone(putFut);
+
+        U.sleep(5000);
+
+        assertNotDone(putFut);
+
+        log.info("Allow reconnect.");
+
+        clientSpi.writeLatch.countDown();
+
+        assertTrue(reconnectLatch.await(5000, MILLISECONDS));
+
+        checkCacheDiscoveryData(srv, client, null, true, true, false);
+
+        checkCacheDiscoveryData(srv, client, "nearCache", true, true, true);
+
+        checkCacheDiscoveryData(srv, client, STATIC_CACHE, true, true, false);
+
+        assertEquals(1, cache.get(1));
+
+        putFut.get();
+
+        assertEquals(2, cache.get(2));
+
+        cache.put(3, 3);
+
+        assertEquals(3, cache.get(3));
+
+        assertNull(nearCache.localPeek(1));
+
+        staticCache.put(10, 10);
+
+        assertEquals(10, staticCache.get(10));
+
+        nearCache.put(20, 20);
+
+        srv.cache(nearCache.getName()).put(20, 21);
+
+        assertEquals(21, nearCache.localPeek(20));
+
+        this.clientMode = false;
+
+        IgniteEx srv2 = startGrid(SRV_CNT + 1);
+
+        Integer key = primaryKey(srv2.cache(null));
+
+        cache.put(key, 4);
+
+        assertEquals(4, cache.get(key));
+
+        checkCacheDiscoveryData(srv2, client, null, true, true, false);
+
+        checkCacheDiscoveryData(srv2, client, "nearCache", true, true, true);
+
+        checkCacheDiscoveryData(srv2, client, STATIC_CACHE, true, true, false);
+
+        staticCache.put(20, 20);
+
+        assertEquals(20, staticCache.get(20));
+
+        srv.cache(nearCache.getName()).put(20, 22);
+
+        assertEquals(22, nearCache.localPeek(20));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectTransactions() throws Exception {
+        clientMode = true;
+
+        IgniteEx client = startGrid(SRV_CNT);
+
+        Ignite srv = clientRouter(client);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+
+        IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
+
+        final IgniteTransactions txs = client.transactions();
+
+        final Transaction tx = txs.txStart(OPTIMISTIC, REPEATABLE_READ);
+
+        cache.put(1, 1);
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                try {
+                    tx.commit();
+
+                    fail();
+                } catch (IgniteClientDisconnectedException e) {
+                    log.info("Expected error: " + e);
+
+                    assertNotNull(e.reconnectFuture());
+                }
+
+                try {
+                    txs.txStart();
+
+                    fail();
+                } catch (IgniteClientDisconnectedException e) {
+                    log.info("Expected error: " + e);
+
+                    assertNotNull(e.reconnectFuture());
+                }
+            }
+        });
+
+        assertNull(txs.tx());
+
+        try (Transaction tx0 = txs.txStart(OPTIMISTIC, REPEATABLE_READ)) {
+            cache.put(1, 1);
+
+            assertEquals(1, cache.get(1));
+
+            tx0.commit();
+        }
+
+        try (Transaction tx0 = txs.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+            cache.put(2, 2);
+
+            assertEquals(2, cache.get(2));
+
+            tx0.commit();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectTransactionInProgress1() throws Exception {
+        clientMode = true;
+
+        IgniteEx client = startGrid(SRV_CNT);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
+
+        reconnectTransactionInProgress1(client, OPTIMISTIC, cache);
+
+        reconnectTransactionInProgress1(client, PESSIMISTIC, cache);
+    }
+
+    /**
+     * @param client Client.
+     * @param txConcurrency Transaction concurrency mode.
+     * @param cache Cache.
+     * @throws Exception If failed.
+     */
+    private void reconnectTransactionInProgress1(IgniteEx client,
+        final TransactionConcurrency txConcurrency,
+        final IgniteCache<Object, Object> cache)
+        throws Exception
+    {
+        Ignite srv = clientRouter(client);
+
+        final TestTcpDiscoverySpi clientSpi = spi(client);
+        final TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        log.info("Block reconnect.");
+
+        clientSpi.writeLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                } else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        final IgniteTransactions txs = client.transactions();
+
+        final CountDownLatch afterPut1 = new CountDownLatch(1);
+
+        final CountDownLatch afterPut2 = new CountDownLatch(1);
+
+        final CountDownLatch putFailed = new CountDownLatch(1);
+
+        IgniteInternalFuture<Boolean> fut = GridTestUtils.runAsync(new Callable<Boolean>() {
+            @Override public Boolean call() throws Exception {
+                try {
+                    log.info("Start tx1: " + txConcurrency);
+
+                    try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
+                        cache.put(1, 1);
+
+                        afterPut1.countDown();
+
+                        afterPut2.await();
+
+                        cache.put(2, 2);
+
+                        fail();
+                    }
+                    catch (CacheException e) {
+                        log.info("Expected exception: " + e);
+
+                        putFailed.countDown();
+
+                        IgniteClientDisconnectedException e0 = (IgniteClientDisconnectedException)e.getCause();
+
+                        e0.reconnectFuture().get();
+                    }
+
+                    log.info("Start tx2: " + txConcurrency);
+
+                    try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
+                        cache.put(1, 1);
+
+                        cache.put(2, 2);
+
+                        tx.commit();
+                    }
+
+                    assertEquals(1, cache.get(1));
+                    assertEquals(2, cache.get(2));
+
+                    try (Transaction tx = txs.txStart(txConcurrency, REPEATABLE_READ)) {
+                        cache.put(3, 3);
+
+                        cache.put(4, 4);
+
+                        tx.commit();
+                    }
+
+                    assertEquals(1, cache.get(1));
+                    assertEquals(2, cache.get(2));
+                    assertEquals(3, cache.get(3));
+                    assertEquals(4, cache.get(4));
+
+                    cache.removeAll();
+
+                    return true;
+                }
+                catch (AssertionFailedError e) {
+                    throw e;
+                }
+                catch (Throwable e) {
+                    log.error("Unexpected error", e);
+
+                    fail("Unexpected error: " + e);
+
+                    return false;
+                }
+            }
+        });
+
+        assertTrue(afterPut1.await(5000, MILLISECONDS));
+
+        assertNotDone(fut);
+
+        srvSpi.failNode(client.localNode().id(), null);
+
+        waitReconnectEvent(disconnectLatch);
+
+        afterPut2.countDown();
+
+        assertTrue(putFailed.await(5000, MILLISECONDS));
+
+        clientSpi.writeLatch.countDown();
+
+        waitReconnectEvent(reconnectLatch);
+
+        assertTrue(fut.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectTransactionInProgress2() throws Exception {
+        clientMode = true;
+
+        final IgniteEx client = startGrid(SRV_CNT);
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        txInProgressFails(client, ccfg, GridNearTxPrepareResponse.class, OPTIMISTIC, 1);
+
+        txInProgressFails(client, ccfg, GridNearTxPrepareResponse.class, PESSIMISTIC, 2);
+
+        txInProgressFails(client, ccfg, GridNearTxFinishResponse.class, OPTIMISTIC, 3);
+
+        txInProgressFails(client, ccfg, GridNearTxFinishResponse.class, PESSIMISTIC, 4);
+
+        txInProgressFails(client, ccfg, GridNearLockResponse.class, PESSIMISTIC, 5);
+    }
+
+    /**
+     * @param client Client.
+     * @param ccfg Cache configuration.
+     * @param msgToBlock Message to block.
+     * @param txConcurrency Transaction concurrency mode.
+     * @param key Key.
+     * @throws Exception If failed.
+     */
+    private void txInProgressFails(final IgniteEx client,
+        final CacheConfiguration<Object, Object> ccfg,
+        Class<?> msgToBlock,
+        final TransactionConcurrency txConcurrency,
+        final Integer key) throws Exception {
+        log.info("Test tx failure [msg=" + msgToBlock + ", txMode=" + txConcurrency + ", key=" + key + ']');
+
+        checkOperationInProgressFails(client, ccfg, msgToBlock,
+            new CI1<IgniteCache<Object, Object>>() {
+                @Override public void apply(IgniteCache<Object, Object> cache) {
+                    try (Transaction tx = client.transactions().txStart(txConcurrency, REPEATABLE_READ)) {
+                        log.info("Put1: " + key);
+
+                        cache.put(key, key);
+
+                        Integer key2 = key + 1;
+
+                        log.info("Put2: " + key2);
+
+                        cache.put(key2, key2);
+
+                        log.info("Commit [key1=" + key + ", key2=" + key2 + ']');
+
+                        tx.commit();
+                    }
+                }
+            }
+        );
+
+        IgniteCache<Object, Object> cache = client.cache(ccfg.getName());
+
+        assertEquals(key, cache.get(key));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectExchangeInProgress() throws Exception {
+        clientMode = true;
+
+        IgniteEx client = startGrid(SRV_CNT);
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        TestCommunicationSpi coordCommSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi();
+
+        coordCommSpi.blockMessages(GridDhtPartitionsFullMessage.class, client.localNode().id());
+
+        clientMode = false;
+
+        startGrid(SRV_CNT + 1);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        assertTrue(reconnectLatch.await(5000, MILLISECONDS));
+
+        try {
+            coordCommSpi.stopBlock(true);
+
+            fail();
+        }
+        catch (IgniteException e) {
+            log.info("Expected error: " + e);
+        }
+
+        CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+        ccfg.setName("newCache");
+
+        ccfg.setCacheMode(REPLICATED);
+
+        log.info("Start new cache.");
+
+        IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
+
+        cache.put(1, 1);
+
+        assertEquals(1, cache.get(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectInitialExchangeInProgress() throws Exception {
+        final UUID clientId = UUID.randomUUID();
+
+        Ignite srv = grid(0);
+
+        final CountDownLatch joinLatch = new CountDownLatch(1);
+
+        srv.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_NODE_JOINED && ((DiscoveryEvent)evt).eventNode().id().equals(clientId)) {
+                    info("Client joined: " + evt);
+
+                    joinLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_NODE_JOINED);
+
+        TestCommunicationSpi srvCommSpi = (TestCommunicationSpi)srv.configuration().getCommunicationSpi();
+
+        srvCommSpi.blockMessages(GridDhtPartitionsFullMessage.class, clientId);
+
+        clientMode = true;
+
+        nodeId = clientId;
+
+        IgniteInternalFuture<Boolean> fut = GridTestUtils.runAsync(new Callable<Boolean>() {
+            @Override public Boolean call() throws Exception {
+                try {
+                    Ignition.start(getConfiguration(getTestGridName(SRV_CNT)));
+
+                    fail();
+
+                    return false;
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    log.info("Expected start error: " + e);
+
+                    try {
+                        e.reconnectFuture().get();
+
+                        fail();
+                    }
+                    catch (IgniteException e0) {
+                        log.info("Expected future error: " + e0);
+                    }
+
+                    return true;
+                }
+                catch (Throwable e) {
+                    log.error("Unexpected error: " + e, e);
+
+                    throw e;
+                }
+            }
+        });
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        assertTrue(joinLatch.await(5000, MILLISECONDS));
+
+        U.sleep(1000);
+
+        assertNotDone(fut);
+
+        srvSpi.failNode(clientId, null);
+
+        srvCommSpi.stopBlock(false);
+
+        assertTrue(fut.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectOperationInProgress() throws Exception {
+        clientMode = true;
+
+        IgniteEx client = startGrid(SRV_CNT);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED)
+                    info("Client disconnected: " + evt);
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED)
+                    info("Client reconnected: " + evt);
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        IgniteInClosure<IgniteCache<Object, Object>> putOp = new CI1<IgniteCache<Object, Object>>() {
+            @Override public void apply(IgniteCache<Object, Object> cache) {
+                cache.put(1, 1);
+            }
+        };
+
+        IgniteInClosure<IgniteCache<Object, Object>> getOp = new CI1<IgniteCache<Object, Object>>() {
+            @Override public void apply(IgniteCache<Object, Object> cache) {
+                cache.get(1);
+            }
+        };
+
+        int cnt = 0;
+
+        for (CacheAtomicityMode atomicityMode : CacheAtomicityMode.values()) {
+            CacheAtomicWriteOrderMode[] writeOrders =
+                atomicityMode == ATOMIC ? CacheAtomicWriteOrderMode.values() :
+                new CacheAtomicWriteOrderMode[]{CacheAtomicWriteOrderMode.CLOCK};
+
+            for (CacheAtomicWriteOrderMode writeOrder : writeOrders) {
+                for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
+                    CacheConfiguration<Object, Object> ccfg = new CacheConfiguration<>();
+
+                    ccfg.setAtomicityMode(atomicityMode);
+
+                    ccfg.setAtomicWriteOrderMode(writeOrder);
+
+                    ccfg.setName("cache-" + cnt++);
+
+                    ccfg.setWriteSynchronizationMode(syncMode);
+
+                    if (syncMode != FULL_ASYNC) {
+                        Class<?> cls = (ccfg.getAtomicityMode() == ATOMIC) ?
+                            GridNearAtomicUpdateResponse.class : GridNearTxPrepareResponse.class;
+
+                        log.info("Test cache put [atomicity=" + atomicityMode +
+                            ", writeOrder=" + writeOrder +
+                            ", syncMode=" + syncMode + ']');
+
+                        checkOperationInProgressFails(client, ccfg, cls, putOp);
+
+                        client.destroyCache(ccfg.getName());
+                    }
+
+                    log.info("Test cache get [atomicity=" + atomicityMode + ", syncMode=" + syncMode + ']');
+
+                    checkOperationInProgressFails(client, ccfg, GridNearGetResponse.class, getOp);
+
+                    client.destroyCache(ccfg.getName());
+                }
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectCacheDestroyed() throws Exception {
+        clientMode = true;
+
+        final IgniteEx client = startGrid(SRV_CNT);
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(client);
+
+        final IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srv.destroyCache(null);
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return clientCache.get(1);
+            }
+        }, IllegalStateException.class, null);
+
+        checkCacheDiscoveryData(srv, client, null, false, false, false);
+
+        IgniteCache<Object, Object> clientCache0 = client.getOrCreateCache(new CacheConfiguration<>());
+
+        checkCacheDiscoveryData(srv, client, null, true, true, false);
+
+        clientCache0.put(1, 1);
+
+        assertEquals(1, clientCache0.get(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectCacheDestroyedAndCreated() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(SRV_CNT);
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(client);
+
+        final IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        assertEquals(ATOMIC,
+            clientCache.getConfiguration(CacheConfiguration.class).getAtomicityMode());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srv.destroyCache(null);
+
+                CacheConfiguration ccfg = new CacheConfiguration();
+
+                ccfg.setAtomicityMode(TRANSACTIONAL);
+
+                srv.getOrCreateCache(ccfg);
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return clientCache.get(1);
+            }
+        }, IllegalStateException.class, null);
+
+        checkCacheDiscoveryData(srv, client, null, true, false, false);
+
+        IgniteCache<Object, Object> clientCache0 = client.cache(null);
+
+        checkCacheDiscoveryData(srv, client, null, true, true, false);
+
+        assertEquals(TRANSACTIONAL,
+            clientCache0.getConfiguration(CacheConfiguration.class).getAtomicityMode());
+
+        clientCache0.put(1, 1);
+
+        assertEquals(1, clientCache0.get(1));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectMarshallerCache() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(SRV_CNT);
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(client);
+
+        final IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+        final IgniteCache<Object, Object> srvCache = srv.cache(null);
+
+        assertNotNull(srvCache);
+
+        clientCache.put(1, new TestClass1());
+        srvCache.put(2, new TestClass2());
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertNotNull(srvCache.get(1));
+                assertNotNull(srvCache.get(2));
+
+                srvCache.put(3, new TestClass3());
+            }
+        });
+
+        srvCache.put(4, new TestClass4());
+
+        assertNotNull(clientCache.get(1));
+        assertNotNull(clientCache.get(2));
+        assertNotNull(clientCache.get(3));
+        assertNotNull(clientCache.get(4));
+
+        clientCache.put(5, new TestClass5());
+
+        assertNotNull(srvCache.get(5));
+        assertNotNull(clientCache.get(5));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectClusterRestart() throws Exception {
+        clientMode = true;
+
+        final Ignite client = startGrid(SRV_CNT);
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final CountDownLatch disconnectLatch = new CountDownLatch(1);
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        final IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        clientCache.put(1, new TestClass1());
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                } else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        for (int i = 0; i < SRV_CNT; i++)
+            stopGrid(i);
+
+        assertTrue(disconnectLatch.await(30_000, MILLISECONDS));
+
+        clientMode = false;
+
+        Ignite srv = startGrid(0);
+
+        assertTrue(reconnectLatch.await(10_000, MILLISECONDS));
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return clientCache.get(1);
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteCache<Object, Object> srvCache = srv.getOrCreateCache(new CacheConfiguration<>());
+
+        srvCache.put(1, new TestClass1());
+        srvCache.put(2, new TestClass2());
+
+        IgniteCache<Object, Object> clientCache2 = client.cache(null);
+
+        assertNotNull(clientCache2);
+
+        assertNotNull(clientCache2.get(1));
+        assertNotNull(clientCache2.get(2));
+    }
+
+    /**
+     *
+     */
+    static class TestClass1 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass2 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass3 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass4 implements Serializable {}
+
+    /**
+     *
+     */
+    static class TestClass5 implements Serializable {}
+
+    /**
+     * @param client Client.
+     * @param ccfg Cache configuration.
+     * @param msgToBlock Message to block.
+     * @param c Cache operation closure.
+     * @throws Exception If failed.
+     */
+    private void checkOperationInProgressFails(IgniteEx client,
+        final CacheConfiguration<Object, Object> ccfg,
+        Class<?> msgToBlock,
+        final IgniteInClosure<IgniteCache<Object, Object>> c)
+        throws Exception
+    {
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final IgniteCache<Object, Object> cache = client.getOrCreateCache(ccfg);
+
+        for (int i = 0; i < SRV_CNT; i++) {
+            TestCommunicationSpi srvCommSpi = (TestCommunicationSpi)grid(i).configuration().getCommunicationSpi();
+
+            srvCommSpi.blockMessages(msgToBlock, client.localNode().id());
+        }
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                IgniteClientDisconnectedException e0 = null;
+
+                try {
+                    c.apply(cache);
+
+                    fail();
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    log.info("Expected exception: " + e);
+
+                    e0 = e;
+                }
+                catch (CacheException e) {
+                    log.info("Expected exception: " + e);
+
+                    assertTrue("Unexpected cause: " + e.getCause(),
+                        e.getCause() instanceof IgniteClientDisconnectedException);
+
+                    e0 = (IgniteClientDisconnectedException)e.getCause();
+                }
+
+                assertNotNull(e0);
+                assertNotNull(e0.reconnectFuture());
+
+                e0.reconnectFuture().get();
+
+                c.apply(cache);
+
+                return null;
+            }
+        });
+
+        Thread.sleep(1000);
+
+        assertNotDone(fut);
+
+        log.info("Fail client: " + client.localNode().id());
+
+        srvSpi.failNode(client.localNode().id(), null);
+
+        fut.get();
+
+        for (int i = 0; i < SRV_CNT; i++)
+            ((TestCommunicationSpi)grid(i).configuration().getCommunicationSpi()).stopBlock(false);
+
+        cache.put(1, 1);
+
+        assertEquals(1, cache.get(1));
+    }
+
+    /**
+     * @param srv Server node.
+     * @param client Client node.
+     * @param cacheName Cache name.
+     * @param cacheExists Cache exists flag.
+     * @param clientCache {@code True} if client node has client cache.
+     * @param clientNear {@code True} if client node has near-enabled client cache.
+     */
+    private void checkCacheDiscoveryData(Ignite srv,
+        Ignite client,
+        String cacheName,
+        boolean cacheExists,
+        boolean clientCache,
+        boolean clientNear)
+    {
+        GridDiscoveryManager srvDisco = ((IgniteKernal)srv).context().discovery();
+        GridDiscoveryManager clientDisco = ((IgniteKernal)client).context().discovery();
+
+        ClusterNode srvNode = ((IgniteKernal)srv).localNode();
+        ClusterNode clientNode = ((IgniteKernal)client).localNode();
+
+        assertFalse(srvDisco.cacheAffinityNode(clientNode, cacheName));
+        assertFalse(clientDisco.cacheAffinityNode(clientNode, cacheName));
+
+        assertEquals(cacheExists, srvDisco.cacheAffinityNode(srvNode, cacheName));
+
+        if (clientNear)
+            assertTrue(srvDisco.cacheNearNode(clientNode, cacheName));
+        else
+            assertEquals(clientCache, srvDisco.cacheClientNode(clientNode, cacheName));
+
+        assertEquals(cacheExists, clientDisco.cacheAffinityNode(srvNode, cacheName));
+
+        if (clientNear)
+            assertTrue(clientDisco.cacheNearNode(clientNode, cacheName));
+        else
+            assertEquals(clientCache, clientDisco.cacheClientNode(clientNode, cacheName));
+
+        if (cacheExists) {
+            if (clientCache || clientNear) {
+                assertTrue(client.cluster().forClientNodes(cacheName).nodes().contains(clientNode));
+                assertTrue(srv.cluster().forClientNodes(cacheName).nodes().contains(clientNode));
+            }
+            else {
+                assertFalse(client.cluster().forClientNodes(cacheName).nodes().contains(clientNode));
+                assertFalse(srv.cluster().forClientNodes(cacheName).nodes().contains(clientNode));
+            }
+        }
+        else {
+            assertTrue(client.cluster().forClientNodes(cacheName).nodes().isEmpty());
+            assertTrue(srv.cluster().forClientNodes(cacheName).nodes().isEmpty());
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** */
+        @LoggerResource
+        private IgniteLogger log;
+
+        /** */
+        private List<T2<ClusterNode, GridIoMessage>> blockedMsgs = new ArrayList<>();
+
+        /** */
+        private Map<Class<?>, Set<UUID>> blockCls = new HashMap<>();
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg) throws IgniteSpiException {
+            if (msg instanceof GridIoMessage) {
+                Object msg0 = ((GridIoMessage)msg).message();
+
+                synchronized (this) {
+                    Set<UUID> blockNodes = blockCls.get(msg0.getClass());
+
+                    if (F.contains(blockNodes, node.id())) {
+                        log.info("Block message [node=" + node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME) +
+                            ", msg=" + msg0 + ']');
+
+                        blockedMsgs.add(new T2<>(node, (GridIoMessage)msg));
+
+                        return;
+                    }
+                }
+            }
+
+            super.sendMessage(node, msg);
+        }
+
+        /**
+         * @param cls Message class.
+         * @param nodeId Node ID.
+         */
+        void blockMessages(Class<?> cls, UUID nodeId) {
+            synchronized (this) {
+                Set<UUID> set = blockCls.get(cls);
+
+                if (set == null) {
+                    set = new HashSet<>();
+
+                    blockCls.put(cls, set);
+                }
+
+                set.add(nodeId);
+            }
+        }
+
+        /**
+         * @param snd Send messages flag.
+         */
+        void stopBlock(boolean snd) {
+            synchronized (this) {
+                blockCls.clear();
+
+                if (snd) {
+                    for (T2<ClusterNode, GridIoMessage> msg : blockedMsgs) {
+                        ClusterNode node = msg.get1();
+
+                        log.info("Send blocked message: [node=" + node.attribute(IgniteNodeAttributes.ATTR_GRID_NAME) +
+                            ", msg=" + msg.get2().message() + ']');
+
+                        super.sendMessage(msg.get1(), msg.get2());
+                    }
+                }
+
+                blockedMsgs.clear();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
new file mode 100644
index 0000000..ed811d9
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCollectionsTest.java
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
+import org.apache.ignite.internal.processors.cache.distributed.near.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectCollectionsTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueueReconnect() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        queueReconnect(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        queueReconnect(colCfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueueReconnectRemoved() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        queueReconnectRemoved(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        queueReconnectRemoved(colCfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueueReconnectInProgress() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        queueReconnectInProgress(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        queueReconnectInProgress(colCfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetReconnect() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        setReconnect(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        setReconnect(colCfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetReconnectRemoved() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        setReconnectRemove(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        setReconnectRemove(colCfg);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSetReconnectInProgress() throws Exception {
+        CollectionConfiguration colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(ATOMIC);
+
+        setReconnectInProgress(colCfg);
+
+        colCfg = new CollectionConfiguration();
+
+        colCfg.setCacheMode(PARTITIONED);
+        colCfg.setAtomicityMode(TRANSACTIONAL);
+
+        setReconnectInProgress(colCfg);
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void setReconnect(CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final String setName = "set-" + colCfg.getAtomicityMode();
+
+        IgniteSet<String> clientSet = client.set(setName, colCfg);
+
+        final IgniteSet<String> srvSet = srv.set(setName, null);
+
+        assertTrue(clientSet.add("1"));
+
+        assertFalse(srvSet.add("1"));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertTrue(srvSet.add("2"));
+            }
+        });
+
+        assertFalse(clientSet.add("2"));
+
+        assertTrue(clientSet.remove("2"));
+
+        assertFalse(srvSet.contains("2"));
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void setReconnectRemove(CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(client);
+
+        final String setName = "set-rm-" + colCfg.getAtomicityMode();
+
+        final IgniteSet<String> clientSet = client.set(setName, colCfg);
+
+        final IgniteSet<String> srvSet = srv.set(setName, null);
+
+        assertTrue(clientSet.add("1"));
+
+        assertFalse(srvSet.add("1"));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvSet.close();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                clientSet.add("fail");
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteSet<String> newClientSet = client.set(setName, colCfg);
+
+        IgniteSet<String> newSrvSet = srv.set(setName, null);
+
+        assertTrue(newClientSet.add("1"));
+
+        assertFalse(newSrvSet.add("1"));
+
+        newSrvSet.close();
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void setReconnectInProgress(final CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        final Ignite srv = clientRouter(client);
+
+        final String setName = "set-in-progress-" + colCfg.getAtomicityMode();
+
+        final IgniteSet<String> clientSet = client.set(setName, colCfg);
+
+        final IgniteSet<String> srvSet = srv.set(setName, null);
+
+        assertTrue(clientSet.add("1"));
+
+        assertFalse(srvSet.add("1"));
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        if (colCfg.getAtomicityMode() == ATOMIC)
+            commSpi.blockMessage(GridNearAtomicUpdateResponse.class);
+        else
+            commSpi.blockMessage(GridNearTxPrepareResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    for (int i = 0; i < 100; i++)
+                        clientSet.add("2");
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+
+        assertTrue(clientSet.add("3"));
+
+        assertFalse(srvSet.add("3"));
+
+        srvSet.close();
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void queueReconnect(CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final String setName = "queue-" + colCfg.getAtomicityMode();
+
+        IgniteQueue<String> clientQueue = client.queue(setName, 10, colCfg);
+
+        final IgniteQueue<String> srvQueue = srv.queue(setName, 10, null);
+
+        assertTrue(clientQueue.offer("1"));
+
+        assertTrue(srvQueue.contains("1"));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                assertTrue(srvQueue.add("2"));
+            }
+        });
+
+        assertTrue(clientQueue.contains("2"));
+
+        assertEquals("1", clientQueue.poll());
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void queueReconnectRemoved(CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final String setName = "queue-rmv" + colCfg.getAtomicityMode();
+
+        final IgniteQueue<String> clientQueue = client.queue(setName, 10, colCfg);
+
+        final IgniteQueue<String> srvQueue = srv.queue(setName, 10, null);
+
+        assertTrue(clientQueue.add("1"));
+
+        assertTrue(srvQueue.add("2"));
+
+        reconnectClientNode(client, srv, new Runnable() {
+            @Override public void run() {
+                srvQueue.close();
+            }
+        });
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                clientQueue.add("fail");
+
+                return null;
+            }
+        }, IllegalStateException.class, null);
+
+        IgniteQueue<String> newClientQueue = client.queue(setName, 10, colCfg);
+
+        IgniteQueue<String> newSrvQueue = srv.queue(setName, 10, null);
+
+        assertTrue(newClientQueue.add("1"));
+
+        assertTrue(newSrvQueue.add("2"));
+    }
+
+    /**
+     * @param colCfg Collection configuration.
+     * @throws Exception If failed.
+     */
+    private void queueReconnectInProgress(final CollectionConfiguration colCfg) throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        final String setName = "queue-rmv" + colCfg.getAtomicityMode();
+
+        final IgniteQueue<String> clientQueue = client.queue(setName, 10, colCfg);
+
+        final IgniteQueue<String> srvQueue = srv.queue(setName, 10, null);
+
+        assertTrue(clientQueue.offer("1"));
+
+        assertTrue(srvQueue.contains("1"));
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        if (colCfg.getAtomicityMode() == ATOMIC)
+            commSpi.blockMessage(GridNearAtomicUpdateResponse.class);
+        else
+            commSpi.blockMessage(GridNearTxPrepareResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    clientQueue.add("2");
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue("Future was not failed. Atomic mode: " + colCfg.getAtomicityMode() + ".", (Boolean)fut.get());
+
+        assertTrue(clientQueue.add("3"));
+
+        assertEquals("1", clientQueue.poll());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
new file mode 100644
index 0000000..e9667a1
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectComputeTest.java
@@ -0,0 +1,192 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectComputeTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectAffinityCallInProgress() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        IgniteCache<Integer, Integer> cache = client.getOrCreateCache("test-cache");
+
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(GridJobExecuteResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    client.compute().affinityCall("test-cache", 40, new IgniteCallable<Object>() {
+                        @Override public Integer call() throws Exception {
+                            return 42;
+                        }
+                    });
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectBroadcastInProgress() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(GridJobExecuteResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    client.compute().broadcast(new IgniteCallable<Object>() {
+                        @Override public Object call() throws Exception {
+                            return 42;
+                        }
+                    });
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectApplyInProgress() throws Exception {
+        final Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        BlockTpcCommunicationSpi commSpi = commSpi(srv);
+
+        commSpi.blockMessage(GridJobExecuteResponse.class);
+
+        final IgniteInternalFuture<Object> fut = GridTestUtils.runAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    client.compute().apply(new IgniteClosure<Integer, Integer>() {
+                        @Override public Integer apply(Integer o) {
+                            return o + 1;
+                        }
+                    }, Arrays.asList(1, 2, 3));
+                }
+                catch (IgniteClientDisconnectedException e) {
+                    checkAndWait(e);
+
+                    return true;
+                }
+
+                return false;
+            }
+        });
+
+        // Check that client waiting operation.
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                return fut.get(200);
+            }
+        }, IgniteFutureTimeoutCheckedException.class, null);
+
+        assertNotDone(fut);
+
+        commSpi.unblockMessage();
+
+        reconnectClientNode(client, srv, null);
+
+        assertTrue((Boolean)fut.get(2, TimeUnit.SECONDS));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
new file mode 100644
index 0000000..2bfdc85b
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectContinuousProcessorTest.java
@@ -0,0 +1,372 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.resources.*;
+
+import javax.cache.event.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectContinuousProcessorTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    private static volatile CountDownLatch latch;
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEventListenerReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        EventListener lsnr = new EventListener();
+
+        UUID opId = client.events().remoteListen(lsnr, null, EventType.EVT_JOB_STARTED);
+
+        lsnr.latch = new CountDownLatch(1);
+
+        log.info("Created remote listener: " + opId);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(reconnectLatch);
+
+        client.compute().run(new DummyJob());
+
+        assertTrue(lsnr.latch.await(5000, MILLISECONDS));
+
+        lsnr.latch = new CountDownLatch(1);
+
+        srv.compute().run(new DummyJob());
+
+        assertTrue(lsnr.latch.await(5000, MILLISECONDS));
+
+        lsnr.latch = new CountDownLatch(1);
+
+        log.info("Stop listen, should not get events anymore.");
+
+        client.events().stopRemoteListen(opId);
+
+        assertFalse(lsnr.latch.await(3000, MILLISECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMessageListenerReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final String topic = "testTopic";
+
+        MessageListener locLsnr  = new MessageListener();
+
+        UUID opId = client.message().remoteListen(topic, new RemoteMessageListener());
+
+        client.message().localListen(topic, locLsnr);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(reconnectLatch);
+
+        locLsnr.latch = new CountDownLatch(1);
+        latch = new CountDownLatch(2);
+
+        client.message().send(topic, "msg1");
+
+        assertTrue(locLsnr.latch.await(5000, MILLISECONDS));
+        assertTrue(latch.await(5000, MILLISECONDS));
+
+        locLsnr.latch = new CountDownLatch(1);
+        latch = new CountDownLatch(2);
+
+        srv.message().send(topic, "msg2");
+
+        assertTrue(locLsnr.latch.await(5000, MILLISECONDS));
+        assertTrue(latch.await(5000, MILLISECONDS));
+
+        log.info("Stop listen, should not get remote messages anymore.");
+
+        client.message().stopRemoteListen(opId);
+
+        srv.message().send(topic, "msg3");
+
+        locLsnr.latch = new CountDownLatch(1);
+        latch = new CountDownLatch(1);
+
+        assertTrue(locLsnr.latch.await(5000, MILLISECONDS));
+        assertFalse(latch.await(3000, MILLISECONDS));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCacheContinuousQueryReconnect() throws Exception {
+        Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> clientCache = client.getOrCreateCache(new CacheConfiguration<>());
+
+        CacheEventListener lsnr = new CacheEventListener();
+
+        ContinuousQuery<Object, Object> qry = new ContinuousQuery<>();
+
+        qry.setAutoUnsubscribe(true);
+
+        qry.setLocalListener(lsnr);
+
+        QueryCursor<?> cur = clientCache.query(qry);
+
+        for (int i = 0; i < 5; i++) {
+            log.info("Iteration: " + i);
+
+            continuousQueryReconnect(client, clientCache, lsnr);
+        }
+
+        log.info("Close cursor, should not get cache events anymore.");
+
+        cur.close();
+
+        lsnr.latch = new CountDownLatch(1);
+
+        clientCache.put(3, 3);
+
+        assertFalse(lsnr.latch.await(3000, MILLISECONDS));
+    }
+
+    /**
+     * @param client Client.
+     * @param clientCache Client cache.
+     * @param lsnr Continuous query listener.
+     * @throws Exception If failed.
+     */
+    private void continuousQueryReconnect(Ignite client,
+        IgniteCache<Object, Object> clientCache,
+        CacheEventListener lsnr)
+        throws Exception
+    {
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        };
+
+        client.events().localListen(p, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(reconnectLatch);
+
+        client.events().stopLocalListen(p);
+
+        lsnr.latch = new CountDownLatch(1);
+
+        clientCache.put(1, 1);
+
+        assertTrue(lsnr.latch.await(5000, MILLISECONDS));
+
+        lsnr.latch = new CountDownLatch(1);
+
+        srv.cache(null).put(2, 2);
+
+        assertTrue(lsnr.latch.await(5000, MILLISECONDS));
+    }
+
+    /**
+     *
+     */
+    private static class EventListener implements P2<UUID, Event> {
+        /** */
+        private volatile CountDownLatch latch;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, Event evt) {
+            assertTrue(ignite.cluster().localNode().isClient());
+
+            ignite.log().info("Received event: " + evt);
+
+            if (latch != null)
+                latch.countDown();
+
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class MessageListener implements P2<UUID, Object> {
+        /** */
+        private volatile CountDownLatch latch;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, Object msg) {
+            assertTrue(ignite.cluster().localNode().isClient());
+
+            ignite.log().info("Local listener received message: " + msg);
+
+            if (latch != null)
+                latch.countDown();
+
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class RemoteMessageListener implements P2<UUID, Object> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(UUID uuid, Object msg) {
+            ignite.log().info("Remote listener received message: " + msg);
+
+            if (latch != null)
+                latch.countDown();
+
+            return true;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class CacheEventListener implements CacheEntryUpdatedListener<Object, Object> {
+        /** */
+        private volatile CountDownLatch latch;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public void onUpdated(Iterable<CacheEntryEvent<?, ?>> evts) {
+            int cnt = 0;
+
+            for (CacheEntryEvent<?, ?> evt : evts) {
+                ignite.log().info("Received cache event: " + evt);
+
+                cnt++;
+            }
+
+            assertEquals(1, cnt);
+
+            if (latch != null)
+                latch.countDown();
+        }
+    }
+
+    /**
+     *
+     */
+    static class DummyJob implements IgniteRunnable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            ignite.log().info("Job run.");
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
new file mode 100644
index 0000000..feeebe5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectDiscoveryStateTest.java
@@ -0,0 +1,123 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.lang.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public class IgniteClientReconnectDiscoveryStateTest extends IgniteClientReconnectAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnect() throws Exception {
+        final Ignite client = ignite(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        long topVer = 4;
+
+        IgniteCluster cluster = client.cluster();
+
+        cluster.nodeLocalMap().put("locMapKey", 10);
+
+        Map<Integer, Integer> nodeCnt = new HashMap<>();
+
+        nodeCnt.put(1, 1);
+        nodeCnt.put(2, 2);
+        nodeCnt.put(3, 3);
+        nodeCnt.put(4, 4);
+
+        for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
+            Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+
+            assertNotNull("No nodes for topology: " + e.getKey(), nodes);
+            assertEquals((int)e.getValue(), nodes.size());
+        }
+
+        ClusterNode locNode = cluster.localNode();
+
+        assertEquals(topVer, locNode.order());
+
+        TestTcpDiscoverySpi srvSpi = spi(clientRouter(client));
+
+        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    IgniteFuture<?> fut = client.cluster().clientReconnectFuture();
+
+                    assertNotNull(fut);
+                    assertFalse(fut.isDone());
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        srvSpi.failNode(client.cluster().localNode().id(), null);
+
+        waitReconnectEvent(reconnectLatch);
+
+        topVer += 2; // Client failed and rejoined.
+
+        locNode = cluster.localNode();
+
+        assertEquals(topVer, locNode.order());
+        assertEquals(topVer, cluster.topologyVersion());
+
+        nodeCnt.put(5, 3);
+        nodeCnt.put(6, 4);
+
+        for (Map.Entry<Integer, Integer> e : nodeCnt.entrySet()) {
+            Collection<ClusterNode> nodes = cluster.topology(e.getKey());
+
+            assertNotNull("No nodes for topology: " + e.getKey(), nodes);
+            assertEquals((int)e.getValue(), nodes.size());
+        }
+
+        assertEquals(10, cluster.nodeLocalMap().get("locMapKey"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/57ac2b3b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
new file mode 100644
index 0000000..1b6523a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectFailoverAbstractTest.java
@@ -0,0 +1,231 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.testframework.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+
+/**
+ *
+ */
+public abstract class IgniteClientReconnectFailoverAbstractTest extends IgniteClientReconnectAbstractTest {
+    /** */
+    private static final Integer THREADS = 1;
+
+    /** */
+    private volatile CyclicBarrier barrier;
+
+    /** */
+    protected static final long TEST_TIME = 90_000;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(30_000);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int serverCount() {
+        return 3;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected int clientCount() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIME * 60_000;
+    }
+
+    /**
+     * @param c Test closure.
+     * @throws Exception If failed.
+     */
+    protected final void reconnectFailover(final Callable<Void> c) throws Exception {
+        final Ignite client = grid(serverCount());
+
+        assertTrue(client.cluster().localNode().isClient());
+
+        Ignite srv = clientRouter(client);
+
+        TestTcpDiscoverySpi srvSpi = spi(srv);
+
+        final AtomicBoolean stop = new AtomicBoolean(false);
+
+        final IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                try {
+                    int iter = 0;
+
+                    while (!stop.get()) {
+                        try {
+                            c.call();
+                        }
+                        catch (CacheException e) {
+                            checkAndWait(e);
+                        }
+                        catch (IgniteClientDisconnectedException e) {
+                            checkAndWait(e);
+                        }
+
+                        if (++iter % 100 == 0)
+                            log.info("Iteration: " + iter);
+
+                        if (barrier != null)
+                            barrier.await();
+                    }
+
+                    return null;
+                } catch (Throwable e) {
+                    log.error("Unexpected error in operation thread: " + e, e);
+
+                    stop.set(true);
+
+                    throw e;
+                }
+            }
+        }, THREADS, "test-operation-thread");
+
+        final AtomicReference<CountDownLatch> disconnected = new AtomicReference<>();
+        final AtomicReference<CountDownLatch> reconnected = new AtomicReference<>();
+
+        IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    CountDownLatch latch = reconnected.get();
+
+                    assertNotNull(latch);
+                    assertEquals(1, latch.getCount());
+
+                    latch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    CountDownLatch latch = disconnected.get();
+
+                    assertNotNull(latch);
+                    assertEquals(1, latch.getCount());
+
+                    latch.countDown();
+                }
+
+                return true;
+            }
+        };
+
+        client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+
+        try {
+            long stopTime = System.currentTimeMillis() + TEST_TIME;
+
+            String err = null;
+
+            while (System.currentTimeMillis() < stopTime && !fut.isDone()) {
+                U.sleep(500);
+
+                CountDownLatch disconnectLatch = new CountDownLatch(1);
+                CountDownLatch reconnectLatch = new CountDownLatch(1);
+
+                disconnected.set(disconnectLatch);
+                reconnected.set(reconnectLatch);
+
+                UUID nodeId = client.cluster().localNode().id();
+
+                log.info("Fail client: " + nodeId);
+
+                srvSpi.failNode(nodeId, null);
+
+                if (!disconnectLatch.await(10_000, MILLISECONDS)) {
+                    err = "Failed to wait for disconnect";
+
+                    break;
+                }
+
+                if (!reconnectLatch.await(10_000, MILLISECONDS)) {
+                    err = "Failed to wait for reconnect";
+
+                    break;
+                }
+
+                barrier = new CyclicBarrier(THREADS + 1, new Runnable() {
+                    @Override public void run() {
+                        barrier = null;
+                    }
+                });
+
+                try {
+                    barrier.await(10, SECONDS);
+                }
+                catch (TimeoutException e) {
+                    err = "Operations hang or fail with unexpected error.";
+
+                    break;
+                }
+            }
+
+            if (err != null) {
+                log.error(err);
+
+                U.dumpThreads(log);
+
+                CyclicBarrier barrier0 = barrier;
+
+                if (barrier0 != null)
+                    barrier0.reset();
+
+                stop.set(true);
+
+                fut.get();
+
+                fail(err);
+            }
+
+            stop.set(true);
+
+            fut.get();
+        }
+        finally {
+            client.events().stopLocalListen(p);
+
+            stop.set(true);
+        }
+    }
+}


[48/50] [abbrv] 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/3194415c
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3194415c
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3194415c

Branch: refs/heads/ignite-788-dev
Commit: 3194415cdd35827ee9c3e150fdb8c9ca069b37f0
Parents: d623d7e 0ac5fe7
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jul 20 14:43:10 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jul 20 14:43:10 2015 +0300

----------------------------------------------------------------------
 pom.xml | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------



[31/50] [abbrv] ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-890'

Posted by nt...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-890'


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

Branch: refs/heads/ignite-788-dev
Commit: 44043fa23ecd282fa69521faaaaa3ef74630f8da
Parents: fbc38d7 5c55812
Author: Denis Magda <dm...@gridgain.com>
Authored: Thu Jul 16 16:36:48 2015 +0300
Committer: Denis Magda <dm...@gridgain.com>
Committed: Thu Jul 16 16:36:48 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  14 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  35 ++++-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      | 140 ++++++++++++-------
 4 files changed, 131 insertions(+), 60 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/44043fa2/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/44043fa2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------

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


[35/50] [abbrv] ignite git commit: # ignite-1124 fixes for atomic cache update, more debug info for hanging tests

Posted by nt...@apache.org.
# ignite-1124 fixes for atomic cache update, more debug info for hanging tests


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

Branch: refs/heads/ignite-788-dev
Commit: f0d24f6b64e308de3170d5e7fb2092cb5b1421b7
Parents: 44043fa
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jul 17 12:16:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jul 17 12:16:06 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    |  12 ++
 .../processors/cache/GridCacheContext.java      |  45 +++--
 .../processors/cache/GridCacheMvccManager.java  |   6 +-
 .../GridCachePartitionExchangeManager.java      |  53 ++++++
 .../distributed/dht/GridDhtTopologyFuture.java  |  14 --
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  11 +-
 .../GridDhtPartitionsExchangeFuture.java        |  31 +---
 .../cache/transactions/IgniteTxManager.java     |   4 +-
 .../distributed/IgniteCache150ClientsTest.java  |   1 +
 .../IgniteCacheClientReconnectTest.java         | 175 +++++++++++++++++++
 .../IgniteCacheServerNodeConcurrentStart.java   |  96 ++++++++++
 .../testframework/junits/GridAbstractTest.java  |   5 +
 .../testsuites/IgniteCacheTestSuite2.java       |   2 +-
 .../testsuites/IgniteCacheTestSuite4.java       |   2 -
 .../testsuites/IgniteClientNodesTestSuite.java  |  42 +++++
 16 files changed, 439 insertions(+), 64 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 0d4ce32..c411f2e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -3009,6 +3009,18 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         return ctx.isDaemon() && U.hasAnnotation(comp.getClass(), SkipDaemon.class);
     }
 
+    /**
+     *
+     */
+    public void dumpDebugInfo() {
+        U.warn(log, "Dumping debug info for node [id=" + ctx.localNodeId() +
+            ", name=" + ctx.gridName() +
+            ", order=" + ctx.discovery().localNode().order() +
+            ", client=" + ctx.clientNode() + ']');
+
+        ctx.cache().context().exchange().dumpDebugInfo();
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index aec08c6..5f17746 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -524,7 +524,21 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return {@code True} if entries should not be deleted from cache immediately.
      */
     public boolean deferredDelete() {
-        return isDht() || isDhtAtomic() || isColocated() || (isNear() && atomic());
+        GridCacheAdapter<K, V> cache = this.cache;
+
+        if (cache == null)
+            throw new IllegalStateException("Cache stopped: " + cacheName);
+
+        return deferredDelete(cache);
+    }
+
+    /**
+     * @param cache Cache.
+     * @return {@code True} if entries should not be deleted from cache immediately.
+     */
+    public boolean deferredDelete(GridCacheAdapter<?, ?> cache) {
+        return cache.isDht() || cache.isDhtAtomic() || cache.isColocated() ||
+            (cache.isNear() && cache.configuration().getAtomicityMode() == ATOMIC);
     }
 
     /**
@@ -765,26 +779,37 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return Partition topology.
      */
     public GridDhtPartitionTopology topology() {
-        assert isNear() || isDht() || isColocated() || isDhtAtomic() : cache;
+        GridCacheAdapter<K, V> cache = this.cache;
 
-        return isNear() ? near().dht().topology() : dht().topology();
+        if (cache == null)
+            throw new IllegalStateException("Cache stopped: " + cacheName);
+
+        assert cache.isNear() || cache.isDht() || cache.isColocated() || cache.isDhtAtomic() : cache;
+
+        return topology(cache);
     }
 
     /**
      * @return Topology version future.
      */
     public GridDhtTopologyFuture topologyVersionFuture() {
-        assert isNear() || isDht() || isColocated() || isDhtAtomic() : cache;
+        GridCacheAdapter<K, V> cache = this.cache;
 
-        GridDhtTopologyFuture fut = null;
+        if (cache == null)
+            throw new IllegalStateException("Cache stopped: " + cacheName);
 
-        if (!isDhtAtomic()) {
-            GridDhtCacheAdapter<K, V> cache = isNear() ? near().dht() : colocated();
+        assert cache.isNear() || cache.isDht() || cache.isColocated() || cache.isDhtAtomic() : cache;
 
-            fut = cache.multiUpdateTopologyFuture();
-        }
+        return topology(cache).topologyVersionFuture();
+    }
 
-        return fut == null ? topology().topologyVersionFuture() : fut;
+    /**
+     * @param cache Cache.
+     * @return Partition topology.
+     */
+    private GridDhtPartitionTopology topology(GridCacheAdapter<K, V> cache) {
+        return cache.isNear() ? ((GridNearCacheAdapter<K, V>)cache).dht().topology() :
+            ((GridDhtCacheAdapter<K, V>)cache).topology();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 36e108f..bfb23e8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -208,8 +208,12 @@ public class GridCacheMvccManager extends GridCacheSharedManagerAdapter {
         exchLog = cctx.logger(getClass().getName() + ".exchange");
 
         pendingExplicit = GridConcurrentFactory.newMap();
+    }
 
-        cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_FAILED, EVT_NODE_LEFT);
+    /** {@inheritDoc} */
+    @Override protected void onKernalStart0(boolean reconnect) throws IgniteCheckedException {
+        if (!reconnect)
+            cctx.gridEvents().addLocalEventListener(discoLsnr, EVT_NODE_FAILED, EVT_NODE_LEFT);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 1f6a8bb..c26f5c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
@@ -949,6 +950,58 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     }
 
     /**
+     *
+     */
+    public void dumpDebugInfo() {
+        U.warn(log, "Ready affinity version: " + readyTopVer.get());
+
+        U.warn(log, "Last exchange future: " + lastInitializedFut);
+
+        U.warn(log, "Pending exchange futures:");
+
+        for (GridDhtPartitionsExchangeFuture fut : pendingExchangeFuts)
+            U.warn(log, ">>> " + fut);
+
+        U.warn(log, "Last 10 exchange futures (total: " + exchFuts.size() + "):");
+
+        int cnt = 0;
+
+        for (GridDhtPartitionsExchangeFuture fut : exchFuts) {
+            U.warn(log, ">>> " + fut);
+
+            if (++cnt == 10)
+                break;
+        }
+
+        dumpPendingObjects();
+    }
+
+    /**
+     *
+     */
+    public void dumpPendingObjects() {
+        U.warn(log, "Pending transactions:");
+
+        for (IgniteInternalTx tx : cctx.tm().activeTransactions())
+            U.warn(log, ">>> " + tx);
+
+        U.warn(log, "Pending explicit locks:");
+
+        for (GridCacheExplicitLockSpan lockSpan : cctx.mvcc().activeExplicitLocks())
+            U.warn(log, ">>> " + lockSpan);
+
+        U.warn(log, "Pending cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().activeFutures())
+            U.warn(log, ">>> " + fut);
+
+        U.warn(log, "Pending atomic cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().atomicFutures())
+            U.warn(log, ">>> " + fut);
+    }
+
+    /**
      * @param deque Deque to poll from.
      * @param time Time to wait.
      * @param w Worker.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
index 8a02ff2..57e3e33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 
@@ -33,18 +31,6 @@ import org.apache.ignite.internal.processors.cache.*;
  */
 public interface GridDhtTopologyFuture extends IgniteInternalFuture<AffinityTopologyVersion> {
     /**
-     * Gets a topology snapshot for the topology version represented by the future. Note that by the time
-     * partition exchange completes some nodes from the snapshot may leave the grid. One should use discovery
-     * service to check if the node is valid.
-     * <p/>
-     * This method will block until the topology future is ready.
-     *
-     * @return Topology snapshot for particular topology version.
-     * @throws IgniteCheckedException If topology future failed.
-     */
-    public GridDiscoveryTopologySnapshot topologySnapshot() throws IgniteCheckedException;
-
-    /**
      * Gets topology version of this future.
      *
      * @return Topology version.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 96e6edc..0a21979 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1159,7 +1159,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 // Enqueue if necessary after locks release.
                 if (deleted != null) {
                     assert !deleted.isEmpty();
-                    assert ctx.deferredDelete();
+                    assert ctx.deferredDelete(this) : this;
 
                     for (IgniteBiTuple<GridDhtCacheEntry, GridCacheVersion> e : deleted)
                         ctx.onDeferredDelete(e.get1(), e.get2());
@@ -2158,7 +2158,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     private void unlockEntries(Collection<GridDhtCacheEntry> locked, AffinityTopologyVersion topVer) {
         // Process deleted entries before locks release.
-        assert ctx.deferredDelete() : this;
+        assert ctx.deferredDelete(this) : this;
 
         // Entries to skip eviction manager notification for.
         // Enqueue entries while holding locks.

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 751c9ba..4c8a161 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -338,6 +338,8 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> completeFuture(AffinityTopologyVersion topVer) {
         if (waitForPartitionExchange() && topologyVersion().compareTo(topVer) < 0) {
+            GridFutureAdapter<Void> fut = null;
+
             synchronized (this) {
                 if (this.topVer == AffinityTopologyVersion.ZERO)
                     return null;
@@ -346,9 +348,14 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                     if (topCompleteFut == null)
                         topCompleteFut = new GridFutureAdapter<>();
 
-                    return topCompleteFut;
+                    fut = topCompleteFut;
                 }
             }
+
+            if (fut != null && isDone())
+                fut.onDone();
+
+            return fut;
         }
 
         return null;
@@ -582,7 +589,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 return;
             }
 
-            GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
+            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
             if (fut.isDone()) {
                 if (!fut.isCacheTopologyValid(cctx)) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 5701749..b3f19f6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -244,17 +244,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /** {@inheritDoc} */
-    @Override public GridDiscoveryTopologySnapshot topologySnapshot() throws IgniteCheckedException {
-        get();
-
-        if (topSnapshot.get() == null)
-            topSnapshot.compareAndSet(null, new GridDiscoveryTopologySnapshot(discoEvt.topologyVersion(),
-                discoEvt.topologyNodes()));
-
-        return topSnapshot.get();
-    }
-
-    /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion() {
         return exchId.topologyVersion();
     }
@@ -853,25 +842,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
         U.warn(log, "Failed to wait for partition release future. Dumping pending objects that might be the cause: " +
             cctx.localNodeId());
 
-        U.warn(log, "Pending transactions:");
-
-        for (IgniteInternalTx tx : cctx.tm().activeTransactions())
-            U.warn(log, ">>> " + tx);
-
-        U.warn(log, "Pending explicit locks:");
-
-        for (GridCacheExplicitLockSpan lockSpan : cctx.mvcc().activeExplicitLocks())
-            U.warn(log, ">>> " + lockSpan);
-
-        U.warn(log, "Pending cache futures:");
-
-        for (GridCacheFuture<?> fut : cctx.mvcc().activeFutures())
-            U.warn(log, ">>> " + fut);
-
-        U.warn(log, "Pending atomic cache futures:");
-
-        for (GridCacheFuture<?> fut : cctx.mvcc().atomicFutures())
-            U.warn(log, ">>> " + fut);
+        cctx.exchange().dumpPendingObjects();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index 82543c2..ee634ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -1984,9 +1984,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter {
             try {
                 cctx.kernalContext().gateway().readLock();
             }
-            catch (IllegalStateException ignore) {
+            catch (IllegalStateException | IgniteClientDisconnectedException ignore) {
                 if (log.isDebugEnabled())
-                    log.debug("Failed to acquire kernal gateway (grid is stopping).");
+                    log.debug("Failed to acquire kernal gateway [err=" + ignore + ']');
 
                 return;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
index 282c7c8..3fc44c0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
@@ -51,6 +51,7 @@ public class IgniteCache150ClientsTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setLocalHost("127.0.0.1");
         cfg.setNetworkTimeout(30_000);
         cfg.setConnectorConfiguration(null);
         cfg.setPeerClassLoadingEnabled(false);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java
new file mode 100644
index 0000000..c438c39
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientReconnectTest.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test for customer scenario.
+ */
+public class IgniteCacheClientReconnectTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int SRV_CNT = 3;
+
+    /** */
+    private static final int CACHES = 10;
+
+    /** */
+    private static final long TEST_TIME = 60_000;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setPeerClassLoadingEnabled(false);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        if (!client) {
+            CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES];
+
+            for (int i = 0; i < CACHES; i++) {
+                CacheConfiguration ccfg = new CacheConfiguration();
+
+                ccfg.setCacheMode(PARTITIONED);
+                ccfg.setAtomicityMode(TRANSACTIONAL);
+                ccfg.setBackups(1);
+                ccfg.setName("cache-" + i);
+
+                ccfgs[i] = ccfg;
+            }
+
+            cfg.setCacheConfiguration(ccfgs);
+        }
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(SRV_CNT);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return TEST_TIME + 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnect() throws Exception {
+        client = true;
+
+        final AtomicBoolean stop = new AtomicBoolean(false);
+
+        final AtomicInteger idx = new AtomicInteger(SRV_CNT);
+
+        final CountDownLatch latch = new CountDownLatch(2);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Ignite ignite = startGrid(idx.getAndIncrement());
+
+                latch.countDown();
+
+                assertTrue(ignite.cluster().localNode().isClient());
+
+                while (!stop.get())
+                    putGet(ignite);
+
+                return null;
+            }
+        }, 2, "client-thread");
+
+        try {
+            assertTrue(latch.await(10_000, MILLISECONDS));
+
+            long end = System.currentTimeMillis() + TEST_TIME;
+
+            int clientIdx = idx.getAndIncrement();
+
+            int cnt = 0;
+
+            while (System.currentTimeMillis() < end) {
+                log.info("Iteration: " + cnt++);
+
+                try (Ignite ignite = startGrid(clientIdx)) {
+                    assertTrue(ignite.cluster().localNode().isClient());
+
+                    assertEquals(6, ignite.cluster().nodes().size());
+
+                    putGet(ignite);
+                }
+            }
+
+            stop.set(true);
+
+            fut.get();
+        }
+        finally {
+            stop.set(true);
+        }
+    }
+
+    /**
+     * @param ignite Ignite.
+     */
+    private void putGet(Ignite ignite) {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < CACHES; i++) {
+            IgniteCache<Object, Object> cache = ignite.cache("cache-" + i);
+
+            assertNotNull(cache);
+
+            Integer key = rnd.nextInt(0, 100_000);
+
+            cache.put(key, key);
+
+            assertEquals(key, cache.get(key));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheServerNodeConcurrentStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheServerNodeConcurrentStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheServerNodeConcurrentStart.java
new file mode 100644
index 0000000..6b5d396
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheServerNodeConcurrentStart.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.processors.cache.distributed;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheServerNodeConcurrentStart extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int ITERATIONS = 2;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinderCleanFrequency(getTestTimeout() * 2);
+
+        CacheConfiguration ccfg1 = new CacheConfiguration();
+
+        ccfg1.setName("cache-1");
+        ccfg1.setCacheMode(REPLICATED);
+        ccfg1.setRebalanceMode(SYNC);
+
+        CacheConfiguration ccfg2 = new CacheConfiguration();
+
+        ccfg2.setName("cache-2");
+        ccfg2.setCacheMode(PARTITIONED);
+        ccfg2.setRebalanceMode(SYNC);
+        ccfg2.setBackups(2);
+
+        CacheConfiguration ccfg3 = new CacheConfiguration();
+
+        ccfg3.setName("cache-3");
+        ccfg3.setCacheMode(PARTITIONED);
+        ccfg3.setRebalanceMode(SYNC);
+        ccfg3.setBackups(0);
+
+        cfg.setCacheConfiguration(ccfg1, ccfg2, ccfg3);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return ITERATIONS * 3 * 60_000;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStart() throws Exception {
+        for (int i = 0; i < ITERATIONS; i++) {
+            log.info("Iteration: " + i);
+
+            long start = System.currentTimeMillis();
+
+            startGridsMultiThreaded(10, false);
+
+            startGridsMultiThreaded(10, 10);
+
+            awaitPartitionMapExchange();
+
+            stopAllGrids();
+
+            log.info("Iteration finished, time: " + (System.currentTimeMillis() - start) / 1000f);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 06a1523..9a55ccf 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -1577,6 +1577,11 @@ public abstract class GridAbstractTest extends TestCase {
                 "Test has been timed out and will be interrupted (threads dump will be taken before interruption) [" +
                 "test=" + getName() + ", timeout=" + getTestTimeout() + ']');
 
+            List<Ignite> nodes = G.allGrids();
+
+            for (Ignite node : nodes)
+                ((IgniteKernal)node).dumpDebugInfo();
+
             // We dump threads to stdout, because we can loose logs in case
             // the build is cancelled on TeamCity.
             U.dumpThreads(null);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 6a59826..741da87 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -136,7 +136,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
-        suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
+        suite.addTest(new TestSuite(IgniteCacheServerNodeConcurrentStart.class));
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index bde3a72..18b2409 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -146,8 +146,6 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteStartCacheInTransactionSelfTest.class);
         suite.addTestSuite(IgniteStartCacheInTransactionAtomicSelfTest.class);
 
-        suite.addTestSuite(IgniteCacheManyClientsTest.class);
-
         suite.addTestSuite(CacheReadThroughRestartSelfTest.class);
         suite.addTestSuite(CacheReadThroughReplicatedRestartSelfTest.class);
         suite.addTestSuite(CacheReadThroughReplicatedAtomicRestartSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/f0d24f6b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientNodesTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientNodesTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientNodesTestSuite.java
new file mode 100644
index 0000000..5cc4e5a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientNodesTestSuite.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+
+/**
+ *
+ */
+public class IgniteClientNodesTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception In case of error.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Client Nodes Reconnect Test Suite");
+
+        suite.addTest(IgniteClientReconnectTestSuite.suite());
+
+        suite.addTestSuite(IgniteCacheManyClientsTest.class);
+        suite.addTestSuite(IgniteCacheClientNodeConcurrentStart.class);
+        suite.addTestSuite(IgniteCacheClientReconnectTest.class);
+
+        return suite;
+    }
+}


[08/50] [abbrv] ignite git commit: # master minor

Posted by nt...@apache.org.
# master minor


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

Branch: refs/heads/ignite-788-dev
Commit: bdd176267f4085acce5ac3c51b5ac2787bd9ace7
Parents: a6d33b1
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jul 15 15:34:43 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jul 15 15:34:43 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAbstractNodeRestartSelfTest.java           | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bdd17626/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
index 7e65f23..ec1a12c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractNodeRestartSelfTest.java
@@ -505,7 +505,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting put thread...");
+                            info("Starting put thread: " + gridIdx);
 
                             IgniteCache<Integer, String> cache = grid(gridIdx).cache(CACHE_NAME);
 
@@ -546,7 +546,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting restart thread...");
+                            info("Starting restart thread: " + gridIdx);
 
                             int cnt = 0;
 
@@ -624,7 +624,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting put thread...");
+                            info("Starting put thread: " + gridIdx);
 
                             Ignite ignite = grid(gridIdx);
 
@@ -705,7 +705,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting restart thread...");
+                            info("Starting restart thread: " + gridIdx);
 
                             int cnt = 0;
 
@@ -780,7 +780,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting put thread...");
+                            info("Starting put thread: " + gridIdx);
 
                             Ignite ignite = grid(gridIdx);
 
@@ -849,7 +849,7 @@ public abstract class GridCacheAbstractNodeRestartSelfTest extends GridCommonAbs
                         try {
                             barrier.await();
 
-                            info("Starting restart thread...");
+                            info("Starting restart thread: " + gridIdx);
 
                             int cnt = 0;