You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/06/23 15:57:36 UTC

[01/40] incubator-ignite git commit: IGNITE-1034 - Drop slow clients.

Repository: incubator-ignite
Updated Branches:
  refs/heads/IgniteReflectionFactory-doc 8e0ce491d -> 3af015f45


IGNITE-1034 - Drop slow clients.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 44bbeceae657058eee322b13e339966c625802db
Parents: ad0a026
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Thu Jun 18 19:06:47 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Thu Jun 18 19:06:47 2015 -0700

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   4 +
 .../discovery/GridDiscoveryManager.java         |  15 +++
 .../ignite/internal/util/nio/GridNioServer.java |  51 ++++++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   5 +
 .../org/apache/ignite/spi/IgniteSpiContext.java |   5 +
 .../communication/tcp/TcpCommunicationSpi.java  |  73 +++++++++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   5 +
 .../IgniteSlowClientDetectionSelfTest.java      | 124 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   5 +
 9 files changed, 287 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 bea4256..885d52c 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
@@ -484,6 +484,10 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.discovery().tryFailNode(nodeId);
                     }
 
+                    @Override public void failNode(UUID nodeId) {
+                        ctx.discovery().failNode(nodeId);
+                    }
+
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
                         ctx.timeout().addTimeoutObject(new GridSpiTimeoutObject(obj));
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 464110c..717cdf3 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
@@ -1502,6 +1502,21 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /**
+     * @param nodeId Node ID to fail.
+     */
+    public void failNode(UUID nodeId) {
+        if (!busyLock.enterBusy())
+            return;
+
+        try {
+            getSpi().failNode(nodeId);
+        }
+        finally {
+            busyLock.leaveBusy();
+        }
+    }
+
+    /**
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 88fad71..b9d246a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -116,6 +116,10 @@ public class GridNioServer<T> {
     @SuppressWarnings("UnusedDeclaration")
     private boolean skipWrite;
 
+    /** For test purposes only. */
+    @SuppressWarnings("UnusedDeclaration")
+    private boolean skipRead;
+
     /** Local address. */
     private final InetSocketAddress locAddr;
 
@@ -145,6 +149,9 @@ public class GridNioServer<T> {
     @GridToStringExclude
     private IgnitePredicate<Message> skipRecoveryPred;
 
+    /** Optional listener to monitor outbound message queue size. */
+    private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+
     /** Static initializer ensures single-threaded execution of workaround. */
     static {
         // This is a workaround for JDK bug (NPE in Selector.open()).
@@ -174,6 +181,7 @@ public class GridNioServer<T> {
      * @param metricsLsnr Metrics listener.
      * @param formatter Message formatter.
      * @param skipRecoveryPred Skip recovery predicate.
+     * @param msgQueueLsnr Message queue size listener.
      * @param filters Filters for this server.
      * @throws IgniteCheckedException If failed.
      */
@@ -195,6 +203,7 @@ public class GridNioServer<T> {
         GridNioMetricsListener metricsLsnr,
         MessageFormatter formatter,
         IgnitePredicate<Message> skipRecoveryPred,
+        IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         A.notNull(addr, "addr");
@@ -215,6 +224,7 @@ public class GridNioServer<T> {
         this.sockRcvBuf = sockRcvBuf;
         this.sockSndBuf = sockSndBuf;
         this.sndQueueLimit = sndQueueLimit;
+        this.msgQueueLsnr = msgQueueLsnr;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -385,6 +395,11 @@ public class GridNioServer<T> {
         else if (msgCnt == 1)
             // Change from 0 to 1 means that worker thread should be waken up.
             clientWorkers.get(ses.selectorIndex()).offer(fut);
+
+        IgniteBiInClosure<GridNioSession, Integer> lsnr0 = msgQueueLsnr;
+
+        if (lsnr0 != null)
+            lsnr0.apply(ses, msgCnt);
     }
 
     /**
@@ -634,6 +649,17 @@ public class GridNioServer<T> {
         * @throws IOException If key read failed.
         */
         @Override protected void processRead(SelectionKey key) throws IOException {
+            if (skipRead) {
+                try {
+                    U.sleep(50);
+                }
+                catch (IgniteInterruptedCheckedException ignored) {
+                    U.warn(log, "Sleep has been interrupted.");
+                }
+
+                return;
+            }
+
             ReadableByteChannel sockCh = (ReadableByteChannel)key.channel();
 
             final GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
@@ -775,6 +801,17 @@ public class GridNioServer<T> {
          * @throws IOException If key read failed.
          */
         @Override protected void processRead(SelectionKey key) throws IOException {
+            if (skipRead) {
+                try {
+                    U.sleep(50);
+                }
+                catch (IgniteInterruptedCheckedException ignored) {
+                    U.warn(log, "Sleep has been interrupted.");
+                }
+
+                return;
+            }
+
             ReadableByteChannel sockCh = (ReadableByteChannel)key.channel();
 
             final GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
@@ -2108,6 +2145,9 @@ public class GridNioServer<T> {
         /** Skip recovery predicate. */
         private IgnitePredicate<Message> skipRecoveryPred;
 
+        /** Message queue size listener. */
+        private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
+
         /**
          * Finishes building the instance.
          *
@@ -2133,6 +2173,7 @@ public class GridNioServer<T> {
                 metricsLsnr,
                 formatter,
                 skipRecoveryPred,
+                msgQueueLsnr,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -2345,5 +2386,15 @@ public class GridNioServer<T> {
 
             return this;
         }
+
+        /**
+         * @param msgQueueLsnr Message queue size listener.
+         * @return Instance of this builder for chaining.
+         */
+        public Builder<T> messageQueueSizeListener(IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr) {
+            this.msgQueueLsnr = msgQueueLsnr;
+
+            return this;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 476f8a8..18191a1 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
@@ -759,6 +759,11 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
+        @Override public void failNode(UUID nodeId) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
             assert ignite instanceof IgniteKernal : ignite;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 f83326c..a655a73 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
@@ -312,6 +312,11 @@ public interface IgniteSpiContext {
     public boolean tryFailNode(UUID nodeId);
 
     /**
+     * @param nodeId Node ID.
+     */
+    public void failNode(UUID nodeId);
+
+    /**
      * @param c Timeout object.
      */
     public void addTimeoutObject(IgniteSpiTimeoutObject c);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 87d5b65..538e9a8 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
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.ipc.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
+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.internal.util.worker.*;
@@ -610,6 +611,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @LoggerResource
     private IgniteLogger log;
 
+    /** Ignite. */
+    @IgniteInstanceResource
+    @GridToStringExclude
+    private Ignite ignite;
+
     /** Local IP address. */
     private String locAddr;
 
@@ -653,6 +659,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Message queue limit. */
     private int msgQueueLimit = DFLT_MSG_QUEUE_LIMIT;
 
+    /** Slow client queue limit. */
+    private int slowClientQueueLimit;
+
     /** Min buffered message count. */
     private int minBufferedMsgCnt = Integer.getInteger(IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT, 512);
 
@@ -1145,6 +1154,30 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    public int getSlowClientQueueLimit() {
+        return slowClientQueueLimit;
+    }
+
+    /**
+     * Sets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @param slowClientQueueLimit Slow cilent queue limit.
+     */
+    public void setSlowClientQueueLimit(int slowClientQueueLimit) {
+        this.slowClientQueueLimit = slowClientQueueLimit;
+    }
+
+    /**
      * Sets the minimum number of messages for this SPI, that are buffered
      * prior to sending.
      * <p>
@@ -1315,6 +1348,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
 
+        if (slowClientQueueLimit > 0 && msgQueueLimit > 0) {
+            if (slowClientQueueLimit >= msgQueueLimit) {
+                U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit. " +
+                    "Slow client queue limit will have no effect.");
+            }
+        }
+
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
         if (shmemSrv != null) {
@@ -1425,6 +1465,15 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
+                IgniteBiInClosure<GridNioSession, Integer> queueSizeMonitor =
+                    !ignite.configuration().isClientMode() && slowClientQueueLimit > 0 ?
+                    new CI2<GridNioSession, Integer>() {
+                        @Override public void apply(GridNioSession ses, Integer qSize) {
+                            checkClientQueueSize(ses, qSize);
+                        }
+                    } :
+                    null;
+
                 GridNioServer<Message> srvr =
                     GridNioServer.<Message>builder()
                         .address(locHost)
@@ -1446,6 +1495,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             new GridConnectionBytesVerifyFilter(log))
                         .messageFormatter(msgFormatter)
                         .skipRecoveryPredicate(skipRecoveryPred)
+                        .messageQueueSizeListener(queueSizeMonitor)
                         .build();
 
                 boundTcpPort = port;
@@ -1860,6 +1910,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Checks client message queue size and initiates client drop if message queue size exceeds the configured limit.
+     *
+     * @param ses Node communication session.
+     * @param msgQueueSize Message queue size.
+     */
+    private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) {
+        if (slowClientQueueLimit > 0 && msgQueueSize > slowClientQueueLimit) {
+            UUID id = ses.meta(NODE_ID_META);
+
+            if (id != null) {
+                ClusterNode node = getSpiContext().node(id);
+
+                if (node != null && node.isClient()) {
+                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+
+                    getSpiContext().failNode(id);
+                }
+            }
+        }
+    }
+
+    /**
      * Establish TCP connection to remote node and returns client.
      *
      * @param node Remote node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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 b743a1a..8eb82ac 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
@@ -3402,6 +3402,11 @@ class ServerImpl extends TcpDiscoveryImpl {
                     failedNodes.remove(node);
 
                     leavingNodes.remove(node);
+
+                    ClientMessageWorker worker = clientMsgWorkers.remove(node.id());
+
+                    if (worker != null)
+                        worker.interrupt();
                 }
 
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/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
new file mode 100644
index 0000000..09b4215
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.event.*;
+
+/**
+ *
+ */
+public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
+
+    public static final String PARTITIONED = "partitioned";
+
+    /**
+     * @return Node count.
+     */
+    private int nodeCount() {
+        return 5;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
+            cfg.setClientMode(true);
+
+        TcpCommunicationSpi commSpi = new TcpCommunicationSpi();
+
+        commSpi.setSlowClientQueueLimit(50);
+        commSpi.setSharedMemoryPort(-1);
+        commSpi.setIdleConnectionTimeout(300_000);
+
+        cfg.setCommunicationSpi(commSpi);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(nodeCount());
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        super.afterTestsStopped();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSlowClient() throws Exception {
+        final IgniteEx slowClient = grid(nodeCount() - 1);
+
+        assertTrue(slowClient.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
+
+        IgniteEx client0 = grid(nodeCount() - 2);
+
+        assertTrue(client0.cluster().localNode().isClient());
+
+        IgniteCache<Object, Object> cache0 = client0.getOrCreateCache(PARTITIONED);
+
+        cache.query(new ContinuousQuery<>().setLocalListener(new Listener()));
+
+        for (int i = 0; i < 100; i++)
+            cache0.put(0, i);
+
+        GridIoManager ioMgr = slowClient.context().io();
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)((Object[])U.field(ioMgr, "spis"))[0];
+
+        GridNioServer nioSrvr = U.field(commSpi, "nioSrvr");
+
+        GridTestUtils.setFieldValue(nioSrvr, "skipRead", true);
+
+        // Initiate messages for client.
+        for (int i = 0; i < 100; i++)
+            cache0.put(0, new byte[10 * 1024]);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
+            }
+        }, getTestTimeout());
+    }
+
+    private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
+            System.out.println(">>>> Received update: " + iterable);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/44bbecea/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index 21f9424..c20ff2e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -502,6 +502,11 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {
         // No-op.
     }


[22/40] incubator-ignite git commit: #ignite-973 - fix for IGNITE-1041

Posted by sb...@apache.org.
#ignite-973 - fix for IGNITE-1041


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: ced62b8708952abfabc4926cfbb28c424d1ce651
Parents: de27ff1
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:35:43 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:35:43 2015 +0300

----------------------------------------------------------------------
 .../processors/query/h2/IgniteH2Indexing.java   |  2 ++
 .../query/h2/twostep/GridMapQueryExecutor.java  | 23 ++++++++++----------
 2 files changed, 13 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ced62b87/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 c162ae6..06c0961 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
@@ -1354,6 +1354,8 @@ public class IgniteH2Indexing implements GridQueryIndexing {
         Schema rmv = schemas.remove(schema);
 
         if (rmv != null) {
+            mapQryExec.onCacheStop(ccfg.getName());
+
             try {
                 dropSchema(schema);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ced62b87/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
index ba95d43..0f38353 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridMapQueryExecutor.java
@@ -124,18 +124,6 @@ public class GridMapQueryExecutor {
             }
         }, EventType.EVT_NODE_FAILED, EventType.EVT_NODE_LEFT);
 
-        // Drop group reservations for dead caches.
-        ctx.event().addLocalEventListener(new GridLocalEventListener() {
-            @Override public void onEvent(Event evt) {
-                String cacheName = ((CacheEvent)evt).cacheName();
-
-                for (T2<String,AffinityTopologyVersion> grpKey : reservations.keySet()) {
-                    if (F.eq(grpKey.get1(), cacheName))
-                        reservations.remove(grpKey);
-                }
-            }
-        }, EventType.EVT_CACHE_STOPPED);
-
         ctx.io().addMessageListener(GridTopic.TOPIC_QUERY, new GridMessageListener() {
             @Override public void onMessage(UUID nodeId, Object msg) {
                 if (!busyLock.enterBusy())
@@ -599,6 +587,17 @@ public class GridMapQueryExecutor {
     }
 
     /**
+     * @param cacheName Cache name.
+     */
+    public void onCacheStop(String cacheName) {
+        // Drop group reservations.
+        for (T2<String,AffinityTopologyVersion> grpKey : reservations.keySet()) {
+            if (F.eq(grpKey.get1(), cacheName))
+                reservations.remove(grpKey);
+        }
+    }
+
+    /**
      *
      */
     private class QueryResults {


[34/40] incubator-ignite git commit: # ignite-sprint-6 race in tcp communication connection, retry full partition map send

Posted by sb...@apache.org.
# ignite-sprint-6 race in tcp communication connection, retry full partition map send


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 285d790ff0ce597fde65a778c381e59a4f1c89aa
Parents: 1605996
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 22 16:40:08 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 16:58:01 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 49 +++++++++++++++-----
 .../communication/tcp/TcpCommunicationSpi.java  | 12 +++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |  1 -
 3 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/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 9f18c98..7c780b0 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
@@ -1078,18 +1078,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 log.debug("Received message for finished future (will reply only to sender) [msg=" + msg +
                     ", fut=" + this + ']');
 
-            try {
-                ClusterNode n = cctx.node(nodeId);
-
-                if (n != null)
-                    sendAllPartitions(F.asList(n), exchId);
-            }
-            catch (IgniteCheckedException e) {
-                scheduleRecheck();
-
-                U.error(log, "Failed to send full partition map to node (will retry after timeout) [node=" + nodeId +
-                    ", exchangeId=" + exchId + ']', e);
-            }
+            sendAllPartitions(nodeId, cctx.gridConfig().getNetworkSendRetryCount());
         }
         else {
             initFut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
@@ -1146,6 +1135,42 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @param nodeId Node ID.
+     * @param retryCnt Number of retries.
+     */
+    private void sendAllPartitions(final UUID nodeId, final int retryCnt) {
+        ClusterNode n = cctx.node(nodeId);
+
+        try {
+            if (n != null)
+                sendAllPartitions(F.asList(n), exchId);
+        }
+        catch (IgniteCheckedException e) {
+            if (e instanceof ClusterTopologyCheckedException || !cctx.discovery().alive(n)) {
+                log.debug("Failed to send full partition map to node, node left grid " +
+                    "[rmtNode=" + nodeId + ", exchangeId=" + exchId + ']');
+
+                return;
+            }
+
+            if (retryCnt > 0) {
+                long timeout = cctx.gridConfig().getNetworkSendRetryDelay();
+
+                LT.error(log, e, "Failed to send full partition map to node (will retry after timeout) " +
+                    "[node=" + nodeId + ", exchangeId=" + exchId + ", timeout=" + timeout + ']');
+
+                cctx.time().addTimeoutObject(new GridTimeoutObjectAdapter(timeout) {
+                    @Override public void onTimeout() {
+                        sendAllPartitions(nodeId, retryCnt - 1);
+                    }
+                });
+            }
+            else
+                U.error(log, "Failed to send full partition map [node=" + n + ", exchangeId=" + exchId + ']', e);
+        }
+    }
+
+    /**
      * @param nodeId Sender node ID.
      * @param msg Full partition info.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/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 1035ee5..addf243d 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
@@ -1748,6 +1748,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                                 assert old == null : "Client already created " +
                                         "[node=" + node + ", client=" + client0 + ", oldClient=" + old + ']';
+
+                                if (client0 instanceof GridTcpNioCommunicationClient) {
+                                    GridTcpNioCommunicationClient tcpClient = ((GridTcpNioCommunicationClient)client0);
+
+                                    if (tcpClient.session().closeTime() > 0 && clients.remove(nodeId, client0)) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Session was closed after client creation, will retry " +
+                                                "[node=" + node + ", client=" + client0 + ']');
+
+                                        client0 = null;
+                                    }
+                                }
                             }
                             else
                                 U.sleep(200);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/285d790f/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index c4a0916..4062931 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -37,7 +37,6 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "messageQueueLimit", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "reconnectCount", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "selectorsCount", 0);
-        checkNegativeSpiProperty(new TcpCommunicationSpi(), "minimumBufferedMessageCount", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectTimeout", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "maxConnectTimeout", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketWriteTimeout", -1);


[12/40] incubator-ignite git commit: # ignite-sprint-7 do not call ping from exchange worker

Posted by sb...@apache.org.
# ignite-sprint-7 do not call ping from exchange worker


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 8fa9d3dd9214e85748a6b7b900fa2433079ce246
Parents: d006389
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 14:37:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 14:37:37 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 64 ++++++++++++------
 .../GridCachePartitionExchangeManager.java      | 70 +++++++++-----------
 .../GridCacheAbstractFailoverSelfTest.java      |  6 +-
 3 files changed, 79 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/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 eef9fde..74a4512 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
@@ -323,7 +323,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * Processes failed messages.
      * @param nodeId niode id.
      * @param msg message.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
@@ -511,6 +511,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
      */
+    @SuppressWarnings("unchecked")
     private IgniteInternalFuture<Object> startFuture(GridCacheMessage cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
@@ -574,6 +575,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      * @throws ClusterTopologyCheckedException If receiver left.
      */
@@ -734,6 +736,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param nodeId ID of node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      */
     public void send(UUID nodeId, GridCacheMessage msg, GridIoPolicy plc) throws IgniteCheckedException {
@@ -795,8 +798,41 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Sends message without retries and node ping in case of error.
+     *
+     * @param node Node to send message to.
+     * @param msg Message.
+     * @param plc IO policy.
+     * @throws IgniteCheckedException If send failed.
+     */
+    public void sendNoRetry(ClusterNode node,
+        GridCacheMessage msg,
+        GridIoPolicy plc)
+        throws IgniteCheckedException
+    {
+        assert node != null;
+        assert msg != null;
+
+        onSend(msg, null);
+
+        try {
+            cctx.gridIO().send(node, TOPIC_CACHE, msg, plc);
+
+            if (log.isDebugEnabled())
+                log.debug("Sent cache message [msg=" + msg + ", node=" + U.toShortString(node) + ']');
+        }
+        catch (IgniteCheckedException e) {
+            if (!cctx.discovery().alive(node.id()))
+                throw new ClusterTopologyCheckedException("Node left grid while sending message to: " + node.id(), e);
+            else
+                throw e;
+        }
+    }
+
+    /**
      * Adds message handler.
      *
+     * @param cacheId Cache ID.
      * @param type Type of message.
      * @param c Handler.
      */
@@ -846,29 +882,15 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         idxClsHandlers.remove(cacheId);
 
-        for (Iterator<ListenerKey> iterator = clsHandlers.keySet().iterator(); iterator.hasNext(); ) {
-            ListenerKey key = iterator.next();
+        for (Iterator<ListenerKey> iter = clsHandlers.keySet().iterator(); iter.hasNext(); ) {
+            ListenerKey key = iter.next();
 
             if (key.cacheId == cacheId)
-                iterator.remove();
+                iter.remove();
         }
     }
 
     /**
-     * @param lsnr Listener to add.
-     */
-    public void addDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().addDisconnectListener(lsnr);
-    }
-
-    /**
-     * @param lsnr Listener to remove.
-     */
-    public void removeDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().removeDisconnectListener(lsnr);
-    }
-
-    /**
      * @param msgCls Message class to check.
      * @return Message index.
      */
@@ -1029,11 +1051,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = cacheId;
+            int res = cacheId;
 
-            result = 31 * result + msgCls.hashCode();
+            res = 31 * res + msgCls.hashCode();
 
-            return result;
+            return res;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/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 3df45cb..81019e9 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
@@ -567,27 +567,21 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         Collection<ClusterNode> rmts = null;
 
-        try {
-            // If this is the oldest node.
-            if (oldest.id().equals(cctx.localNodeId())) {
-                rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
-
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
+        // If this is the oldest node.
+        if (oldest.id().equals(cctx.localNodeId())) {
+            rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
 
-                sendAllPartitions(rmts);
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing local partitions from non-oldest node: " +
-                        cctx.localNodeId());
+            if (log.isDebugEnabled())
+                log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
 
-                sendLocalPartitions(oldest, null);
-            }
+            sendAllPartitions(rmts);
         }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to refresh partition map [oldest=" + oldest.id() + ", rmts=" + U.nodeIds(rmts) +
-                ", loc=" + cctx.localNodeId() + ']', e);
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Refreshing local partitions from non-oldest node: " +
+                    cctx.localNodeId());
+
+            sendLocalPartitions(oldest, null);
         }
     }
 
@@ -616,10 +610,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param nodes Nodes.
      * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes)
-        throws IgniteCheckedException {
+    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes) {
         GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE);
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
@@ -634,7 +626,19 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
-        cctx.io().safeSend(nodes, m, SYSTEM_POOL, null);
+        for (ClusterNode node : nodes) {
+            try {
+                cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
+            }
+            catch (ClusterTopologyCheckedException ignore) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
+                        node.id() + ", msg=" + m + ']');
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send partitions full message [node=" + node + ']', e);
+            }
+        }
 
         return true;
     }
@@ -642,11 +646,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param node Node.
      * @param id ID.
-     * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
-        throws IgniteCheckedException {
+    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
             cctx.kernalContext().clientNode(),
             cctx.versions().last());
@@ -669,16 +670,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
 
         try {
-            cctx.io().send(node, m, SYSTEM_POOL);
-
-            return true;
+            cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
                     node.id() + ", msg=" + m + ']');
-
-            return false;
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send local partition map to node [node=" + node + ", exchId=" + id + ']', e);
         }
     }
 
@@ -902,13 +902,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
 
         try {
-            try {
-                sendLocalPartitions(node, msg.exchangeId());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send local partition map to node [nodeId=" + node.id() + ", exchId=" +
-                    msg.exchangeId() + ']', e);
-            }
+            sendLocalPartitions(node, msg.exchangeId());
         }
         finally {
             leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8fa9d3dd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index b6cd88e..3c74674 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -74,9 +74,11 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
-        discoSpi.setSocketTimeout(10_000);
-        discoSpi.setAckTimeout(10_000);
+        discoSpi.setSocketTimeout(30_000);
+        discoSpi.setAckTimeout(30_000);
         discoSpi.setNetworkTimeout(60_000);
+        discoSpi.setHeartbeatFrequency(30_000);
+        discoSpi.setReconnectCount(2);
 
         return cfg;
     }


[28/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: a4ce455bbff7dbbf1084a7e9cc60424b53396471
Parents: 7c70267 6f5e676
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 22 12:50:21 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 22 12:50:21 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 45 ++++++++++----------
 1 file changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------



[05/40] incubator-ignite git commit: 1.1.4-SNAPSHOT

Posted by sb...@apache.org.
1.1.4-SNAPSHOT


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 4d9868a0efd8a55cc7ec4abc9472053bf17c05be
Parents: ad0a026
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Jun 19 12:24:27 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Jun 19 12:24:27 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/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/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/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 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 8422402..834b5b4 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index 1119301..e9cc980 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 3b1f3c6..1d3b1f3 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index dfde946..6b28630 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index b45b503..0c9cd3d 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 94154ea..5b7f518 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index b81cc63..d84cef0 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/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 32d6d1f..1be362f 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.1.3-SNAPSHOT
+ignite.version=1.1.4-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index 0ee5e75..d849117 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index 633df8b..e62738e 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index 2da3a6c..ff582bd 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index 4f71637..bcad0b0 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index ef26445..1884271 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index b4da498..dac4b3b 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 451e801..91811d2 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index 0b0f2a6..d100d4a 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index b4d6a5a..d44f9a8 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 2c5e15e..7cb43cc 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index d718a65..1c44422 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 18ec434..686a3ce 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 71c1ec9..2ce2de1 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 635b874..5863f4a 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 0c73aa0..86d0ab6 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 92b9da8..59c5d63 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 22f538f..90e7031 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index 5e5d78e..f664aa0 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 244fa3c..577ae57 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index 85f57bb..d43d337 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index dcd74d8..3e9ac77 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 41e4f8a..7b07260 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 7970f4b..5d7bf4c 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/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 f5116bd..c3d9045 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index 885afe1..f8627da 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index 51680fa..dd5cbe0 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.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 97901ba..4e8a50f 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index c4cb122..84a8336 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4d9868a0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 84e1c79..64f9e25 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.3-SNAPSHOT</version>
+    <version>1.1.4-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[27/40] incubator-ignite git commit: # review droppng of slow clients

Posted by sb...@apache.org.
# review droppng of slow clients


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 7c70267d2b4b7fc3fb7a1efda1ee785cebebcc41
Parents: 1cc0000
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Mon Jun 22 12:49:58 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Mon Jun 22 12:49:58 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |  3 ++
 .../communication/tcp/TcpCommunicationSpi.java  | 49 ++++++++++----------
 .../tcp/TcpCommunicationSpiMBean.java           |  8 ++++
 3 files changed, 36 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index b166f39..542fa30 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -236,7 +236,10 @@ public final class IgniteSystemProperties {
     /**
      * System property to disable buffered communication if node sends less messages count than
      * specified by this property. Default value is {@code 512}.
+     *
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     public static final String IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT = "IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT";
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/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 1ec2128..1035ee5 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
@@ -29,7 +29,6 @@ import org.apache.ignite.internal.util.ipc.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
-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.internal.util.worker.*;
@@ -51,7 +50,6 @@ import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.IgniteSystemProperties.*;
 import static org.apache.ignite.events.EventType.*;
 
 /**
@@ -657,9 +655,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Slow client queue limit. */
     private int slowClientQueueLimit;
 
-    /** Min buffered message count. */
-    private int minBufferedMsgCnt = Integer.getInteger(IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT, 512);
-
     /** NIO server. */
     private GridNioServer<Message> nioSrvr;
 
@@ -935,24 +930,29 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      *
      * @param connBufSize Connection buffer size.
      * @see #setConnectionBufferFlushFrequency(long)
+     * @deprecated Not used any more.
      */
+    @Deprecated
     @IgniteSpiConfiguration(optional = true)
     public void setConnectionBufferSize(int connBufSize) {
         // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public int getConnectionBufferSize() {
         return 0;
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @IgniteSpiConfiguration(optional = true)
     @Override public void setConnectionBufferFlushFrequency(long connBufFlushFreq) {
         // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public long getConnectionBufferFlushFrequency() {
         return 0;
     }
@@ -1156,13 +1156,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * Sets slow client queue limit.
      * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * When set to a positive number, communication SPI will monitor clients outbound message queue sizes and will drop
      * those clients whose queue exceeded this limit.
      * <p/>
      * Usually this value should be set to the same value as {@link #getMessageQueueLimit()} which controls
-     * message back-pressure for server nodes. The default value for this parameter is {@link #DFLT_MSG_QUEUE_LIMIT}.
+     * message back-pressure for server nodes. The default value for this parameter is {@code 0}
+     * which means {@code unlimited}.
      *
-     * @param slowClientQueueLimit Slow cilent queue limit.
+     * @param slowClientQueueLimit Slow client queue limit.
      */
     public void setSlowClientQueueLimit(int slowClientQueueLimit) {
         this.slowClientQueueLimit = slowClientQueueLimit;
@@ -1171,20 +1172,20 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * Sets the minimum number of messages for this SPI, that are buffered
      * prior to sending.
-     * <p>
-     * Defaults to either {@code 512} or {@link IgniteSystemProperties#IGNITE_MIN_BUFFERED_COMMUNICATION_MSG_CNT}
-     * system property (if specified).
      *
      * @param minBufferedMsgCnt Minimum buffered message count.
+     * @deprecated Not used any more.
      */
     @IgniteSpiConfiguration(optional = true)
+    @Deprecated
     public void setMinimumBufferedMessageCount(int minBufferedMsgCnt) {
-        this.minBufferedMsgCnt = minBufferedMsgCnt;
+        // No-op.
     }
 
     /** {@inheritDoc} */
+    @Deprecated
     @Override public int getMinimumBufferedMessageCount() {
-        return minBufferedMsgCnt;
+        return 0;
     }
 
     /** {@inheritDoc} */
@@ -1248,7 +1249,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
         assertParameter(reconCnt > 0, "reconnectCnt > 0");
         assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
-        assertParameter(minBufferedMsgCnt >= 0, "minBufferedMsgCnt >= 0");
         assertParameter(connTimeout >= 0, "connTimeout >= 0");
         assertParameter(maxConnTimeout >= connTimeout, "maxConnTimeout >= connTimeout");
         assertParameter(sockWriteTimeout >= 0, "sockWriteTimeout >= 0");
@@ -1326,7 +1326,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
             log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
-            log.debug(configInfo("minBufferedMsgCnt", minBufferedMsgCnt));
             log.debug(configInfo("connTimeout", connTimeout));
             log.debug(configInfo("maxConnTimeout", maxConnTimeout));
             log.debug(configInfo("reconCnt", reconCnt));
@@ -1339,11 +1338,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             U.quietAndWarn(log, "'TCP_NO_DELAY' for communication is off, which should be used with caution " +
                 "since may produce significant delays with some scenarios.");
 
-        if (slowClientQueueLimit > 0 && msgQueueLimit > 0) {
-            if (slowClientQueueLimit >= msgQueueLimit) {
-                U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit. " +
-                    "Slow client queue limit will have no effect.");
-            }
+        if (slowClientQueueLimit > 0 && msgQueueLimit > 0 && slowClientQueueLimit >= msgQueueLimit) {
+            U.quietAndWarn(log, "Slow client queue limit is set to a value greater than message queue limit " +
+                "(slow client queue limit will have no effect) [msgQueueLimit=" + msgQueueLimit +
+                    ", slowClientQueueLimit=" + slowClientQueueLimit + ']');
         }
 
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
@@ -1916,13 +1914,16 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
-                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [" +
-                        "srvNode=" + getSpiContext().localNode().id() +
+                    String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
+                        "the client will be dropped " +
+                        "(consider changing 'slowClientQueueLimit' configuration property) " +
+                        "[srvNode=" + getSpiContext().localNode().id() +
                         ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
-                    LT.warn(log, null, msg);
+                    U.quietAndWarn(
+                        log,
+                        msg);
 
                     getSpiContext().failNode(id, msg);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c70267d/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index 1971d99..067ca41 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -168,7 +168,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * connection buffer.
      *
      * @return Flush frequency.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Connection buffer flush frequency.")
     public long getConnectionBufferFlushFrequency();
 
@@ -183,7 +185,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      *
      * @param connBufFlushFreq Flush frequency.
      * @see #getConnectionBufferSize()
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Sets connection buffer flush frequency.")
     public void setConnectionBufferFlushFrequency(long connBufFlushFreq);
 
@@ -193,7 +197,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * If set to {@code 0} connection buffer is disabled.
      *
      * @return Connection buffer size.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Connection buffer size.")
     public int getConnectionBufferSize();
 
@@ -238,7 +244,9 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      * prior to sending.
      *
      * @return Minimum buffered message count.
+     * @deprecated Not used anymore.
      */
+    @Deprecated
     @MXBeanDescription("Minimum buffered message count.")
     public int getMinimumBufferedMessageCount();
 


[32/40] incubator-ignite git commit: Merge branch 'ignite-sprint-6' into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' into ignite-sprint-7

Conflicts:
	modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: cd06e1d317e3f828cb0f64ef2e2903be8d1fea8b
Parents: ec9cfca 1605996
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 16:02:08 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 16:02:08 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../ignite/internal/util/nio/GridNioServer.java |  64 +++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  92 ++++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |  19 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  80 +++++---
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 ++++++++++++++++++
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 ++-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   3 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 32 files changed, 916 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cd06e1d3/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --cc modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index ae4acc2,b9205a9..14093ba
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@@ -62,7 -62,8 +62,8 @@@ public class IgniteCacheQuerySelfTestSu
          suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-         suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
 -        // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
++        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
          suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
          suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);


[40/40] incubator-ignite git commit: # ignite-reflectionFactory: review

Posted by sb...@apache.org.
# ignite-reflectionFactory: review


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 3af015f457a5a3909753484c2c035b523b74fbac
Parents: 1f8bbe3
Author: ashutak <as...@gridgain.com>
Authored: Tue Jun 23 16:57:44 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jun 23 16:57:44 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/configuration/IgniteReflectionFactory.java | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3af015f4/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
index 2c106d2..af73737 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteReflectionFactory.java
@@ -26,14 +26,14 @@ import java.lang.reflect.*;
 import java.util.*;
 
 /**
- * Factory implementation that use reflection to creat instance of given class.
- * <p/>
+ * Factory implementation that use reflection to create instance of given class.
+ * <p>
  * There are 2 modes of factory: singleton and non-sigletton.
- * <p/>
+ * <p>
  * Class that should be created by {@link IgniteReflectionFactory} (component class) have to be
  * public java POJO with public setters for field
  * for which property injection will be used (see {@link #setProperties(Map)}).
- * <p/>
+ * <p>
  * <h1 class="header">Configuration</h1>
  * <h2 class="header">Mandatory</h2>
  * The following configuration parameters are mandatory:


[18/40] incubator-ignite git commit: #IGNITE-1034 - Corrected messages after code review.

Posted by sb...@apache.org.
#IGNITE-1034 - Corrected messages after code review.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 03dc03d42eeabedbb82862461b523421e49744b9
Parents: e1b8800
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 11:32:23 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 11:32:23 2015 -0700

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/03dc03d4/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 3997768..1ec2128 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
@@ -1917,7 +1917,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                 if (node != null && node.isClient()) {
                     String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
-                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [" +
+                        "srvNode=" + getSpiContext().localNode().id() +
+                        ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
                     LT.warn(log, null, msg);


[06/40] incubator-ignite git commit: 1.1.6-SNAPSHOT

Posted by sb...@apache.org.
1.1.6-SNAPSHOT


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: d0063897052b97398a93bd3ac32fd573d0d7f3d3
Parents: 188cd82
Author: Ignite Teamcity <ig...@apache.org>
Authored: Fri Jun 19 12:56:32 2015 +0300
Committer: Ignite Teamcity <ig...@apache.org>
Committed: Fri Jun 19 12:56:32 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                  | 2 +-
 modules/aop/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/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/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 +-
 pom.xml                                           | 2 +-
 37 files changed, 37 insertions(+), 37 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/examples/pom.xml
----------------------------------------------------------------------
diff --git a/examples/pom.xml b/examples/pom.xml
index 834b5b4..bcff11e 100644
--- a/examples/pom.xml
+++ b/examples/pom.xml
@@ -28,7 +28,7 @@
     </parent>
 
     <artifactId>ignite-examples</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/aop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aop/pom.xml b/modules/aop/pom.xml
index e9cc980..5eea18d 100644
--- a/modules/aop/pom.xml
+++ b/modules/aop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aop</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/aws/pom.xml
----------------------------------------------------------------------
diff --git a/modules/aws/pom.xml b/modules/aws/pom.xml
index 1d3b1f3..05ecbc1 100644
--- a/modules/aws/pom.xml
+++ b/modules/aws/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-aws</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 6b28630..7c4d2c8 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-clients</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/cloud/pom.xml
----------------------------------------------------------------------
diff --git a/modules/cloud/pom.xml b/modules/cloud/pom.xml
index 0c9cd3d..bb32689 100644
--- a/modules/cloud/pom.xml
+++ b/modules/cloud/pom.xml
@@ -29,7 +29,7 @@
     </parent>
 
     <artifactId>ignite-cloud</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <jcloud.version>1.9.0</jcloud.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/codegen/pom.xml
----------------------------------------------------------------------
diff --git a/modules/codegen/pom.xml b/modules/codegen/pom.xml
index 5b7f518..1ea5fe5 100644
--- a/modules/codegen/pom.xml
+++ b/modules/codegen/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-codegen</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/core/pom.xml
----------------------------------------------------------------------
diff --git a/modules/core/pom.xml b/modules/core/pom.xml
index d84cef0..79fbc2f 100644
--- a/modules/core/pom.xml
+++ b/modules/core/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-core</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/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 1be362f..b25b2ca 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.1.4-SNAPSHOT
+ignite.version=1.1.6-SNAPSHOT
 ignite.build=0
 ignite.revision=DEV
 ignite.rel.date=01011970

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/extdata/p2p/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/p2p/pom.xml b/modules/extdata/p2p/pom.xml
index d849117..26f2e50 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/extdata/uri/pom.xml
----------------------------------------------------------------------
diff --git a/modules/extdata/uri/pom.xml b/modules/extdata/uri/pom.xml
index e62738e..1690ab9 100644
--- a/modules/extdata/uri/pom.xml
+++ b/modules/extdata/uri/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-extdata-uri</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/gce/pom.xml
----------------------------------------------------------------------
diff --git a/modules/gce/pom.xml b/modules/gce/pom.xml
index ff582bd..127c7a4 100644
--- a/modules/gce/pom.xml
+++ b/modules/gce/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-gce</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/geospatial/pom.xml
----------------------------------------------------------------------
diff --git a/modules/geospatial/pom.xml b/modules/geospatial/pom.xml
index bcad0b0..e28c139 100644
--- a/modules/geospatial/pom.xml
+++ b/modules/geospatial/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-geospatial</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 1884271..637f097 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hadoop</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/hibernate/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hibernate/pom.xml b/modules/hibernate/pom.xml
index dac4b3b..d001b02 100644
--- a/modules/hibernate/pom.xml
+++ b/modules/hibernate/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-hibernate</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/indexing/pom.xml
----------------------------------------------------------------------
diff --git a/modules/indexing/pom.xml b/modules/indexing/pom.xml
index 91811d2..b044092 100644
--- a/modules/indexing/pom.xml
+++ b/modules/indexing/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-indexing</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/jcl/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jcl/pom.xml b/modules/jcl/pom.xml
index d100d4a..b764c96 100644
--- a/modules/jcl/pom.xml
+++ b/modules/jcl/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jcl</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/jta/pom.xml
----------------------------------------------------------------------
diff --git a/modules/jta/pom.xml b/modules/jta/pom.xml
index d44f9a8..531e6f2 100644
--- a/modules/jta/pom.xml
+++ b/modules/jta/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-jta</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/log4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/log4j/pom.xml b/modules/log4j/pom.xml
index 7cb43cc..af62389 100644
--- a/modules/log4j/pom.xml
+++ b/modules/log4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-log4j</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/mesos/pom.xml
----------------------------------------------------------------------
diff --git a/modules/mesos/pom.xml b/modules/mesos/pom.xml
index 1c44422..07bee9e 100644
--- a/modules/mesos/pom.xml
+++ b/modules/mesos/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-mesos</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <jetty.version>9.2.10.v20150310</jetty.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/rest-http/pom.xml
----------------------------------------------------------------------
diff --git a/modules/rest-http/pom.xml b/modules/rest-http/pom.xml
index 686a3ce..2d7fe6d 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/scalar-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar-2.10/pom.xml b/modules/scalar-2.10/pom.xml
index 2ce2de1..c4b8d60 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/scalar/pom.xml
----------------------------------------------------------------------
diff --git a/modules/scalar/pom.xml b/modules/scalar/pom.xml
index 5863f4a..a048939 100644
--- a/modules/scalar/pom.xml
+++ b/modules/scalar/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-scalar</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/schedule/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schedule/pom.xml b/modules/schedule/pom.xml
index 86d0ab6..735035f 100644
--- a/modules/schedule/pom.xml
+++ b/modules/schedule/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-schedule</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/schema-import/pom.xml
----------------------------------------------------------------------
diff --git a/modules/schema-import/pom.xml b/modules/schema-import/pom.xml
index 59c5d63..2898327 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/slf4j/pom.xml
----------------------------------------------------------------------
diff --git a/modules/slf4j/pom.xml b/modules/slf4j/pom.xml
index 90e7031..fb7c86e 100644
--- a/modules/slf4j/pom.xml
+++ b/modules/slf4j/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-slf4j</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spark-2.10/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark-2.10/pom.xml b/modules/spark-2.10/pom.xml
index f664aa0..1b8b04e 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spark/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spark/pom.xml b/modules/spark/pom.xml
index 577ae57..8c00ec8 100644
--- a/modules/spark/pom.xml
+++ b/modules/spark/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spark</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/spring/pom.xml
----------------------------------------------------------------------
diff --git a/modules/spring/pom.xml b/modules/spring/pom.xml
index d43d337..35638fe 100644
--- a/modules/spring/pom.xml
+++ b/modules/spring/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-spring</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/ssh/pom.xml
----------------------------------------------------------------------
diff --git a/modules/ssh/pom.xml b/modules/ssh/pom.xml
index 3e9ac77..23ae1df 100644
--- a/modules/ssh/pom.xml
+++ b/modules/ssh/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-ssh</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/tools/pom.xml
----------------------------------------------------------------------
diff --git a/modules/tools/pom.xml b/modules/tools/pom.xml
index 7b07260..9e5685d 100644
--- a/modules/tools/pom.xml
+++ b/modules/tools/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-tools</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/urideploy/pom.xml
----------------------------------------------------------------------
diff --git a/modules/urideploy/pom.xml b/modules/urideploy/pom.xml
index 5d7bf4c..0add261 100644
--- a/modules/urideploy/pom.xml
+++ b/modules/urideploy/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-urideploy</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/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 c3d9045..8806438 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/visor-console/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-console/pom.xml b/modules/visor-console/pom.xml
index f8627da..8c6e4c9 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/visor-plugins/pom.xml
----------------------------------------------------------------------
diff --git a/modules/visor-plugins/pom.xml b/modules/visor-plugins/pom.xml
index dd5cbe0..e4162c0 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.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <!-- Ignite dependencies -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/web/pom.xml
----------------------------------------------------------------------
diff --git a/modules/web/pom.xml b/modules/web/pom.xml
index 4e8a50f..e6a23bd 100644
--- a/modules/web/pom.xml
+++ b/modules/web/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-web</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <dependencies>
         <dependency>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/modules/yardstick/pom.xml
----------------------------------------------------------------------
diff --git a/modules/yardstick/pom.xml b/modules/yardstick/pom.xml
index 84a8336..5307447 100644
--- a/modules/yardstick/pom.xml
+++ b/modules/yardstick/pom.xml
@@ -31,7 +31,7 @@
     </parent>
 
     <artifactId>ignite-yardstick</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
 
     <properties>
         <yardstick.version>0.7.0</yardstick.version>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0063897/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 64f9e25..44cb0ce 100644
--- a/pom.xml
+++ b/pom.xml
@@ -32,7 +32,7 @@
 
     <groupId>org.apache.ignite</groupId>
     <artifactId>apache-ignite</artifactId>
-    <version>1.1.4-SNAPSHOT</version>
+    <version>1.1.6-SNAPSHOT</version>
     <packaging>pom</packaging>
 
     <properties>


[37/40] incubator-ignite git commit: # ignite-sprint-7 minor

Posted by sb...@apache.org.
# ignite-sprint-7 minor


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 36433bc3d96a67975ce99a0774e608ccf50c2a21
Parents: c7ba154
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 13:31:34 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 13:31:34 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/testframework/junits/GridAbstractTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/36433bc3/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 9c42920..9a63b3a 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
@@ -606,7 +606,7 @@ public abstract class GridAbstractTest extends TestCase {
                 Thread.sleep(1000);
         }
 
-        throw new Exception("Failed to wait for proper topology.");
+        throw new Exception("Failed to wait for proper topology: " + cnt);
     }
 
     /** */


[23/40] incubator-ignite git commit: #ignite-973 - minor

Posted by sb...@apache.org.
#ignite-973 - minor


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 774bc94ff7491735cf966f6f7a9b2a1887071dcf
Parents: ced62b8
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:46:08 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:46:08 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/IgniteCacheOffheapEvictQueryTest.java         | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/774bc94f/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index fc6c125..72bfcaa 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -69,7 +69,7 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
      */
     public void testEvictAndRemove() throws Exception {
         final int KEYS_CNT = 3000;
-        final int THREADS_CNT = 256;
+        final int THREADS_CNT = 50;
 
         final IgniteCache<Integer,Integer> c = startGrid().cache(null);
 


[16/40] incubator-ignite git commit: # fix for jta issue

Posted by sb...@apache.org.
# fix for jta issue


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 20ac4b80ce90422268e90e9ef3cd9a84cbcfac2d
Parents: 4d9868a
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jun 19 21:27:35 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jun 19 21:27:35 2015 +0300

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java  |  1 +
 .../processors/cache/jta/GridCacheXAResource.java | 18 +++++++++---------
 .../processors/cache/GridCacheJtaSelfTest.java    |  2 +-
 3 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/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 3df45cb..ad4cf50 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
@@ -771,6 +771,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         }
                     }
                 }
+
                 if (skipped > 10)
                     fut.cleanUp();
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
index 505f558..442cdde 100644
--- a/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
+++ b/modules/jta/src/main/java/org/apache/ignite/internal/processors/cache/jta/GridCacheXAResource.java
@@ -209,7 +209,14 @@ public final class GridCacheXAResource implements XAResource {
 
     /** {@inheritDoc} */
     @Override public int getTransactionTimeout() {
-        return (int)cacheTx.timeout();
+        return (int)(cacheTx.timeout() / 1000);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean setTransactionTimeout(int i) {
+        cacheTx.timeout(i * 1000);
+
+        return true;
     }
 
     /** {@inheritDoc} */
@@ -217,7 +224,7 @@ public final class GridCacheXAResource implements XAResource {
         if (xar == this)
             return true;
 
-        if  (!(xar instanceof GridCacheXAResource))
+        if (!(xar instanceof GridCacheXAResource))
             return false;
 
         GridCacheXAResource other = (GridCacheXAResource)xar;
@@ -225,13 +232,6 @@ public final class GridCacheXAResource implements XAResource {
         return cctx == other.cctx;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean setTransactionTimeout(int i) {
-        cacheTx.timeout(i);
-
-        return true;
-    }
-
     /**
      *
      * @return {@code true} if jta was already committed or rolled back.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/20ac4b80/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
index cb7b4d6..b299f8d 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheJtaSelfTest.java
@@ -152,7 +152,7 @@ public class GridCacheJtaSelfTest extends GridCacheAbstractSelfTest {
     @SuppressWarnings("ConstantConditions")
     public void testJtaTwoCaches() throws Exception {
         fail("https://issues.apache.org/jira/browse/IGNITE-591");
-        
+
         UserTransaction jtaTx = jotm.getUserTransaction();
 
         IgniteEx ignite = grid(0);


[20/40] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: de27ff1b8788b622937b17bfeaf8941242914d1a
Parents: bcfbb92 415264e
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:21:46 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:21:46 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../GridCachePartitionExchangeManager.java      |   1 +
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  71 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 21 files changed, 467 insertions(+), 33 deletions(-)
----------------------------------------------------------------------



[19/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 415264e3e59d98e711fe37d7e4c12ef2416d9e7d
Parents: 03dc03d e0e1994
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 11:32:38 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 11:32:38 2015 -0700

----------------------------------------------------------------------
 .../cache/GridCachePartitionExchangeManager.java  |  1 +
 .../processors/cache/jta/GridCacheXAResource.java | 18 +++++++++---------
 .../processors/cache/GridCacheJtaSelfTest.java    |  2 +-
 3 files changed, 11 insertions(+), 10 deletions(-)
----------------------------------------------------------------------



[08/40] incubator-ignite git commit: # ignite-sprint-6 fixed NPE in comm spi unit tests

Posted by sb...@apache.org.
# ignite-sprint-6 fixed NPE in comm spi unit tests


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 895771a79d8acca3baf06c23bf3f82483ec62d6f
Parents: ef4abeb
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 13:06:05 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 13:06:05 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/895771a7/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 84c1a57..bbb9b1c 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
@@ -1458,8 +1458,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
                 };
 
+                boolean clientMode = Boolean.TRUE.equals(ignite.configuration().isClientMode());
+
                 IgniteBiInClosure<GridNioSession, Integer> queueSizeMonitor =
-                    !ignite.configuration().isClientMode() && slowClientQueueLimit > 0 ?
+                    !clientMode && slowClientQueueLimit > 0 ?
                     new CI2<GridNioSession, Integer>() {
                         @Override public void apply(GridNioSession ses, Integer qSize) {
                             checkClientQueueSize(ses, qSize);


[39/40] incubator-ignite git commit: Merge branch 'ignite-sprint-7' into IgniteReflectionFactory-doc

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-7' into IgniteReflectionFactory-doc


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 1f8bbe3208d7ee9890363bb5353652998df1fd24
Parents: 8e0ce49 3b49184
Author: ashutak <as...@gridgain.com>
Authored: Tue Jun 23 16:54:55 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Tue Jun 23 16:54:55 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |   2 +-
 modules/aop/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 +-
 .../apache/ignite/IgniteSystemProperties.java   |   3 +
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../processors/cache/GridCacheIoManager.java    |  64 +++--
 .../GridCachePartitionExchangeManager.java      |  71 +++---
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../GridDhtPartitionsExchangeFuture.java        |  49 +++-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++-
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 104 +++++++-
 .../tcp/TcpCommunicationSpiMBean.java           |  19 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  80 +++++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 .../core/src/main/resources/ignite.properties   |   2 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++
 .../GridCacheAbstractFailoverSelfTest.java      |   6 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 ++++++++++
 .../GridCacheReplicatedFailoverSelfTest.java    |   5 +
 .../GridTcpCommunicationSpiConfigSelfTest.java  |   1 -
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 +++-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../testframework/junits/GridAbstractTest.java  |   2 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |  80 +------
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +--
 .../internal/processors/hadoop/HadoopUtils.java | 237 -------------------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 ++++++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 ++-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 196 +++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 .../cache/jta/GridCacheXAResource.java          |  18 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/mesos/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar-2.10/pom.xml                     |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spark-2.10/pom.xml                      |   2 +-
 modules/spark/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   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 +-
 pom.xml                                         |   2 +-
 98 files changed, 1364 insertions(+), 599 deletions(-)
----------------------------------------------------------------------



[36/40] incubator-ignite git commit: # ignite-sprint-7 disabled hanging test

Posted by sb...@apache.org.
# ignite-sprint-7 disabled hanging test


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: c7ba1541959b78451e81fb701550894b09fe6eb9
Parents: 1c66078
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 12:55:17 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 12:55:17 2015 +0300

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


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c7ba1541/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
index 3461dd4..4269ea6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFailoverSelfTest.java
@@ -28,6 +28,11 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheReplicatedFailoverSelfTest extends GridCacheAbstractFailoverTxSelfTest {
     /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-882");
+    }
+
+    /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return REPLICATED;
     }


[24/40] incubator-ignite git commit: Merge branches 'ignite-973' and 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-6

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


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 1cc00000a641f3d52964710377a8c29d9ff59947
Parents: 309eeb0 774bc94
Author: S.Vladykin <sv...@gridgain.com>
Authored: Mon Jun 22 09:47:29 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Mon Jun 22 09:47:29 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../processors/query/h2/IgniteH2Indexing.java   |   2 +
 .../query/h2/twostep/GridMapQueryExecutor.java  |  23 ++-
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 179 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 7 files changed, 233 insertions(+), 40 deletions(-)
----------------------------------------------------------------------



[29/40] incubator-ignite git commit: #IGNITE-GG-10449 Fixed license.

Posted by sb...@apache.org.
#IGNITE-GG-10449 Fixed license.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 7c1ecc3dd2e240e48958c8fb1fd812a6df7e92b5
Parents: a3d8e60
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 13:15:55 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 13:15:55 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheOffheapEvictQueryTest.java    | 17 +++++++++++++++++
 1 file changed, 17 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7c1ecc3d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index 72bfcaa..45d744e 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -1,3 +1,20 @@
+/*
+ * 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.*;


[14/40] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-1034' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-1034' into ignite-sprint-6


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 7a8b572bf93b2d72aeb19cdad1f904fed49d5d17
Parents: d699faa d34f329
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 17:27:55 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 17:27:55 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  66 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 18 files changed, 451 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[07/40] incubator-ignite git commit: # ignite-sprint-6 added test for lastSuccessfulAddress

Posted by sb...@apache.org.
# ignite-sprint-6 added test for lastSuccessfulAddress


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: d699faa34d66b8e5f1487daf3386c4d677d8c1e5
Parents: 4d9868a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 13:01:51 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 13:01:51 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 ++++++++++++++++++--
 1 file changed, 41 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d699faa3/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index ad12753..9a44c24 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -149,9 +149,47 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
      */
     public void testThreeNodesStartStop() throws Exception {
         try {
-            startGrid(1);
-            startGrid(2);
-            startGrid(3);
+            IgniteEx ignite1 = startGrid(1);
+            IgniteEx ignite2 = startGrid(2);
+            IgniteEx ignite3 = startGrid(3);
+
+            TcpDiscoverySpi spi1 = (TcpDiscoverySpi)ignite1.configuration().getDiscoverySpi();
+            TcpDiscoverySpi spi2 = (TcpDiscoverySpi)ignite2.configuration().getDiscoverySpi();
+            TcpDiscoverySpi spi3 = (TcpDiscoverySpi)ignite3.configuration().getDiscoverySpi();
+
+            TcpDiscoveryNode node = (TcpDiscoveryNode)spi1.getNode(ignite2.localNode().id());
+
+            assertNotNull(node);
+            assertNotNull(node.lastSuccessfulAddress());
+
+            LinkedHashSet<InetSocketAddress> addrs = spi1.getNodeAddresses(node);
+
+            assertEquals(addrs.iterator().next(), node.lastSuccessfulAddress());
+
+            assertTrue(spi1.pingNode(ignite3.localNode().id()));
+
+            node = (TcpDiscoveryNode)spi1.getNode(ignite3.localNode().id());
+
+            assertNotNull(node);
+            assertNotNull(node.lastSuccessfulAddress());
+
+            addrs = spi1.getNodeAddresses(node);
+            assertEquals(addrs.iterator().next(), node.lastSuccessfulAddress());
+
+            node = (TcpDiscoveryNode)spi2.getNode(ignite1.localNode().id());
+
+            assertNotNull(node);
+            assertNotNull(node.lastSuccessfulAddress());
+
+            node = (TcpDiscoveryNode)spi2.getNode(ignite3.localNode().id());
+
+            assertNotNull(node);
+            assertNotNull(node.lastSuccessfulAddress());
+
+            node = (TcpDiscoveryNode)spi3.getNode(ignite1.localNode().id());
+
+            assertNotNull(node);
+            assertNotNull(node.lastSuccessfulAddress());
         }
         finally {
             stopAllGrids();


[30/40] incubator-ignite git commit: Merge branch 'ignite-gg-10449' into ignite-sprint-6

Posted by sb...@apache.org.
Merge branch 'ignite-gg-10449' into ignite-sprint-6


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: de0930d3b92141a360ea6a2cc346035158572dcf
Parents: a4ce455 7c1ecc3
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 14:05:28 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 14:05:28 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 23 ++++++++++++++++++++
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 17 +++++++++++++++
 2 files changed, 40 insertions(+)
----------------------------------------------------------------------


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


[38/40] incubator-ignite git commit: [IGNITE-980]: Investigate if we should close() the FileSystems in MapRed task implementations.

Posted by sb...@apache.org.
[IGNITE-980]: Investigate if we should close() the FileSystems in MapRed task implementations.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 3b49184b48ca55d5cf4dcdd7485ab8aba1bb9cff
Parents: 36433bc
Author: iveselovskiy <iv...@gridgain.com>
Authored: Tue Jun 23 13:58:06 2015 +0300
Committer: iveselovskiy <iv...@gridgain.com>
Committed: Tue Jun 23 13:58:06 2015 +0300

----------------------------------------------------------------------
 .../processors/hadoop/HadoopJobInfo.java        |   4 +-
 .../hadoop/counter/HadoopCounterWriter.java     |   5 +-
 modules/hadoop/pom.xml                          |  78 ------
 .../fs/IgniteHadoopFileSystemCounterWriter.java |   9 +-
 .../processors/hadoop/HadoopClassLoader.java    |  29 +++
 .../processors/hadoop/HadoopDefaultJobInfo.java |  27 +--
 .../internal/processors/hadoop/HadoopUtils.java | 237 -------------------
 .../hadoop/SecondaryFileSystemProvider.java     |   3 +-
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  11 +
 .../hadoop/fs/HadoopLazyConcurrentMap.java      |   5 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |  25 +-
 .../child/HadoopChildProcessRunner.java         |   3 +-
 .../processors/hadoop/v2/HadoopV2Job.java       |  84 ++++++-
 .../hadoop/v2/HadoopV2JobResourceManager.java   |  22 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |  37 ++-
 .../apache/ignite/igfs/IgfsEventsTestSuite.java |   5 +-
 .../processors/hadoop/HadoopMapReduceTest.java  |   2 +-
 .../processors/hadoop/HadoopTasksV1Test.java    |   7 +-
 .../processors/hadoop/HadoopTasksV2Test.java    |   7 +-
 .../processors/hadoop/HadoopV2JobSelfTest.java  |   6 +-
 .../collections/HadoopAbstractMapTest.java      |   3 +-
 .../testsuites/IgniteHadoopTestSuite.java       |   2 +-
 .../IgniteIgfsLinuxAndMacOSTestSuite.java       |   3 +-
 23 files changed, 237 insertions(+), 377 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
index 51faf5d..d3735d5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobInfo.java
@@ -55,12 +55,14 @@ public interface HadoopJobInfo extends Serializable {
      * This method will be called once for the same ID on one node, though it can be called on the same host
      * multiple times from different processes (in case of multiple nodes on the same host or external execution).
      *
+     * @param jobCls The job class.
      * @param jobId Job ID.
      * @param log Logger.
      * @return Job.
      * @throws IgniteCheckedException If failed.
      */
-    HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
+    public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+        HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException;
 
     /**
      * @return Number of reducers configured for job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
index ce67c57..f21a1e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/counter/HadoopCounterWriter.java
@@ -28,10 +28,9 @@ public interface HadoopCounterWriter {
     /**
      * Writes counters of given job to some statistics storage.
      *
-     * @param jobInfo Job info.
-     * @param jobId Job id.
+     * @param job The job.
      * @param cntrs Counters.
      * @throws IgniteCheckedException If failed.
      */
-    public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs) throws IgniteCheckedException;
+    public void write(HadoopJob job, HadoopCounters cntrs) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/modules/hadoop/pom.xml b/modules/hadoop/pom.xml
index 637f097..e300ba0 100644
--- a/modules/hadoop/pom.xml
+++ b/modules/hadoop/pom.xml
@@ -143,84 +143,6 @@
                 <artifactId>maven-surefire-plugin</artifactId>
                 <version>2.17</version>
                 <configuration>
-                    <classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-annotations</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-auth</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-codec:commons-codec</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.httpcomponents:httpclient</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.httpcomponents:httpcore</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-common</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.guava:guava</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-cli:commons-cli</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.commons:commons-math3</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>xmlenc:xmlenc</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-httpclient:commons-httpclient</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-net:commons-net</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.servlet:servlet-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.mortbay.jetty:jetty</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.mortbay.jetty:jetty-util</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-core</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-json</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jettison:jettison</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.xml.bind:jaxb-impl</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-jaxrs</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-xc</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-server</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>asm:asm</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>tomcat:jasper-compiler</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>tomcat:jasper-runtime</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.servlet.jsp:jsp-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-el:commons-el</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-logging:commons-logging</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>net.java.dev.jets3t:jets3t</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.jamesmurty.utils:java-xmlbuilder</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-core-asl</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.codehaus.jackson:jackson-mapper-asl
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.avro:avro</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.thoughtworks.paranamer:paranamer</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.xerial.snappy:snappy-java</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.protobuf:protobuf-java</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.jcraft:jsch</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.code.findbugs:jsr305</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.zookeeper:zookeeper</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>org.apache.commons:commons-compress</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.tukaani:xz</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-hdfs</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-daemon:commons-daemon</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-mapreduce-client-common
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-common</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.xml.bind:jaxb-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.xml.stream:stax-api</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.activation:activation</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.inject:guice</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>javax.inject:javax.inject</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey.contribs:jersey-guice</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-client</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.sun.jersey:jersey-client</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-yarn-server-common
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.google.inject.extensions:guice-servlet
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>io.netty:netty</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.apache.hadoop:hadoop-mapreduce-client-core
-                        </classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-beanutils:commons-beanutils</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.hamcrest:hamcrest-core</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-collections:commons-collections</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-http</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>commons-io:commons-io</classpathDependencyExcludes>
-                        <!--<classpathDependencyExcludes>commons-lang:commons-lang</classpathDependencyExcludes>-->
-                        <classpathDependencyExcludes>commons-configuration:commons-configuration
-                        </classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-server</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-util</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>org.eclipse.jetty:jetty-io</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>aopalliance:aopalliance</classpathDependencyExcludes>
-                        <classpathDependencyExcludes>com.beust:jcommander</classpathDependencyExcludes>
-                    </classpathDependencyExcludes>
                 </configuration>
             </plugin>
         </plugins>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
index d910507..f76b354 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/hadoop/fs/IgniteHadoopFileSystemCounterWriter.java
@@ -25,6 +25,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -48,11 +49,15 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
     private static final String DEFAULT_COUNTER_WRITER_DIR = "/user/" + USER_MACRO;
 
     /** {@inheritDoc} */
-    @Override public void write(HadoopJobInfo jobInfo, HadoopJobId jobId, HadoopCounters cntrs)
+    @Override public void write(HadoopJob job, HadoopCounters cntrs)
         throws IgniteCheckedException {
 
         Configuration hadoopCfg = HadoopUtils.safeCreateConfiguration();
 
+        final HadoopJobInfo jobInfo = job.info();
+
+        final HadoopJobId jobId = job.id();
+
         for (Map.Entry<String, String> e : ((HadoopDefaultJobInfo)jobInfo).properties().entrySet())
             hadoopCfg.set(e.getKey(), e.getValue());
 
@@ -72,7 +77,7 @@ public class IgniteHadoopFileSystemCounterWriter implements HadoopCounterWriter
         try {
             hadoopCfg.set(MRJobConfig.USER_NAME, user);
 
-            FileSystem fs = HadoopUtils.fileSystemForMrUser(jobStatPath.toUri(), hadoopCfg, true);
+            FileSystem fs = ((HadoopV2Job)job).fileSystem(jobStatPath.toUri(), hadoopCfg);
 
             fs.mkdirs(jobStatPath);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
index eb98ff9..0988fe0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopClassLoader.java
@@ -67,6 +67,28 @@ public class HadoopClassLoader extends URLClassLoader {
     private final String name;
 
     /**
+     * Gets name for Job class loader. The name is specific for local node id.
+     * @param locNodeId The local node id.
+     * @return The class loader name.
+     */
+    public static String nameForJob(UUID locNodeId) {
+        return "hadoop-job-node-" + locNodeId.toString();
+    }
+
+    /**
+     * Gets name for the task class loader. Task class loader
+     * @param info The task info.
+     * @param prefix Get only prefix (without task type and number)
+     * @return The class loader name.
+     */
+    public static String nameForTask(HadoopTaskInfo info, boolean prefix) {
+        if (prefix)
+            return "hadoop-task-" + info.jobId() + "-";
+        else
+            return "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber();
+    }
+
+    /**
      * @param urls Urls.
      */
     public HadoopClassLoader(URL[] urls, String name) {
@@ -568,4 +590,11 @@ public class HadoopClassLoader extends URLClassLoader {
     @Override public String toString() {
         return S.toString(HadoopClassLoader.class, this);
     }
+
+    /**
+     * Getter for name field.
+     */
+    public String name() {
+        return name;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
index 2e855d0..95e03c8 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopDefaultJobInfo.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -48,9 +47,6 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     /** User name. */
     private String user;
 
-    /** */
-    private static volatile Class<?> jobCls;
-
     /**
      * Default constructor required by {@link Externalizable}.
      */
@@ -82,24 +78,15 @@ public class HadoopDefaultJobInfo implements HadoopJobInfo, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
-        try {
-            Class<?> jobCls0 = jobCls;
+    @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+            HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        assert jobCls != null;
 
-            if (jobCls0 == null) { // It is enough to have only one class loader with only Hadoop classes.
-                synchronized (HadoopDefaultJobInfo.class) {
-                    if ((jobCls0 = jobCls) == null) {
-                        HadoopClassLoader ldr = new HadoopClassLoader(null, "hadoop-job");
-
-                        jobCls = jobCls0 = ldr.loadClass(HadoopV2Job.class.getName());
-                    }
-                }
-            }
-
-            Constructor<?> constructor = jobCls0.getConstructor(HadoopJobId.class, HadoopDefaultJobInfo.class,
-                IgniteLogger.class);
+        try {
+            Constructor<? extends HadoopJob> constructor = jobCls.getConstructor(HadoopJobId.class,
+                HadoopDefaultJobInfo.class, IgniteLogger.class);
 
-            return (HadoopJob)constructor.newInstance(jobId, this, log);
+            return constructor.newInstance(jobId, this, log);
         }
         // NB: java.lang.NoClassDefFoundError may be thrown from Class#getConstructor() call.
         catch (Throwable t) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
index 68a9ef6..f87e610 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopUtils.java
@@ -26,16 +26,10 @@ import org.apache.hadoop.mapreduce.JobPriority;
 import org.apache.hadoop.mapreduce.JobStatus;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.ignite.*;
-import org.apache.ignite.hadoop.fs.v1.*;
-import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v2.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
 
 import java.io.*;
-import java.net.*;
 import java.util.*;
 
 /**
@@ -63,34 +57,6 @@ public class HadoopUtils {
     /** Old reducer class attribute. */
     private static final String OLD_REDUCE_CLASS_ATTR = "mapred.reducer.class";
 
-    /** Lazy per-user cache for the file systems. It is cleared and nulled in #close() method. */
-    private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fileSysLazyMap = new HadoopLazyConcurrentMap<>(
-        new HadoopLazyConcurrentMap.ValueFactory<FsCacheKey, FileSystem>() {
-            @Override public FileSystem createValue(FsCacheKey key) {
-                try {
-                    assert key != null;
-
-                    // Explicitly disable FileSystem caching:
-                    URI uri = key.uri();
-
-                    String scheme = uri.getScheme();
-
-                    // Copy the configuration to avoid altering the external object.
-                    Configuration cfg = new Configuration(key.configuration());
-
-                    String prop = HadoopUtils.disableFsCachePropertyName(scheme);
-
-                    cfg.setBoolean(prop, true);
-
-                    return FileSystem.get(uri, cfg, key.user());
-                }
-                catch (IOException | InterruptedException ioe) {
-                    throw new IgniteException(ioe);
-                }
-            }
-        }
-    );
-
     /**
      * Constructor.
      */
@@ -378,207 +344,4 @@ public class HadoopUtils {
         }
     }
 
-    /**
-     * Gets non-null user name as per the Hadoop viewpoint.
-     * @param cfg the Hadoop job configuration, may be null.
-     * @return the user name, never null.
-     */
-    private static String getMrHadoopUser(Configuration cfg) throws IOException {
-        String user = cfg.get(MRJobConfig.USER_NAME);
-
-        if (user == null)
-            user = IgniteHadoopFileSystem.getFsHadoopUser();
-
-        return user;
-    }
-
-    /**
-     * Common method to get the V1 file system in MapRed engine.
-     * It creates the filesystem for the user specified in the
-     * configuration with {@link MRJobConfig#USER_NAME} property.
-     * @param uri the file system uri.
-     * @param cfg the configuration.
-     * @return the file system
-     * @throws IOException
-     */
-    public static FileSystem fileSystemForMrUser(@Nullable URI uri, Configuration cfg, boolean doCacheFs) throws IOException {
-        final String usr = getMrHadoopUser(cfg);
-
-        assert usr != null;
-
-        if (uri == null)
-            uri = FileSystem.getDefaultUri(cfg);
-
-        final FileSystem fs;
-
-        if (doCacheFs) {
-            try {
-                fs = getWithCaching(uri, cfg, usr);
-            }
-            catch (IgniteException ie) {
-                throw new IOException(ie);
-            }
-        }
-        else {
-            try {
-                fs = FileSystem.get(uri, cfg, usr);
-            }
-            catch (InterruptedException ie) {
-                Thread.currentThread().interrupt();
-
-                throw new IOException(ie);
-            }
-        }
-
-        assert fs != null;
-        assert !(fs instanceof IgniteHadoopFileSystem) || F.eq(usr, ((IgniteHadoopFileSystem)fs).user());
-
-        return fs;
-    }
-
-    /**
-     * Note that configuration is not a part of the key.
-     * It is used solely to initialize the first instance
-     * that is created for the key.
-     */
-    public static final class FsCacheKey {
-        /** */
-        private final URI uri;
-
-        /** */
-        private final String usr;
-
-        /** */
-        private final String equalityKey;
-
-        /** */
-        private final Configuration cfg;
-
-        /**
-         * Constructor
-         */
-        public FsCacheKey(URI uri, String usr, Configuration cfg) {
-            assert uri != null;
-            assert usr != null;
-            assert cfg != null;
-
-            this.uri = fixUri(uri, cfg);
-            this.usr = usr;
-            this.cfg = cfg;
-
-            this.equalityKey = createEqualityKey();
-        }
-
-        /**
-         * Creates String key used for equality and hashing.
-         */
-        private String createEqualityKey() {
-            GridStringBuilder sb = new GridStringBuilder("(").a(usr).a(")@");
-
-            if (uri.getScheme() != null)
-                sb.a(uri.getScheme().toLowerCase());
-
-            sb.a("://");
-
-            if (uri.getAuthority() != null)
-                sb.a(uri.getAuthority().toLowerCase());
-
-            return sb.toString();
-        }
-
-        /**
-         * The URI.
-         */
-        public URI uri() {
-            return uri;
-        }
-
-        /**
-         * The User.
-         */
-        public String user() {
-            return usr;
-        }
-
-        /**
-         * The Configuration.
-         */
-        public Configuration configuration() {
-            return cfg;
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("SimplifiableIfStatement")
-        @Override public boolean equals(Object obj) {
-            if (obj == this)
-                return true;
-
-            if (obj == null || getClass() != obj.getClass())
-                return false;
-
-            return equalityKey.equals(((FsCacheKey)obj).equalityKey);
-        }
-
-        /** {@inheritDoc} */
-        @Override public int hashCode() {
-            return equalityKey.hashCode();
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return equalityKey;
-        }
-    }
-
-    /**
-     * Gets FileSystem caching it in static Ignite cache. The cache is a singleton
-     * for each class loader.
-     *
-     * <p/>Note that the file systems in the cache are keyed by a triplet {scheme, authority, user}.
-     * The Configuration is not a part of the key. This means that for the given key file system is
-     * initialized only once with the Configuration passed in upon the file system creation.
-     *
-     * @param uri The file system URI.
-     * @param cfg The configuration.
-     * @param usr The user to create file system for.
-     * @return The file system: either created, or taken from the cache.
-     */
-    private static FileSystem getWithCaching(URI uri, Configuration cfg, String usr) {
-        FsCacheKey key = new FsCacheKey(uri, usr, cfg);
-
-        return fileSysLazyMap.getOrCreate(key);
-    }
-
-    /**
-     * Gets the property name to disable file system cache.
-     * @param scheme The file system URI scheme.
-     * @return The property name. If scheme is null,
-     * returns "fs.null.impl.disable.cache".
-     */
-    public static String disableFsCachePropertyName(@Nullable String scheme) {
-        return String.format("fs.%s.impl.disable.cache", scheme);
-    }
-
-    /**
-     * Takes Fs URI using logic similar to that used in FileSystem#get(1,2,3).
-     * @param uri0 The uri.
-     * @param cfg The cfg.
-     * @return Correct URI.
-     */
-    public static URI fixUri(URI uri0, Configuration cfg) {
-        if (uri0 == null)
-            return FileSystem.getDefaultUri(cfg);
-
-        String scheme = uri0.getScheme();
-        String authority = uri0.getAuthority();
-
-        if (authority == null) {
-            URI dfltUri = FileSystem.getDefaultUri(cfg);
-
-            if (scheme == null || (scheme.equals(dfltUri.getScheme()) && dfltUri.getAuthority() != null))
-                return dfltUri;
-        }
-
-        return uri0;
-    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
index dd679de..ef04b0f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/SecondaryFileSystemProvider.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.hadoop;
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.security.*;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -76,7 +77,7 @@ public class SecondaryFileSystemProvider {
         }
 
         // Disable caching:
-        String prop = HadoopUtils.disableFsCachePropertyName(uri.getScheme());
+        String prop = HadoopFileSystemsUtils.disableFsCachePropertyName(uri.getScheme());
 
         cfg.setBoolean(prop, true);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
index d90bc28..382bbd0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopFileSystemsUtils.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.hadoop.fs;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
+import org.jetbrains.annotations.*;
 
 /**
  * Utilities for configuring file systems to support the separate working directory per each thread.
@@ -37,4 +38,14 @@ public class HadoopFileSystemsUtils {
         cfg.set("fs.AbstractFileSystem." + FsConstants.LOCAL_FS_URI.getScheme() + ".impl",
                 HadoopLocalFileSystemV2.class.getName());
     }
+
+    /**
+     * Gets the property name to disable file system cache.
+     * @param scheme The file system URI scheme.
+     * @return The property name. If scheme is null,
+     * returns "fs.null.impl.disable.cache".
+     */
+    public static String disableFsCachePropertyName(@Nullable String scheme) {
+        return String.format("fs.%s.impl.disable.cache", scheme);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
index 71b38c4..c7565d3 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/fs/HadoopLazyConcurrentMap.java
@@ -51,6 +51,8 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
      */
     public HadoopLazyConcurrentMap(ValueFactory<K, V> factory) {
         this.factory = factory;
+
+        assert getClass().getClassLoader() == Ignite.class.getClassLoader();
     }
 
     /**
@@ -105,6 +107,9 @@ public class HadoopLazyConcurrentMap<K, V extends Closeable> {
         closeLock.writeLock().lock();
 
         try {
+            if (closed)
+                return;
+
             closed = true;
 
             Exception err = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 2f07817..194ae33 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -82,6 +83,9 @@ public class HadoopJobTracker extends HadoopComponent {
     /** Component busy lock. */
     private GridSpinReadWriteLock busyLock;
 
+    /** Class to create HadoopJob instances from. */
+    private Class<? extends HadoopJob> jobCls;
+
     /** Closure to check result of async transform of system cache. */
     private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
         @Override public void apply(IgniteInternalFuture<?> gridFut) {
@@ -95,12 +99,27 @@ public class HadoopJobTracker extends HadoopComponent {
     };
 
     /** {@inheritDoc} */
-    @Override public void start(HadoopContext ctx) throws IgniteCheckedException {
+    @SuppressWarnings("unchecked")
+    @Override public void start(final HadoopContext ctx) throws IgniteCheckedException {
         super.start(ctx);
 
         busyLock = new GridSpinReadWriteLock();
 
         evtProcSvc = Executors.newFixedThreadPool(1);
+
+        UUID nodeId = ctx.localNodeId();
+
+        assert jobCls == null;
+
+        HadoopClassLoader ldr = new HadoopClassLoader(null, HadoopClassLoader.nameForJob(nodeId));
+
+        try {
+            jobCls = (Class<HadoopV2Job>)ldr.loadClass(HadoopV2Job.class.getName());
+        }
+        catch (Exception ioe) {
+            throw new IgniteCheckedException("Failed to load job class [class="
+                + HadoopV2Job.class.getName() + ']', ioe);
+        }
     }
 
     /**
@@ -838,7 +857,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                             HadoopCounters cntrs = meta.counters();
 
-                            writer.write(job.info(), jobId, cntrs);
+                            writer.write(job, cntrs);
                         }
                     }
                     catch (Exception e) {
@@ -986,7 +1005,7 @@ public class HadoopJobTracker extends HadoopComponent {
                 jobInfo = meta.jobInfo();
             }
 
-            job = jobInfo.createJob(jobId, log);
+            job = jobInfo.createJob(jobCls, jobId, log);
 
             job.initialize(false, ctx.localNodeId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index 040552a..b0b0b8c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.processors.hadoop.shuffle.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.*;
 import org.apache.ignite.internal.processors.hadoop.taskexecutor.external.communication.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.unsafe.*;
@@ -115,7 +116,7 @@ public class HadoopChildProcessRunner {
 
                 assert job == null;
 
-                job = req.jobInfo().createJob(req.jobId(), log);
+                job = req.jobInfo().createJob(HadoopV2Job.class, req.jobId(), log);
 
                 job.initialize(true, nodeDesc.processId());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
index d754039..33a218d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2Job.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop.v2;
 
+import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
 import org.apache.hadoop.io.*;
 import org.apache.hadoop.mapred.*;
@@ -30,15 +31,18 @@ import org.apache.ignite.internal.processors.hadoop.v1.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
 import java.io.*;
 import java.lang.reflect.*;
+import java.net.*;
 import java.util.*;
 import java.util.Queue;
 import java.util.concurrent.*;
 
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.*;
 
 /**
  * Hadoop job implementation for v2 API.
@@ -70,7 +74,10 @@ public class HadoopV2Job implements HadoopJob {
     private final Queue<Class<? extends HadoopTaskContext>> taskCtxClsPool = new ConcurrentLinkedQueue<>();
 
     /** All created contexts. */
-    private final Queue<Class<?>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
+    private final Queue<Class<? extends HadoopTaskContext>> fullCtxClsQueue = new ConcurrentLinkedDeque<>();
+
+    /** File system cache map. */
+    private final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap = createHadoopLazyConcurrentMap();
 
     /** Local node ID */
     private volatile UUID locNodeId;
@@ -103,7 +110,7 @@ public class HadoopV2Job implements HadoopJob {
 
         jobCtx = new JobContextImpl(jobConf, hadoopJobID);
 
-        rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log);
+        rsrcMgr = new HadoopV2JobResourceManager(jobId, jobCtx, log, this);
     }
 
     /** {@inheritDoc} */
@@ -134,7 +141,7 @@ public class HadoopV2Job implements HadoopJob {
             Path jobDir = new Path(jobDirPath);
 
             try {
-                FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf, true);
+                FileSystem fs = fileSystem(jobDir.toUri(), jobConf);
 
                 JobSplit.TaskSplitMetaInfo[] metaInfos = SplitMetaInfoReader.readSplitMetaInfo(hadoopJobID, fs, jobConf,
                     jobDir);
@@ -180,6 +187,7 @@ public class HadoopV2Job implements HadoopJob {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public HadoopTaskContext getTaskContext(HadoopTaskInfo info) throws IgniteCheckedException {
         T2<HadoopTaskType, Integer> locTaskId = new T2<>(info.type(),  info.taskNumber());
 
@@ -201,7 +209,7 @@ public class HadoopV2Job implements HadoopJob {
                 // Note that the classloader identified by the task it was initially created for,
                 // but later it may be reused for other tasks.
                 HadoopClassLoader ldr = new HadoopClassLoader(rsrcMgr.classPath(),
-                    "hadoop-task-" + info.jobId() + "-" + info.type() + "-" + info.taskNumber());
+                    HadoopClassLoader.nameForTask(info, false));
 
                 cls = (Class<? extends HadoopTaskContext>)ldr.loadClass(HadoopV2TaskContext.class.getName());
 
@@ -243,8 +251,13 @@ public class HadoopV2Job implements HadoopJob {
 
     /** {@inheritDoc} */
     @Override public void initialize(boolean external, UUID locNodeId) throws IgniteCheckedException {
+        assert locNodeId != null;
+
         this.locNodeId = locNodeId;
 
+        assert ((HadoopClassLoader)getClass().getClassLoader()).name()
+            .equals(HadoopClassLoader.nameForJob(this.locNodeId));
+
         Thread.currentThread().setContextClassLoader(jobConf.getClassLoader());
 
         try {
@@ -274,17 +287,26 @@ public class HadoopV2Job implements HadoopJob {
             // Stop the daemon threads that have been created
             // with the task class loaders:
             while (true) {
-                Class<?> cls = fullCtxClsQueue.poll();
+                Class<? extends HadoopTaskContext> cls = fullCtxClsQueue.poll();
 
                 if (cls == null)
                     break;
 
                 try {
-                    Class<?> daemonCls = cls.getClassLoader().loadClass(HadoopClassLoader.HADOOP_DAEMON_CLASS_NAME);
+                    final ClassLoader ldr = cls.getClassLoader();
 
-                    Method m = daemonCls.getMethod("dequeueAndStopAll");
+                    try {
+                        // Stop Hadoop daemons for this *task*:
+                        stopHadoopFsDaemons(ldr);
+                    }
+                    catch (Exception e) {
+                        if (err == null)
+                            err = e;
+                    }
 
-                    m.invoke(null);
+                    // Also close all the FileSystems cached in
+                    // HadoopLazyConcurrentMap for this *task* class loader:
+                    closeCachedTaskFileSystems(ldr);
                 }
                 catch (Throwable e) {
                     if (err == null)
@@ -297,11 +319,46 @@ public class HadoopV2Job implements HadoopJob {
 
             assert fullCtxClsQueue.isEmpty();
 
+            try {
+                // Close all cached file systems for this *Job*:
+                fsMap.close();
+            }
+            catch (Exception e) {
+                if (err == null)
+                    err = e;
+            }
+
             if (err != null)
                 throw U.cast(err);
         }
     }
 
+    /**
+     * Stops Hadoop Fs daemon threads.
+     * @param ldr The task ClassLoader to stop the daemons for.
+     * @throws Exception On error.
+     */
+    private void stopHadoopFsDaemons(ClassLoader ldr) throws Exception {
+        Class<?> daemonCls = ldr.loadClass(HadoopClassLoader.HADOOP_DAEMON_CLASS_NAME);
+
+        Method m = daemonCls.getMethod("dequeueAndStopAll");
+
+        m.invoke(null);
+    }
+
+    /**
+     * Closes all the file systems user by task
+     * @param ldr The task class loader.
+     * @throws Exception On error.
+     */
+    private void closeCachedTaskFileSystems(ClassLoader ldr) throws Exception {
+        Class<?> clazz = ldr.loadClass(HadoopV2TaskContext.class.getName());
+
+        Method m = clazz.getMethod("close");
+
+        m.invoke(null);
+    }
+
     /** {@inheritDoc} */
     @Override public void prepareTaskEnvironment(HadoopTaskInfo info) throws IgniteCheckedException {
         rsrcMgr.prepareTaskWorkDir(taskLocalDir(locNodeId, info));
@@ -331,4 +388,15 @@ public class HadoopV2Job implements HadoopJob {
     public JobConf jobConf() {
         return jobConf;
     }
+
+    /**
+     * Gets file system for this job.
+     * @param uri The uri.
+     * @param cfg The configuration.
+     * @return The file system.
+     * @throws IOException On error.
+     */
+    public FileSystem fileSystem(@Nullable URI uri, Configuration cfg) throws IOException {
+        return fileSystemForMrUserWithCaching(uri, cfg, fsMap);
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
index 2f64e77..912cc3f 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2JobResourceManager.java
@@ -39,9 +39,10 @@ import java.util.*;
  * Provides all resources are needed to the job execution. Downloads the main jar, the configuration and additional
  * files are needed to be placed on local files system.
  */
-public class HadoopV2JobResourceManager {
+class HadoopV2JobResourceManager {
     /** File type Fs disable caching property name. */
-    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME = HadoopUtils.disableFsCachePropertyName("file");
+    private static final String FILE_DISABLE_CACHING_PROPERTY_NAME =
+        HadoopFileSystemsUtils.disableFsCachePropertyName("file");
 
     /** Hadoop job context. */
     private final JobContextImpl ctx;
@@ -61,16 +62,20 @@ public class HadoopV2JobResourceManager {
     /** Staging directory to delivery job jar and config to the work nodes. */
     private Path stagingDir;
 
+    /** The job. */
+    private final HadoopV2Job job;
+
     /**
      * Creates new instance.
      * @param jobId Job ID.
      * @param ctx Hadoop job context.
      * @param log Logger.
      */
-    public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteLogger log) {
+    public HadoopV2JobResourceManager(HadoopJobId jobId, JobContextImpl ctx, IgniteLogger log, HadoopV2Job job) {
         this.jobId = jobId;
         this.ctx = ctx;
         this.log = log.getLogger(HadoopV2JobResourceManager.class);
+        this.job = job;
     }
 
     /**
@@ -115,7 +120,7 @@ public class HadoopV2JobResourceManager {
                 stagingDir = new Path(new URI(mrDir));
 
                 if (download) {
-                    FileSystem fs = HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), cfg, true);
+                    FileSystem fs = job.fileSystem(stagingDir.toUri(), cfg);
 
                     if (!fs.exists(stagingDir))
                         throw new IgniteCheckedException("Failed to find map-reduce submission " +
@@ -210,7 +215,7 @@ public class HadoopV2JobResourceManager {
 
             FileSystem dstFs = FileSystem.getLocal(cfg);
 
-            FileSystem srcFs = HadoopUtils.fileSystemForMrUser(srcPath.toUri(), cfg, true);
+            FileSystem srcFs = job.fileSystem(srcPath.toUri(), cfg);
 
             if (extract) {
                 File archivesPath = new File(jobLocDir.getAbsolutePath(), ".cached-archives");
@@ -292,8 +297,11 @@ public class HadoopV2JobResourceManager {
      */
     public void cleanupStagingDirectory() {
         try {
-            if (stagingDir != null)
-                HadoopUtils.fileSystemForMrUser(stagingDir.toUri(), ctx.getJobConf(), true).delete(stagingDir, true);
+            if (stagingDir != null) {
+                FileSystem fs = job.fileSystem(stagingDir.toUri(), ctx.getJobConf());
+
+                fs.delete(stagingDir, true);
+            }
         }
         catch (Exception e) {
             log.error("Failed to remove job staging directory [path=" + stagingDir + ", jobId=" + jobId + ']' , e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
index e89feba..6eff475 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/v2/HadoopV2TaskContext.java
@@ -33,6 +33,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.counter.HadoopCounters;
+import org.apache.ignite.internal.processors.hadoop.fs.*;
 import org.apache.ignite.internal.processors.hadoop.v1.*;
 import org.apache.ignite.internal.processors.igfs.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -44,6 +45,7 @@ import java.security.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.internal.processors.hadoop.fs.HadoopFileSystemCacheUtils.*;
 import static org.apache.ignite.internal.processors.hadoop.fs.HadoopParameters.*;
 import static org.apache.ignite.internal.processors.hadoop.HadoopUtils.*;
 
@@ -54,6 +56,22 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     /** */
     private static final boolean COMBINE_KEY_GROUPING_SUPPORTED;
 
+    /** Lazy per-user file system cache used by the Hadoop task. */
+    private static final HadoopLazyConcurrentMap<FsCacheKey, FileSystem> fsMap
+        = createHadoopLazyConcurrentMap();
+
+    /**
+     * This method is called with reflection upon Job finish with class loader of each task.
+     * This will clean up all the Fs created for specific task.
+     * Each class loader sees uses its own instance of <code>fsMap<code/> since the class loaders
+     * are different.
+     *
+     * @throws IgniteCheckedException On error.
+     */
+    public static void close() throws IgniteCheckedException {
+        fsMap.close();
+    }
+
     /**
      * Check for combiner grouping support (available since Hadoop 2.3).
      */
@@ -91,7 +109,7 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private volatile HadoopTask task;
 
     /** Local node ID */
-    private UUID locNodeId;
+    private final UUID locNodeId;
 
     /** Counters for task. */
     private final HadoopCounters cntrs = new HadoopCountersImpl();
@@ -423,7 +441,22 @@ public class HadoopV2TaskContext extends HadoopTaskContext {
     private Object readExternalSplit(HadoopExternalSplit split) throws IgniteCheckedException {
         Path jobDir = new Path(jobConf().get(MRJobConfig.MAPREDUCE_JOB_DIR));
 
-        try (FileSystem fs = fileSystemForMrUser(jobDir.toUri(), jobConf(), false);
+        FileSystem fs;
+
+        try {
+            // This assertion uses .startsWith() instead of .equals() because task class loaders may
+            // be reused between tasks of the same job.
+            assert ((HadoopClassLoader)getClass().getClassLoader()).name()
+                .startsWith(HadoopClassLoader.nameForTask(taskInfo(), true));
+
+            // We also cache Fs there, all them will be cleared explicitly upon the Job end.
+            fs = fileSystemForMrUserWithCaching(jobDir.toUri(), jobConf(), fsMap);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException(e);
+        }
+
+        try (
             FSDataInputStream in = fs.open(JobSubmissionFiles.getJobSplitFile(jobDir))) {
 
             in.seek(split.offset());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
index fb21e2d..e7c7f8a 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsEventsTestSuite.java
@@ -21,7 +21,6 @@ import junit.framework.*;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.hadoop.fs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.jetbrains.annotations.*;
@@ -38,7 +37,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suite() throws Exception {
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite FS Events Test Suite");
 
@@ -58,7 +57,7 @@ public class IgfsEventsTestSuite extends TestSuite {
      * @throws Exception Thrown in case of the failure.
      */
     public static TestSuite suiteNoarchOnly() throws Exception {
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite IGFS Events Test Suite Noarch Only");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
index 66c14b5..da6f9c7 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopMapReduceTest.java
@@ -272,7 +272,7 @@ public class HadoopMapReduceTest extends HadoopAbstractWordCountTest {
             @Override public boolean apply() {
                 return igfs.exists(statPath);
             }
-        }, 10000);
+        }, 20_000);
 
         final long apiEvtCnt0 = apiEvtCnt;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
index 48e83cc..f59be19 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV1Test.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.hadoop.mapred.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.io.*;
 import java.util.*;
@@ -44,9 +45,11 @@ public class HadoopTasksV1Test extends HadoopTasksAllVersionsTest {
 
         HadoopDefaultJobInfo jobInfo = createJobInfo(jobConf);
 
-        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+        UUID uuid = new UUID(0, 0);
 
-        return jobInfo.createJob(jobId, log);
+        HadoopJobId jobId = new HadoopJobId(uuid, 0);
+
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
index e73fae3..1570807 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopTasksV2Test.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.*;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
+import org.apache.ignite.internal.processors.hadoop.v2.*;
 
 import java.util.*;
 
@@ -62,9 +63,11 @@ public class HadoopTasksV2Test extends HadoopTasksAllVersionsTest {
 
         HadoopDefaultJobInfo jobInfo = createJobInfo(hadoopJob.getConfiguration());
 
-        HadoopJobId jobId = new HadoopJobId(new UUID(0, 0), 0);
+        UUID uuid = new UUID(0, 0);
 
-        return jobInfo.createJob(jobId, log);
+        HadoopJobId jobId = new HadoopJobId(uuid, 0);
+
+        return jobInfo.createJob(HadoopV2Job.class, jobId, log);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
index f3b9307..b8f62e6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/HadoopV2JobSelfTest.java
@@ -68,9 +68,11 @@ public class HadoopV2JobSelfTest extends HadoopAbstractSelfTest {
 
         HadoopDefaultJobInfo info = createJobInfo(cfg);
 
-        HadoopJobId id = new HadoopJobId(UUID.randomUUID(), 1);
+        final UUID uuid = UUID.randomUUID();
 
-        HadoopJob job = info.createJob(id, log);
+        HadoopJobId id = new HadoopJobId(uuid, 1);
+
+        HadoopJob job = info.createJob(HadoopV2Job.class, id, log);
 
         HadoopTaskContext taskCtx = job.getTaskContext(new HadoopTaskInfo(HadoopTaskType.MAP, null, 0, 0,
             null));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
index 9395c5e..b5e2ab5 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/shuffle/collections/HadoopAbstractMapTest.java
@@ -136,7 +136,8 @@ public abstract class HadoopAbstractMapTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public HadoopJob createJob(HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
+        @Override public HadoopJob createJob(Class<? extends HadoopJob> jobCls,
+                HadoopJobId jobId, IgniteLogger log) throws IgniteCheckedException {
             assert false;
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
index 4be5d72..2ab3e8c 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteHadoopTestSuite.java
@@ -50,7 +50,7 @@ public class IgniteHadoopTestSuite extends TestSuite {
         downloadHadoop();
         downloadHive();
 
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        final ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite Hadoop MR Test Suite");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3b49184b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
index 8982d83..22beea6 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/testsuites/IgniteIgfsLinuxAndMacOSTestSuite.java
@@ -20,7 +20,6 @@ package org.apache.ignite.testsuites;
 import junit.framework.*;
 import org.apache.ignite.igfs.*;
 import org.apache.ignite.internal.processors.igfs.*;
-import org.apache.ignite.internal.processors.hadoop.*;
 
 import static org.apache.ignite.testsuites.IgniteHadoopTestSuite.*;
 
@@ -36,7 +35,7 @@ public class IgniteIgfsLinuxAndMacOSTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         downloadHadoop();
 
-        HadoopClassLoader ldr = new HadoopClassLoader(null, "test");
+        ClassLoader ldr = TestSuite.class.getClassLoader();
 
         TestSuite suite = new TestSuite("Ignite IGFS Test Suite For Linux And Mac OS");
 


[03/40] incubator-ignite git commit: # ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test

Posted by sb...@apache.org.
# ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 462495f2977668ae9353adecef554c4f15dd70f3
Parents: 44bbece
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 10:12:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 11:47:14 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |  8 +--
 .../discovery/GridDiscoveryManager.java         | 19 ++++--
 .../continuous/CacheContinuousQueryHandler.java |  8 +++
 .../ignite/internal/util/nio/GridNioServer.java | 13 ++--
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  4 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  6 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 20 +++---
 .../tcp/TcpCommunicationSpiMBean.java           | 11 ++++
 .../ignite/spi/discovery/DiscoverySpi.java      |  3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 12 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 12 +++-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java | 18 +++++
 ...ridFailFastNodeFailureDetectionSelfTest.java | 17 ++++-
 .../IgniteSlowClientDetectionSelfTest.java      | 69 +++++++++++++++++++-
 .../testframework/GridSpiTestContext.java       |  4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 18 files changed, 189 insertions(+), 45 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 885d52c..40a5ea5 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
@@ -480,12 +480,12 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.io().messageFactory();
                     }
 
-                    @Override public boolean tryFailNode(UUID nodeId) {
-                        return ctx.discovery().tryFailNode(nodeId);
+                    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
+                        return ctx.discovery().tryFailNode(nodeId, warning);
                     }
 
-                    @Override public void failNode(UUID nodeId) {
-                        ctx.discovery().failNode(nodeId);
+                    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+                        ctx.discovery().failNode(nodeId, warning);
                     }
 
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 717cdf3..1e4b972 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
@@ -386,9 +386,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
                 else {
-                    minorTopVer = 0;
+                    if (type != EVT_NODE_SEGMENTED) {
+                        minorTopVer = 0;
 
-                    verChanged = true;
+                        verChanged = true;
+                    }
+                    else
+                        verChanged = false;
                 }
 
                 AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
@@ -1481,15 +1485,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning message to be shown on all nodes.
      * @return Whether node is failed.
      */
-    public boolean tryFailNode(UUID nodeId) {
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return false;
 
         try {
             if (!getSpi().pingNode(nodeId)) {
-                getSpi().failNode(nodeId);
+                getSpi().failNode(nodeId, warning);
 
                 return true;
             }
@@ -1503,13 +1508,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID to fail.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public void failNode(UUID nodeId) {
+    public void failNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return;
 
         try {
-            getSpi().failNode(nodeId);
+            getSpi().failNode(nodeId, warning);
         }
         finally {
             busyLock.leaveBusy();
@@ -1520,6 +1526,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.
+     * @param discoCache Discovery cache.
      * @return {@code True} if topology was updated.
      */
     private boolean updateTopologyVersionIfGreater(AffinityTopologyVersion updated, DiscoCache discoCache) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 ad78b92..ff2905f 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
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.managers.deployment.*;
@@ -226,6 +227,13 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
                             ctx.continuous().addNotification(nodeId, routineId, evt.entry(), topic, sync, true);
                         }
+                        catch (ClusterTopologyCheckedException ex) {
+                            IgniteLogger log = ctx.log(getClass());
+
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send event notification to node, node left cluster " +
+                                    "[node=" + nodeId + ", err=" + ex + ']');
+                        }
                         catch (IgniteCheckedException ex) {
                             U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index b9d246a..24e1e08 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -505,13 +505,18 @@ public class GridNioServer<T> {
     public GridNioFuture<GridNioSession> createSession(final SocketChannel ch,
         @Nullable Map<Integer, ?> meta) {
         try {
-            ch.configureBlocking(false);
+            if (!closed) {
+                ch.configureBlocking(false);
 
-            NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, false, meta);
+                NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, false, meta);
 
-            offerBalanced(req);
+                offerBalanced(req);
 
-            return req;
+                return req;
+            }
+            else
+                return new GridNioFinishedFuture<>(
+                    new IgniteCheckedException("Failed to create session, server is stopped."));
         }
         catch (IOException e) {
             return new GridNioFinishedFuture<>(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 18191a1..5e557bd 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
@@ -754,12 +754,12 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public boolean tryFailNode(UUID nodeId) {
+        @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
             return false;
         }
 
         /** {@inheritDoc} */
-        @Override public void failNode(UUID nodeId) {
+        @Override public void failNode(UUID nodeId, @Nullable String warning) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 a655a73..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
@@ -307,14 +307,16 @@ public interface IgniteSpiContext {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      * @return If node was failed.
      */
-    public boolean tryFailNode(UUID nodeId);
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param c Timeout object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 538e9a8..84c1a57 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
@@ -1153,15 +1153,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return msgQueueLimit;
     }
 
-    /**
-     * Gets slow client queue limit.
-     * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
-     * those clients whose queue exceeded this limit.
-     *
-     * @return Slow client queue limit.
-     */
-    public int getSlowClientQueueLimit() {
+    /** {@inheritDoc} */
+    @Override public int getSlowClientQueueLimit() {
         return slowClientQueueLimit;
     }
 
@@ -1923,10 +1916,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
+
+                    LT.warn(log, null, msg);
 
-                    getSpiContext().failNode(id);
+                    getSpiContext().failNode(id, msg);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index fe4f581..1971d99 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -268,4 +268,15 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Maximum number of unacknowledged messages.")
     public int getUnacknowledgedMessagesBufferSize();
+
+    /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    @MXBeanDescription("Slow client queue limit.")
+    public int getSlowClientQueueLimit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index b952087..11a18b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -151,8 +151,9 @@ public interface DiscoverySpi extends IgniteSpi {
      * Initiates failure of provided node.
      *
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * Whether or not discovery is started in client mode.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 fef6f4f..e255e08 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
@@ -335,13 +335,15 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = rmtNodes.get(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -1432,6 +1434,14 @@ class ClientImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
+                if (msg.warning() != null) {
+                    ClusterNode creatorNode = rmtNodes.get(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 8eb82ac..2458f85 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
@@ -584,13 +584,15 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = ring.node(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -3409,6 +3411,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         worker.interrupt();
                 }
 
+                if (msg.warning() != null && !msg.creatorNodeId().equals(getLocalNodeId())) {
+                    ClusterNode creatorNode = ring.node(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 94097c9..ace917f 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
@@ -175,8 +175,9 @@ abstract class TcpDiscoveryImpl {
 
     /**
      * @param nodeId Node id.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public abstract void failNode(UUID nodeId);
+    public abstract void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param gridName Grid name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 c36ac76..1d1916a 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
@@ -373,8 +373,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        impl.failNode(nodeId);
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+        impl.failNode(nodeId, warning);
     }
 
     /** {@inheritDoc} */
@@ -385,7 +385,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** {@inheritDoc} */
     @Override public boolean isClientMode() {
         if (impl == null)
-            throw new IllegalStateException("TcpDiscoverySpi has not started");
+            throw new IllegalStateException("TcpDiscoverySpi has not started.");
 
         return impl instanceof ClientImpl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
index 8cb8414..93ecdaa 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -37,6 +38,9 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     /** Internal order of the failed node. */
     private final long order;
 
+    /** */
+    private String warning;
+
     /**
      * Constructor.
      *
@@ -55,6 +59,20 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
+     * @param warning Warning message to be shown on all nodes.
+     */
+    public void warning(String warning) {
+        this.warning = warning;
+    }
+
+    /**
+     * @return Warning message to be shown on all nodes.
+     */
+    @Nullable public String warning() {
+        return warning;
+    }
+
+    /**
      * Gets ID of the failed node.
      *
      * @return ID of the failed node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 992d7bf..238115d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -50,7 +50,12 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(IP_FINDER);
-        disco.setHeartbeatFrequency(10000);
+        disco.setHeartbeatFrequency(10_000);
+
+        // Set parameters for fast ping failure.
+        disco.setSocketTimeout(100);
+        disco.setNetworkTimeout(100);
+        disco.setReconnectCount(2);
 
         cfg.setDiscoverySpi(disco);
 
@@ -66,8 +71,6 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testFailFast() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-933");
-
         startGridsMultiThreaded(5);
 
         final CountDownLatch failLatch = new CountDownLatch(4);
@@ -87,6 +90,8 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         Ignite ignite1 = ignite(0);
         Ignite ignite2 = ignite(1);
 
+        final CountDownLatch evtLatch = new CountDownLatch(1);
+
         ignite1.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
                 respond(rcvMsg);
@@ -95,12 +100,18 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         ignite2.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
+                evtLatch.countDown();
+
                 respond(rcvMsg);
             }
         });
 
         ignite1.message(ignite1.cluster().forRemotes()).send(null, "Message");
 
+        evtLatch.await(); // Wait when connection is established.
+
+        log.info("Fail node: " + ignite1.cluster().localNode());
+
         failNode(ignite1);
 
         assert failLatch.await(1000, MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/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 09b4215..27c2a61 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
@@ -19,24 +19,37 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.tcp.*;
+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 javax.cache.event.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
 
 /**
  *
  */
 public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
-
+    /** */
     public static final String PARTITIONED = "partitioned";
 
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @return Node count.
      */
@@ -48,6 +61,8 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
         if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
             cfg.setClientMode(true);
 
@@ -66,7 +81,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(nodeCount());
+        startGrids(nodeCount());
     }
 
     /** {@inheritDoc} */
@@ -82,6 +97,45 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     public void testSlowClient() throws Exception {
         final IgniteEx slowClient = grid(nodeCount() - 1);
 
+        final ClusterNode slowClientNode = slowClient.localNode();
+
+        final CountDownLatch evtSegmentedLatch = new CountDownLatch(1);
+
+        slowClient.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_SEGMENTED);
+
+                DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+                assertEquals(slowClientNode, evt0.eventNode());
+                assertEquals(5L, evt0.topologyVersion());
+
+                evtSegmentedLatch.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        final CountDownLatch evtFailedLatch = new CountDownLatch(nodeCount() - 1);
+
+        for (int i = 0; i < nodeCount() - 1; i++) {
+            grid(i).events().localListen(new IgnitePredicate<Event>() {
+                @Override public boolean apply(Event evt) {
+                    assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_FAILED);
+
+                    DiscoveryEvent evt0 = (DiscoveryEvent) evt;
+
+                    assertEquals(slowClientNode, evt0.eventNode());
+                    assertEquals(6L, evt0.topologyVersion());
+                    assertEquals(4, evt0.topologyNodes().size());
+
+                    evtFailedLatch.countDown();
+
+                    return false;
+                }
+            }, EventType.EVT_NODE_FAILED);
+        }
+
         assertTrue(slowClient.cluster().localNode().isClient());
 
         IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
@@ -109,14 +163,23 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < 100; i++)
             cache0.put(0, new byte[10 * 1024]);
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
             }
         }, getTestTimeout());
+
+        assertTrue(wait);
+
+        assertTrue("Failed to wait for client failed event", evtFailedLatch.await(5000, MILLISECONDS));
+        assertTrue("Failed to wait for client segmented event", evtSegmentedLatch.await(5000, MILLISECONDS));
     }
 
+    /**
+     *
+     */
     private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        /** {@inheritDoc} */
         @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
             System.out.println(">>>> Received update: " + iterable);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index c20ff2e..08268af 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -497,12 +497,12 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tryFailNode(UUID nodeId) {
+    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/462495f2/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index b4977ce..2d14728 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -91,6 +91,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridMessageListenSelfTest.class);
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
+        suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
 
         return suite;
     }


[33/40] incubator-ignite git commit: # ignite-sprint-7: fix compilation after merge.

Posted by sb...@apache.org.
# ignite-sprint-7: fix compilation after merge.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 5d8e404f8d871f96a11238d865c6a7c656080638
Parents: cd06e1d
Author: ashutak <as...@gridgain.com>
Authored: Mon Jun 22 16:28:29 2015 +0300
Committer: ashutak <as...@gridgain.com>
Committed: Mon Jun 22 16:28:29 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5d8e404f/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index 14093ba..181ff0c 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -62,7 +62,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryMultiThreadedOffHeapTieredSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
-        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+        suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);


[09/40] incubator-ignite git commit: # ignite-1034

Posted by sb...@apache.org.
# ignite-1034


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: d34f3294fbe120c443448e4df201fc9337725706
Parents: 895771a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 13:27:26 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 13:27:26 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/communication/tcp/TcpCommunicationSpi.java       | 5 -----
 1 file changed, 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d34f3294/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 bbb9b1c..a540b7c 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
@@ -611,11 +611,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @LoggerResource
     private IgniteLogger log;
 
-    /** Ignite. */
-    @IgniteInstanceResource
-    @GridToStringExclude
-    private Ignite ignite;
-
     /** Local IP address. */
     private String locAddr;
 


[25/40] incubator-ignite git commit: # ignite-sprint-6 minor

Posted by sb...@apache.org.
# ignite-sprint-6 minor


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 6f5e676a2599a4e5441dc51f67763bdd10b2db24
Parents: 1cc0000
Author: sboikov <sb...@gridgain.com>
Authored: Mon Jun 22 11:06:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 11:06:38 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 45 ++++++++++----------
 1 file changed, 23 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6f5e676a/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 e255e08..b524e13 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
@@ -36,6 +36,7 @@ import java.io.*;
 import java.net.*;
 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.*;
@@ -79,7 +80,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     private volatile long topVer;
 
     /** Join error. Contains error what occurs on join process. */
-    private IgniteSpiException joinErr;
+    private final AtomicReference<IgniteSpiException> joinErr = new AtomicReference<>();
 
     /** Joined latch. */
     private final CountDownLatch joinLatch = new CountDownLatch(1);
@@ -171,8 +172,10 @@ class ClientImpl extends TcpDiscoveryImpl {
         try {
             joinLatch.await();
 
-            if (joinErr != null)
-                throw joinErr;
+            IgniteSpiException err = joinErr.get();
+
+            if (err != null)
+                throw err;
         }
         catch (InterruptedException e) {
             throw new IgniteSpiException("Thread has been interrupted.", e);
@@ -645,7 +648,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     private void joinError(IgniteSpiException err) {
         assert err != null;
 
-        joinErr = err;
+        joinErr.compareAndSet(null, err);
 
         joinLatch.countDown();
     }
@@ -1000,8 +1003,16 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (log.isDebugEnabled())
                             log.error("Reconnect error [join=" + join + ", timeout=" + timeout + ']', e);
 
-                        if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
+                        if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout) {
+                            String msg = join ? "Failed to connect to cluster (consider increasing 'joinTimeout' " +
+                                "configuration  property) [joinTimeout=" + spi.joinTimeout + ", err=" + e + ']' :
+                                "Failed to reconnect to cluster (consider increasing 'networkTimeout' " +
+                                    "configuration  property) [networkTimeout=" + spi.netTimeout + ", err=" + e + ']';
+
+                            U.warn(log, msg);
+
                             throw e;
+                        }
                         else
                             U.warn(log, "Failed to reconnect to cluster (will retry): " + e);
                     }
@@ -1062,9 +1073,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 final Socket sock = joinTopology(false, spi.joinTimeout);
 
                 if (sock == null) {
-                    joinErr = new IgniteSpiException("Join process timed out.");
-
-                    joinLatch.countDown();
+                    joinError(new IgniteSpiException("Join process timed out."));
 
                     return;
                 }
@@ -1089,11 +1098,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (msg == JOIN_TIMEOUT) {
                         if (joinLatch.getCount() > 0) {
-                            joinErr = new IgniteSpiException("Join process timed out, did not receive response for " +
+                            joinError(new IgniteSpiException("Join process timed out, did not receive response for " +
                                 "join request (consider increasing 'joinTimeout' configuration property) " +
-                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + sock +']');
-
-                            joinLatch.countDown();
+                                "[joinTimeout=" + spi.joinTimeout + ", sock=" + sock +']'));
 
                             break;
                         }
@@ -1159,9 +1166,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 err = spi.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
 
                             if (err != null) {
-                                joinErr = err;
-
-                                joinLatch.countDown();
+                                joinError(err);
 
                                 break;
                             }
@@ -1174,12 +1179,8 @@ class ClientImpl extends TcpDiscoveryImpl {
             finally {
                 U.closeQuiet(currSock);
 
-                if (joinLatch.getCount() > 0) {
-                    // This should not occurs.
-                    joinErr = new IgniteSpiException("Some error in join process.");
-
-                    joinLatch.countDown();
-                }
+                if (joinLatch.getCount() > 0)
+                    joinError(new IgniteSpiException("Some error in join process.")); // This should not occur.
 
                 if (reconnector != null) {
                     reconnector.cancel();
@@ -1297,7 +1298,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer, msg));
 
-                    joinErr = null;
+                    joinErr.set(null);;
 
                     joinLatch.countDown();
 


[11/40] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-973


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: bcfbb9236e860115dfc70da9f717aaba7cb9544c
Parents: 23512df d699faa
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 19 13:31:39 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 19 13:31:39 2015 +0300

----------------------------------------------------------------------
 examples/pom.xml                                |  2 +-
 modules/aop/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 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 42 +++++++++----------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 35 ++++++++++++----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 42 +++++++++++++------
 .../tcp/internal/TcpDiscoveryNode.java          | 18 ++++++++
 .../core/src/main/resources/ignite.properties   |  2 +-
 .../apache/ignite/internal/GridSelfTest.java    | 20 +++++----
 .../DataStreamerMultiThreadedSelfTest.java      |  3 ++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 ++++++++++++++++++--
 modules/extdata/p2p/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/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 +-
 pom.xml                                         |  2 +-
 44 files changed, 189 insertions(+), 89 deletions(-)
----------------------------------------------------------------------



[04/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-1034' into ignite-1034

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


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: ef4abeba881f079cab08af9ade99fe52ecf5d26d
Parents: 462495f 49dc4a5
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 12:02:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 12:02:31 2015 +0300

----------------------------------------------------------------------

----------------------------------------------------------------------



[02/40] incubator-ignite git commit: # ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test

Posted by sb...@apache.org.
# ignite-1034 fixed assert in discovery manager, warning on all nodes, improved test


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 49dc4a50197b8a7203bfc0684d5f77f46bda0297
Parents: 44bbece
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 10:12:20 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Jun 19 11:41:00 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |  8 +--
 .../discovery/GridDiscoveryManager.java         | 19 ++++--
 .../continuous/CacheContinuousQueryHandler.java |  8 +++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  4 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  6 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 20 +++---
 .../tcp/TcpCommunicationSpiMBean.java           | 11 ++++
 .../ignite/spi/discovery/DiscoverySpi.java      |  3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 12 +++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 12 +++-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java | 18 +++++
 ...ridFailFastNodeFailureDetectionSelfTest.java | 17 ++++-
 .../IgniteSlowClientDetectionSelfTest.java      | 69 +++++++++++++++++++-
 .../testframework/GridSpiTestContext.java       |  4 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |  1 +
 17 files changed, 180 insertions(+), 41 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 885d52c..40a5ea5 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
@@ -480,12 +480,12 @@ public abstract class GridManagerAdapter<T extends IgniteSpi> implements GridMan
                         return ctx.io().messageFactory();
                     }
 
-                    @Override public boolean tryFailNode(UUID nodeId) {
-                        return ctx.discovery().tryFailNode(nodeId);
+                    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
+                        return ctx.discovery().tryFailNode(nodeId, warning);
                     }
 
-                    @Override public void failNode(UUID nodeId) {
-                        ctx.discovery().failNode(nodeId);
+                    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+                        ctx.discovery().failNode(nodeId, warning);
                     }
 
                     @Override public void addTimeoutObject(IgniteSpiTimeoutObject obj) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 717cdf3..1e4b972 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
@@ -386,9 +386,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                         verChanged = false;
                 }
                 else {
-                    minorTopVer = 0;
+                    if (type != EVT_NODE_SEGMENTED) {
+                        minorTopVer = 0;
 
-                    verChanged = true;
+                        verChanged = true;
+                    }
+                    else
+                        verChanged = false;
                 }
 
                 AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer);
@@ -1481,15 +1485,16 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning message to be shown on all nodes.
      * @return Whether node is failed.
      */
-    public boolean tryFailNode(UUID nodeId) {
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return false;
 
         try {
             if (!getSpi().pingNode(nodeId)) {
-                getSpi().failNode(nodeId);
+                getSpi().failNode(nodeId, warning);
 
                 return true;
             }
@@ -1503,13 +1508,14 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /**
      * @param nodeId Node ID to fail.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public void failNode(UUID nodeId) {
+    public void failNode(UUID nodeId, @Nullable String warning) {
         if (!busyLock.enterBusy())
             return;
 
         try {
-            getSpi().failNode(nodeId);
+            getSpi().failNode(nodeId, warning);
         }
         finally {
             busyLock.leaveBusy();
@@ -1520,6 +1526,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * Updates topology version if current version is smaller than updated.
      *
      * @param updated Updated topology version.
+     * @param discoCache Discovery cache.
      * @return {@code True} if topology was updated.
      */
     private boolean updateTopologyVersionIfGreater(AffinityTopologyVersion updated, DiscoCache discoCache) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 ad78b92..ff2905f 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
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.managers.deployment.*;
@@ -226,6 +227,13 @@ class CacheContinuousQueryHandler<K, V> implements GridContinuousHandler {
 
                             ctx.continuous().addNotification(nodeId, routineId, evt.entry(), topic, sync, true);
                         }
+                        catch (ClusterTopologyCheckedException ex) {
+                            IgniteLogger log = ctx.log(getClass());
+
+                            if (log.isDebugEnabled())
+                                log.debug("Failed to send event notification to node, node left cluster " +
+                                    "[node=" + nodeId + ", err=" + ex + ']');
+                        }
                         catch (IgniteCheckedException ex) {
                             U.error(ctx.log(getClass()), "Failed to send event notification to node: " + nodeId, ex);
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 18191a1..5e557bd 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
@@ -754,12 +754,12 @@ public abstract class IgniteSpiAdapter implements IgniteSpi, IgniteSpiManagement
         }
 
         /** {@inheritDoc} */
-        @Override public boolean tryFailNode(UUID nodeId) {
+        @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
             return false;
         }
 
         /** {@inheritDoc} */
-        @Override public void failNode(UUID nodeId) {
+        @Override public void failNode(UUID nodeId, @Nullable String warning) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 a655a73..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
@@ -307,14 +307,16 @@ public interface IgniteSpiContext {
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      * @return If node was failed.
      */
-    public boolean tryFailNode(UUID nodeId);
+    public boolean tryFailNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param c Timeout object.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 538e9a8..84c1a57 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
@@ -1153,15 +1153,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return msgQueueLimit;
     }
 
-    /**
-     * Gets slow client queue limit.
-     * <p/>
-     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
-     * those clients whose queue exceeded this limit.
-     *
-     * @return Slow client queue limit.
-     */
-    public int getSlowClientQueueLimit() {
+    /** {@inheritDoc} */
+    @Override public int getSlowClientQueueLimit() {
         return slowClientQueueLimit;
     }
 
@@ -1923,10 +1916,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    LT.warn(log, null, "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\'): " + node);
+                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
+                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                        ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
+
+                    LT.warn(log, null, msg);
 
-                    getSpiContext().failNode(id);
+                    getSpiContext().failNode(id, msg);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index fe4f581..1971d99 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -268,4 +268,15 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Maximum number of unacknowledged messages.")
     public int getUnacknowledgedMessagesBufferSize();
+
+    /**
+     * Gets slow client queue limit.
+     * <p/>
+     * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
+     * those clients whose queue exceeded this limit.
+     *
+     * @return Slow client queue limit.
+     */
+    @MXBeanDescription("Slow client queue limit.")
+    public int getSlowClientQueueLimit();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index b952087..11a18b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -151,8 +151,9 @@ public interface DiscoverySpi extends IgniteSpi {
      * Initiates failure of provided node.
      *
      * @param nodeId Node ID.
+     * @param warning Warning to be shown on all cluster nodes.
      */
-    public void failNode(UUID nodeId);
+    public void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * Whether or not discovery is started in client mode.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 fef6f4f..e255e08 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
@@ -335,13 +335,15 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = rmtNodes.get(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -1432,6 +1434,14 @@ class ClientImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
+                if (msg.warning() != null) {
+                    ClusterNode creatorNode = rmtNodes.get(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 8eb82ac..2458f85 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
@@ -584,13 +584,15 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         ClusterNode node = ring.node(nodeId);
 
         if (node != null) {
             TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
                 node.id(), node.order());
 
+            msg.warning(warning);
+
             msgWorker.addMessage(msg);
         }
     }
@@ -3409,6 +3411,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                         worker.interrupt();
                 }
 
+                if (msg.warning() != null && !msg.creatorNodeId().equals(getLocalNodeId())) {
+                    ClusterNode creatorNode = ring.node(msg.creatorNodeId());
+
+                    U.warn(log, "Received EVT_NODE_FAILED event with warning [" +
+                        "nodeInitiatedEvt=" + (creatorNode != null ? creatorNode : msg.creatorNodeId()) +
+                        ", msg=" + msg.warning() + ']');
+                }
+
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);
 
                 spi.stats.onNodeFailed();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 94097c9..ace917f 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
@@ -175,8 +175,9 @@ abstract class TcpDiscoveryImpl {
 
     /**
      * @param nodeId Node id.
+     * @param warning Warning message to be shown on all nodes.
      */
-    public abstract void failNode(UUID nodeId);
+    public abstract void failNode(UUID nodeId, @Nullable String warning);
 
     /**
      * @param gridName Grid name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 c36ac76..1d1916a 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
@@ -373,8 +373,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        impl.failNode(nodeId);
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
+        impl.failNode(nodeId, warning);
     }
 
     /** {@inheritDoc} */
@@ -385,7 +385,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** {@inheritDoc} */
     @Override public boolean isClientMode() {
         if (impl == null)
-            throw new IllegalStateException("TcpDiscoverySpi has not started");
+            throw new IllegalStateException("TcpDiscoverySpi has not started.");
 
         return impl instanceof ClientImpl;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
index 8cb8414..93ecdaa 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/messages/TcpDiscoveryNodeFailedMessage.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.spi.discovery.tcp.messages;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
 
 import java.util.*;
 
@@ -37,6 +38,9 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     /** Internal order of the failed node. */
     private final long order;
 
+    /** */
+    private String warning;
+
     /**
      * Constructor.
      *
@@ -55,6 +59,20 @@ public class TcpDiscoveryNodeFailedMessage extends TcpDiscoveryAbstractMessage {
     }
 
     /**
+     * @param warning Warning message to be shown on all nodes.
+     */
+    public void warning(String warning) {
+        this.warning = warning;
+    }
+
+    /**
+     * @return Warning message to be shown on all nodes.
+     */
+    @Nullable public String warning() {
+        return warning;
+    }
+
+    /**
      * Gets ID of the failed node.
      *
      * @return ID of the failed node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
index 992d7bf..238115d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridFailFastNodeFailureDetectionSelfTest.java
@@ -50,7 +50,12 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(IP_FINDER);
-        disco.setHeartbeatFrequency(10000);
+        disco.setHeartbeatFrequency(10_000);
+
+        // Set parameters for fast ping failure.
+        disco.setSocketTimeout(100);
+        disco.setNetworkTimeout(100);
+        disco.setReconnectCount(2);
 
         cfg.setDiscoverySpi(disco);
 
@@ -66,8 +71,6 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testFailFast() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-933");
-
         startGridsMultiThreaded(5);
 
         final CountDownLatch failLatch = new CountDownLatch(4);
@@ -87,6 +90,8 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
         Ignite ignite1 = ignite(0);
         Ignite ignite2 = ignite(1);
 
+        final CountDownLatch evtLatch = new CountDownLatch(1);
+
         ignite1.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
                 respond(rcvMsg);
@@ -95,12 +100,18 @@ public class GridFailFastNodeFailureDetectionSelfTest extends GridCommonAbstract
 
         ignite2.message().localListen(null, new MessagingListenActor<Object>() {
             @Override protected void receive(UUID nodeId, Object rcvMsg) throws Throwable {
+                evtLatch.countDown();
+
                 respond(rcvMsg);
             }
         });
 
         ignite1.message(ignite1.cluster().forRemotes()).send(null, "Message");
 
+        evtLatch.await(); // Wait when connection is established.
+
+        log.info("Fail node: " + ignite1.cluster().localNode());
+
         failNode(ignite1);
 
         assert failLatch.await(1000, MILLISECONDS);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/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 09b4215..27c2a61 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
@@ -19,24 +19,37 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.events.*;
+import org.apache.ignite.events.EventType;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.tcp.*;
+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 javax.cache.event.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
 
 /**
  *
  */
 public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
-
+    /** */
     public static final String PARTITIONED = "partitioned";
 
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /**
      * @return Node count.
      */
@@ -48,6 +61,8 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
         if (getTestGridName(nodeCount() - 1).equals(gridName) || getTestGridName(nodeCount() - 2).equals(gridName))
             cfg.setClientMode(true);
 
@@ -66,7 +81,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
-        startGridsMultiThreaded(nodeCount());
+        startGrids(nodeCount());
     }
 
     /** {@inheritDoc} */
@@ -82,6 +97,45 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
     public void testSlowClient() throws Exception {
         final IgniteEx slowClient = grid(nodeCount() - 1);
 
+        final ClusterNode slowClientNode = slowClient.localNode();
+
+        final CountDownLatch evtSegmentedLatch = new CountDownLatch(1);
+
+        slowClient.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_SEGMENTED);
+
+                DiscoveryEvent evt0 = (DiscoveryEvent)evt;
+
+                assertEquals(slowClientNode, evt0.eventNode());
+                assertEquals(5L, evt0.topologyVersion());
+
+                evtSegmentedLatch.countDown();
+
+                return false;
+            }
+        }, EventType.EVT_NODE_SEGMENTED);
+
+        final CountDownLatch evtFailedLatch = new CountDownLatch(nodeCount() - 1);
+
+        for (int i = 0; i < nodeCount() - 1; i++) {
+            grid(i).events().localListen(new IgnitePredicate<Event>() {
+                @Override public boolean apply(Event evt) {
+                    assertEquals("Unexpected event: " + evt, evt.type(), EventType.EVT_NODE_FAILED);
+
+                    DiscoveryEvent evt0 = (DiscoveryEvent) evt;
+
+                    assertEquals(slowClientNode, evt0.eventNode());
+                    assertEquals(6L, evt0.topologyVersion());
+                    assertEquals(4, evt0.topologyNodes().size());
+
+                    evtFailedLatch.countDown();
+
+                    return false;
+                }
+            }, EventType.EVT_NODE_FAILED);
+        }
+
         assertTrue(slowClient.cluster().localNode().isClient());
 
         IgniteCache<Object, Object> cache = slowClient.getOrCreateCache(PARTITIONED);
@@ -109,14 +163,23 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < 100; i++)
             cache0.put(0, new byte[10 * 1024]);
 
-        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
                 return Ignition.state(slowClient.name()) == IgniteState.STOPPED_ON_SEGMENTATION;
             }
         }, getTestTimeout());
+
+        assertTrue(wait);
+
+        assertTrue("Failed to wait for client failed event", evtFailedLatch.await(5000, MILLISECONDS));
+        assertTrue("Failed to wait for client segmented event", evtSegmentedLatch.await(5000, MILLISECONDS));
     }
 
+    /**
+     *
+     */
     private static class Listener implements CacheEntryUpdatedListener<Object, Object> {
+        /** {@inheritDoc} */
         @Override public void onUpdated(Iterable iterable) throws CacheEntryListenerException {
             System.out.println(">>>> Received update: " + iterable);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
index c20ff2e..08268af 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridSpiTestContext.java
@@ -497,12 +497,12 @@ public class GridSpiTestContext implements IgniteSpiContext {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean tryFailNode(UUID nodeId) {
+    @Override public boolean tryFailNode(UUID nodeId, @Nullable String warning) {
         return false;
     }
 
     /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
+    @Override public void failNode(UUID nodeId, @Nullable String warning) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/49dc4a50/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index b4977ce..2d14728 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -91,6 +91,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridMessageListenSelfTest.class);
         suite.addTestSuite(GridFailFastNodeFailureDetectionSelfTest.class);
         suite.addTestSuite(OffHeapTieredTransactionSelfTest.class);
+        suite.addTestSuite(IgniteSlowClientDetectionSelfTest.class);
 
         return suite;
     }


[35/40] incubator-ignite git commit: Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-sprint-7


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 1c660781355f870271760dd6dc0392741fdb15f6
Parents: 5d8e404 285d790
Author: AKuznetsov <ak...@gridgain.com>
Authored: Tue Jun 23 10:00:32 2015 +0700
Committer: AKuznetsov <ak...@gridgain.com>
Committed: Tue Jun 23 10:00:32 2015 +0700

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        | 49 +++++++++++++++-----
 .../communication/tcp/TcpCommunicationSpi.java  | 12 +++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |  1 -
 3 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------



[26/40] incubator-ignite git commit: #IGNITE-GG-10449 marshal credentials in client discovery.

Posted by sb...@apache.org.
#IGNITE-GG-10449 marshal credentials in client discovery.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: a3d8e603a5c9a7fbf3f81a7dd2aded2a8139f578
Parents: 1cc0000
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 11:33:53 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 11:33:53 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 23 ++++++++++++++++++++
 1 file changed, 23 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a3d8e603/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 e255e08..68e5dbd 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
@@ -362,6 +362,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         long startTime = U.currentTimeMillis();
 
+        // Marshal credentials for backward compatibility and security.
+        marshalCredentials(locNode);
+
         while (true) {
             if (Thread.currentThread().isInterrupted())
                 throw new InterruptedException();
@@ -541,6 +544,26 @@ class ClientImpl extends TcpDiscoveryImpl {
         return null;
     }
 
+    /**
+     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+     *
+     * @param node Node to marshall credentials for.
+     * @throws IgniteSpiException If marshalling failed.
+     */
+    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            // Use security-unsafe getter.
+            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
+                spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+
+            node.setAttributes(attrs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+        }
+    }
 
     /**
      * @param topVer New topology version.


[31/40] incubator-ignite git commit: #IGNITE Added message on init stop.

Posted by sb...@apache.org.
#IGNITE Added message on init stop.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 16059962e303f7580c6524b1e0f93c2dcce9eaa4
Parents: de0930d
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Mon Jun 22 15:30:48 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Mon Jun 22 15:30:48 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/MarshallerContextImpl.java  |  12 +-
 .../cache/GridCacheDaemonNodeStopSelfTest.java  | 119 +++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite3.java       |   1 +
 3 files changed, 130 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
index 48f24ac..9f7c983 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/MarshallerContextImpl.java
@@ -124,10 +124,18 @@ public class MarshallerContextImpl extends MarshallerContextAdapter {
 
     /** {@inheritDoc} */
     @Override protected String className(int id) throws IgniteCheckedException {
-        if (cache == null)
+        GridCacheAdapter<Integer, String> cache0 = cache;
+
+        if (cache0 == null) {
             U.awaitQuiet(latch);
 
-        String clsName = cache.get(id);
+            cache0 = cache;
+
+            if (cache0 == null)
+                throw new IllegalStateException("Failed to initialize marshaller context (grid is stopping).");
+        }
+
+        String clsName = cache0.get(id);
 
         if (clsName == null) {
             File file = new File(workDir, id + ".classname");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
new file mode 100644
index 0000000..c56ad1c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheDaemonNodeStopSelfTest.java
@@ -0,0 +1,119 @@
+/*
+ * 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.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 static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ * Test cache operations with daemon node.
+ */
+public class GridCacheDaemonNodeStopSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** Daemon flag. */
+    protected boolean daemon;
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        daemon = false;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        c.setDaemon(daemon);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        c.setDiscoverySpi(disco);
+
+        c.setConnectorConfiguration(null);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setCacheMode(LOCAL);
+        cc.setAtomicityMode(ATOMIC);
+
+        TransactionConfiguration cfg = new TransactionConfiguration();
+
+        c.setTransactionConfiguration(cfg);
+
+        c.setCacheConfiguration(cc);
+
+        return c;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testStartStop() throws Exception {
+        try {
+            daemon = true;
+
+            final IgniteEx daemonNode = startGrid(0);
+
+            final IgniteInternalFuture<Object> f = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    daemonNode.context().marshallerContext().getClass(1,
+                        GridCacheDaemonNodeStopSelfTest.class.getClassLoader());
+
+                    return null;
+                }
+            });
+
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    f.get(300);
+
+                    return null;
+                }
+            }, IgniteFutureTimeoutCheckedException.class);
+
+            // Stop grid.
+            stopGrid(0);
+
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    f.get(5, TimeUnit.SECONDS);
+
+                    return null;
+                }
+            }, IllegalStateException.class);
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16059962/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
index 5947d33..b83f50f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite3.java
@@ -100,6 +100,7 @@ public class IgniteCacheTestSuite3 extends TestSuite {
         suite.addTestSuite(GridCacheDaemonNodeLocalSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodePartitionedSelfTest.class);
         suite.addTestSuite(GridCacheDaemonNodeReplicatedSelfTest.class);
+        suite.addTestSuite(GridCacheDaemonNodeStopSelfTest.class);
 
         // Write-behind.
         suite.addTest(IgniteCacheWriteBehindTestSuite.suite());


[21/40] incubator-ignite git commit: # ignite-sprint-6 do not call ping from exchange worker (cherry picked from commit 8fa9d3d)

Posted by sb...@apache.org.
# ignite-sprint-6 do not call ping from exchange worker (cherry picked from commit 8fa9d3d)


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 309eeb0127546c2775f8514e1759497fccf1c8b4
Parents: 415264e
Author: sboikov <sb...@gridgain.com>
Authored: Fri Jun 19 15:37:37 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Jun 22 09:24:54 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 64 ++++++++++++------
 .../GridCachePartitionExchangeManager.java      | 70 +++++++++-----------
 .../GridCacheAbstractFailoverSelfTest.java      |  6 +-
 3 files changed, 79 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/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 eef9fde..74a4512 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
@@ -323,7 +323,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * Processes failed messages.
      * @param nodeId niode id.
      * @param msg message.
-     * @throws IgniteCheckedException
+     * @throws IgniteCheckedException If failed.
      */
     private void processFailedMessage(UUID nodeId, GridCacheMessage msg) throws IgniteCheckedException {
         GridCacheContext ctx = cctx.cacheContext(msg.cacheId());
@@ -511,6 +511,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
      */
+    @SuppressWarnings("unchecked")
     private IgniteInternalFuture<Object> startFuture(GridCacheMessage cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
@@ -574,6 +575,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      * @throws ClusterTopologyCheckedException If receiver left.
      */
@@ -734,6 +736,7 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
      *
      * @param nodeId ID of node to send the message to.
      * @param msg Message to send.
+     * @param plc IO policy.
      * @throws IgniteCheckedException If sending failed.
      */
     public void send(UUID nodeId, GridCacheMessage msg, GridIoPolicy plc) throws IgniteCheckedException {
@@ -795,8 +798,41 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Sends message without retries and node ping in case of error.
+     *
+     * @param node Node to send message to.
+     * @param msg Message.
+     * @param plc IO policy.
+     * @throws IgniteCheckedException If send failed.
+     */
+    public void sendNoRetry(ClusterNode node,
+        GridCacheMessage msg,
+        GridIoPolicy plc)
+        throws IgniteCheckedException
+    {
+        assert node != null;
+        assert msg != null;
+
+        onSend(msg, null);
+
+        try {
+            cctx.gridIO().send(node, TOPIC_CACHE, msg, plc);
+
+            if (log.isDebugEnabled())
+                log.debug("Sent cache message [msg=" + msg + ", node=" + U.toShortString(node) + ']');
+        }
+        catch (IgniteCheckedException e) {
+            if (!cctx.discovery().alive(node.id()))
+                throw new ClusterTopologyCheckedException("Node left grid while sending message to: " + node.id(), e);
+            else
+                throw e;
+        }
+    }
+
+    /**
      * Adds message handler.
      *
+     * @param cacheId Cache ID.
      * @param type Type of message.
      * @param c Handler.
      */
@@ -846,29 +882,15 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         idxClsHandlers.remove(cacheId);
 
-        for (Iterator<ListenerKey> iterator = clsHandlers.keySet().iterator(); iterator.hasNext(); ) {
-            ListenerKey key = iterator.next();
+        for (Iterator<ListenerKey> iter = clsHandlers.keySet().iterator(); iter.hasNext(); ) {
+            ListenerKey key = iter.next();
 
             if (key.cacheId == cacheId)
-                iterator.remove();
+                iter.remove();
         }
     }
 
     /**
-     * @param lsnr Listener to add.
-     */
-    public void addDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().addDisconnectListener(lsnr);
-    }
-
-    /**
-     * @param lsnr Listener to remove.
-     */
-    public void removeDisconnectListener(GridDisconnectListener lsnr) {
-        cctx.kernalContext().io().removeDisconnectListener(lsnr);
-    }
-
-    /**
      * @param msgCls Message class to check.
      * @return Message index.
      */
@@ -1029,11 +1051,11 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
         /** {@inheritDoc} */
         @Override public int hashCode() {
-            int result = cacheId;
+            int res = cacheId;
 
-            result = 31 * result + msgCls.hashCode();
+            res = 31 * res + msgCls.hashCode();
 
-            return result;
+            return res;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/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 ad4cf50..edd0ad7 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
@@ -567,27 +567,21 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
 
         Collection<ClusterNode> rmts = null;
 
-        try {
-            // If this is the oldest node.
-            if (oldest.id().equals(cctx.localNodeId())) {
-                rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
-
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
+        // If this is the oldest node.
+        if (oldest.id().equals(cctx.localNodeId())) {
+            rmts = CU.remoteNodes(cctx, AffinityTopologyVersion.NONE);
 
-                sendAllPartitions(rmts);
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Refreshing local partitions from non-oldest node: " +
-                        cctx.localNodeId());
+            if (log.isDebugEnabled())
+                log.debug("Refreshing partitions from oldest node: " + cctx.localNodeId());
 
-                sendLocalPartitions(oldest, null);
-            }
+            sendAllPartitions(rmts);
         }
-        catch (IgniteCheckedException e) {
-            U.error(log, "Failed to refresh partition map [oldest=" + oldest.id() + ", rmts=" + U.nodeIds(rmts) +
-                ", loc=" + cctx.localNodeId() + ']', e);
+        else {
+            if (log.isDebugEnabled())
+                log.debug("Refreshing local partitions from non-oldest node: " +
+                    cctx.localNodeId());
+
+            sendLocalPartitions(oldest, null);
         }
     }
 
@@ -616,10 +610,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param nodes Nodes.
      * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes)
-        throws IgniteCheckedException {
+    private boolean sendAllPartitions(Collection<? extends ClusterNode> nodes) {
         GridDhtPartitionsFullMessage m = new GridDhtPartitionsFullMessage(null, null, AffinityTopologyVersion.NONE);
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
@@ -634,7 +626,19 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         if (log.isDebugEnabled())
             log.debug("Sending all partitions [nodeIds=" + U.nodeIds(nodes) + ", msg=" + m + ']');
 
-        cctx.io().safeSend(nodes, m, SYSTEM_POOL, null);
+        for (ClusterNode node : nodes) {
+            try {
+                cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
+            }
+            catch (ClusterTopologyCheckedException ignore) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
+                        node.id() + ", msg=" + m + ']');
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to send partitions full message [node=" + node + ']', e);
+            }
+        }
 
         return true;
     }
@@ -642,11 +646,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /**
      * @param node Node.
      * @param id ID.
-     * @return {@code True} if message was sent, {@code false} if node left grid.
-     * @throws IgniteCheckedException If failed.
      */
-    private boolean sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id)
-        throws IgniteCheckedException {
+    private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
             cctx.kernalContext().clientNode(),
             cctx.versions().last());
@@ -669,16 +670,15 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             log.debug("Sending local partitions [nodeId=" + node.id() + ", msg=" + m + ']');
 
         try {
-            cctx.io().send(node, m, SYSTEM_POOL);
-
-            return true;
+            cctx.io().sendNoRetry(node, m, SYSTEM_POOL);
         }
         catch (ClusterTopologyCheckedException ignore) {
             if (log.isDebugEnabled())
                 log.debug("Failed to send partition update to node because it left grid (will ignore) [node=" +
                     node.id() + ", msg=" + m + ']');
-
-            return false;
+        }
+        catch (IgniteCheckedException e) {
+            U.error(log, "Failed to send local partition map to node [node=" + node + ", exchId=" + id + ']', e);
         }
     }
 
@@ -903,13 +903,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
             return;
 
         try {
-            try {
-                sendLocalPartitions(node, msg.exchangeId());
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to send local partition map to node [nodeId=" + node.id() + ", exchId=" +
-                    msg.exchangeId() + ']', e);
-            }
+            sendLocalPartitions(node, msg.exchangeId());
         }
         finally {
             leaveBusy();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/309eeb01/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index b6cd88e..3c74674 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -74,9 +74,11 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         TcpDiscoverySpi discoSpi = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
-        discoSpi.setSocketTimeout(10_000);
-        discoSpi.setAckTimeout(10_000);
+        discoSpi.setSocketTimeout(30_000);
+        discoSpi.setAckTimeout(30_000);
         discoSpi.setNetworkTimeout(60_000);
+        discoSpi.setHeartbeatFrequency(30_000);
+        discoSpi.setReconnectCount(2);
 
         return cfg;
     }


[13/40] incubator-ignite git commit: Merge branch 'ignite-sprint-6' into ignite-sprint-7

Posted by sb...@apache.org.
Merge branch 'ignite-sprint-6' into ignite-sprint-7

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/extdata/p2p/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/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/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/ec9cfcaf
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/ec9cfcaf
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/ec9cfcaf

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: ec9cfcaf8b245be6fe4723b1e2d08526d3a6d4cb
Parents: 8fa9d3d d699faa
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jun 19 15:32:21 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jun 19 15:32:21 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java | 44 ++++++++++++++++++--
 1 file changed, 41 insertions(+), 3 deletions(-)
----------------------------------------------------------------------



[17/40] incubator-ignite git commit: Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6

Posted by sb...@apache.org.
Merge remote-tracking branch 'origin/ignite-sprint-6' into ignite-sprint-6


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: e0e1994dd51d81b0d7d47000db1678f0268c0bfd
Parents: 20ac4b8 e1b8800
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Fri Jun 19 21:28:12 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Fri Jun 19 21:28:12 2015 +0300

----------------------------------------------------------------------
 .../internal/managers/GridManagerAdapter.java   |   8 +-
 .../discovery/GridDiscoveryManager.java         |  30 ++-
 .../continuous/CacheContinuousQueryHandler.java |   8 +
 .../ignite/internal/util/nio/GridNioServer.java |  64 ++++++-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   7 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   9 +-
 .../communication/tcp/TcpCommunicationSpi.java  |  69 +++++++
 .../tcp/TcpCommunicationSpiMBean.java           |  11 ++
 .../ignite/spi/discovery/DiscoverySpi.java      |   3 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  12 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  17 +-
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |   3 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   6 +-
 .../messages/TcpDiscoveryNodeFailedMessage.java |  18 ++
 ...ridFailFastNodeFailureDetectionSelfTest.java |  17 +-
 .../IgniteSlowClientDetectionSelfTest.java      | 187 +++++++++++++++++++
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |  44 ++++-
 .../testframework/GridSpiTestContext.java       |   7 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |   1 +
 19 files changed, 495 insertions(+), 26 deletions(-)
----------------------------------------------------------------------



[15/40] incubator-ignite git commit: #IGNITE-1034 - Corrected messages after code review.

Posted by sb...@apache.org.
#IGNITE-1034 - Corrected messages after code review.


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: e1b880030aae7ef9cfa6a812d17a97cf8055dfd5
Parents: 7a8b572
Author: Alexey Goncharuk <ag...@gridgain.com>
Authored: Fri Jun 19 10:57:04 2015 -0700
Committer: Alexey Goncharuk <ag...@gridgain.com>
Committed: Fri Jun 19 10:57:04 2015 -0700

----------------------------------------------------------------------
 .../ignite/spi/communication/tcp/TcpCommunicationSpi.java     | 7 +++++--
 1 file changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e1b88003/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 a540b7c..3997768 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
@@ -1158,6 +1158,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * <p/>
      * When set to a positive number, communication SPI will monitor clients outbound queue sizes and will drop
      * those clients whose queue exceeded this limit.
+     * <p/>
+     * Usually this value should be set to the same value as {@link #getMessageQueueLimit()} which controls
+     * message back-pressure for server nodes. The default value for this parameter is {@link #DFLT_MSG_QUEUE_LIMIT}.
      *
      * @param slowClientQueueLimit Slow cilent queue limit.
      */
@@ -1913,8 +1916,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ClusterNode node = getSpiContext().node(id);
 
                 if (node != null && node.isClient()) {
-                    String msg = "Client node outbound queue size exceed configured slow client queue limit, " +
-                        "will fail the node (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
+                    String msg = "Client node outbound queue size exceeded slowClientQueueLimit, " +
+                        "the client will be dropped (consider changing \'slowClientQueueLimit\') [clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
                     LT.warn(log, null, msg);


[10/40] incubator-ignite git commit: ignite-973 - swap manager fix + offheap processor signatures fix

Posted by sb...@apache.org.
ignite-973 - swap manager fix + offheap processor signatures fix


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

Branch: refs/heads/IgniteReflectionFactory-doc
Commit: 23512df815c196f9a2292ad948b130380f123770
Parents: b23ea74
Author: S.Vladykin <sv...@gridgain.com>
Authored: Fri Jun 19 13:31:27 2015 +0300
Committer: S.Vladykin <sv...@gridgain.com>
Committed: Fri Jun 19 13:31:27 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheSwapManager.java  |  12 +-
 .../offheap/GridOffHeapProcessor.java           |  19 +-
 .../apache/ignite/internal/util/GridDebug.java  |  37 ++--
 .../cache/IgniteCacheOffheapEvictQueryTest.java | 179 +++++++++++++++++++
 .../IgniteCacheQuerySelfTestSuite.java          |   1 +
 5 files changed, 220 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
index d0d9049..f709e03 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSwapManager.java
@@ -977,15 +977,15 @@ public class GridCacheSwapManager extends GridCacheManagerAdapter {
 
         // First try offheap.
         if (offheapEnabled) {
-            byte[] val = offheap.remove(spaceName, part, key.value(cctx.cacheObjectContext(), false),
-                key.valueBytes(cctx.cacheObjectContext()));
-
-            if(val != null && cctx.config().isStatisticsEnabled())
-                cctx.cache().metrics0().onOffHeapRemove();
+            // TODO Pass closure c to offheap.remove and apply it before the actual remove.
+            byte[] val = offheap.remove(spaceName, part, key, key.valueBytes(cctx.cacheObjectContext()));
 
             if (val != null) {
+                if (cctx.config().isStatisticsEnabled())
+                    cctx.cache().metrics0().onOffHeapRemove();
+
                 if (c != null)
-                    c.apply(val); // Probably we should read value and apply closure before removing...
+                    c.apply(val);
 
                 return;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
index a99c4c0..81bf9f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/offheap/GridOffHeapProcessor.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.offheap;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.*;
@@ -101,7 +102,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Key bytes
      * @throws IgniteCheckedException If failed.
      */
-    private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
+    private byte[] keyBytes(KeyCacheObject key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
         assert key != null;
 
         return keyBytes != null ? keyBytes : marsh.marshal(key);
@@ -130,7 +131,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If offheap space contains value for the given key.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean contains(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public boolean contains(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -147,7 +148,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] get(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    @Nullable public byte[] get(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -166,7 +167,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Tuple where first value is pointer and second is value size.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, Object key,
+    @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable String spaceName, int part, KeyCacheObject key,
         byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -182,7 +183,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param keyBytes Key bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public void enableEviction(@Nullable String spaceName, int part, Object key, byte[] keyBytes)
+    public void enableEviction(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -201,7 +202,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public <T> T getValue(@Nullable String spaceName, int part, Object key, byte[] keyBytes,
+    @Nullable public <T> T getValue(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes,
         @Nullable ClassLoader ldr) throws IgniteCheckedException {
         byte[] valBytes = get(spaceName, part, key, keyBytes);
 
@@ -221,7 +222,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] remove(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    @Nullable public byte[] remove(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, keyBytes));
@@ -237,7 +238,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @param valBytes Value bytes.
      * @throws IgniteCheckedException If failed.
      */
-    public void put(@Nullable String spaceName, int part, Object key, byte[] keyBytes, byte[] valBytes)
+    public void put(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes, byte[] valBytes)
         throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
@@ -258,7 +259,7 @@ public class GridOffHeapProcessor extends GridProcessorAdapter {
      * @return {@code true} If succeeded.
      * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, Object key, byte[] keyBytes) throws IgniteCheckedException {
+    public boolean removex(@Nullable String spaceName, int part, KeyCacheObject key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.removex(part, U.hash(key), keyBytes(key, keyBytes));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
index aadec74..98c8664 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridDebug.java
@@ -19,6 +19,7 @@ package 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.jetbrains.annotations.*;
 
 import java.io.*;
@@ -173,13 +174,6 @@ public class GridDebug {
     }
 
     /**
-     * Dump collected data to stdout.
-     */
-    public static void dump() {
-        dump(que.get());
-    }
-
-    /**
      * Dumps given number of last events.
      *
      * @param n Number of last elements to dump.
@@ -204,7 +198,7 @@ public class GridDebug {
      * @param que Queue.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public static void dump(ConcurrentLinkedQueue<Item> que) {
+    public static void dump(Collection<Item> que) {
         if (que == null)
             return;
 
@@ -226,7 +220,7 @@ public class GridDebug {
      */
     public static String dumpWithStop(Object... x) {
         debug(x);
-        return dumpWithReset(null);
+        return dumpWithReset(null, null);
     }
 
     /**
@@ -235,16 +229,20 @@ public class GridDebug {
      * @return Empty string (useful for assertions like {@code assert x == 0 : D.dumpWithReset();} ).
      */
     public static String dumpWithReset() {
-        return dumpWithReset(new ConcurrentLinkedQueue<Item>());
+        return dumpWithReset(new ConcurrentLinkedQueue<Item>(), null);
     }
 
     /**
      * Dump existing queue to stdout and atomically replace it with given.
      *
      * @param q2 Queue.
+     * @param filter Filter for logged debug items.
      * @return Empty string.
      */
-    private static String dumpWithReset(@Nullable ConcurrentLinkedQueue<Item> q2) {
+    public static String dumpWithReset(
+        @Nullable ConcurrentLinkedQueue<Item> q2,
+        @Nullable IgnitePredicate<Item> filter
+    ) {
         ConcurrentLinkedQueue<Item> q;
 
         do {
@@ -255,7 +253,20 @@ public class GridDebug {
         }
         while (!que.compareAndSet(q, q2));
 
-        dump(q);
+        Collection<Item> col = null;
+
+        if (filter == null)
+            col = q;
+        else if (q != null) {
+            col = new ArrayList<>();
+
+            for (Item item : q) {
+                if (filter.apply(item))
+                    col.add(item);
+            }
+        }
+
+        dump(col);
 
         return "";
     }
@@ -281,7 +292,7 @@ public class GridDebug {
      */
     private static String formatEntry(long ts, String threadName, long threadId, Object... data) {
         return "<" + DEBUG_DATE_FMT.format(new Date(ts)) + "><~DBG~><" + threadName + " id:" + threadId + "> " +
-            Arrays.toString(data);
+            Arrays.deepToString(data);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
new file mode 100644
index 0000000..fc6c125
--- /dev/null
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -0,0 +1,179 @@
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.query.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+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.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.swapspace.inmemory.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import javax.cache.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+
+/**
+ */
+public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(ipFinder);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
+
+        CacheConfiguration<?,?> cacheCfg = defaultCacheConfiguration();
+
+        cacheCfg.setCacheMode(PARTITIONED);
+        cacheCfg.setAtomicityMode(TRANSACTIONAL);
+        cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC);
+        cacheCfg.setSwapEnabled(true);
+        cacheCfg.setBackups(0);
+        cacheCfg.setMemoryMode(CacheMemoryMode.ONHEAP_TIERED);
+        cacheCfg.setEvictionPolicy(null);
+        cacheCfg.setNearConfiguration(null);
+
+        cacheCfg.setSqlOnheapRowCacheSize(128);
+
+        cacheCfg.setIndexedTypes(
+            Integer.class, Integer.class
+        );
+
+        cacheCfg.setOffHeapMaxMemory(2000); // Small offheap for evictions from offheap to swap.
+
+        cfg.setCacheConfiguration(cacheCfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testEvictAndRemove() throws Exception {
+        final int KEYS_CNT = 3000;
+        final int THREADS_CNT = 256;
+
+        final IgniteCache<Integer,Integer> c = startGrid().cache(null);
+
+        for (int i = 0; i < KEYS_CNT; i++) {
+            c.put(i, i);
+
+            if ((i & 1) == 0)
+                c.localEvict(F.asList(i));
+        }
+
+        X.println("___ Cache loaded...");
+
+        final CyclicBarrier b = new CyclicBarrier(THREADS_CNT, new Runnable() {
+            @Override public void run() {
+                X.println("___ go!");
+            }
+        });
+
+        final AtomicInteger keys = new AtomicInteger(KEYS_CNT);
+
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
+            @Override public void run() {
+                Random rnd = new GridRandom();
+
+                try {
+                    b.await();
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteInterruptedException(e);
+                }
+                catch (BrokenBarrierException e) {
+                    throw new IllegalStateException(e);
+                }
+
+                while (keys.get() > 0) {
+                    int k = rnd.nextInt(KEYS_CNT);
+
+                    try {
+                        switch (rnd.nextInt(4)) {
+                            case 0:
+                                c.localEvict(F.asList(k));
+
+                                break;
+
+                            case 1:
+                                c.get(k);
+
+                                break;
+
+                            case 2:
+                                if (c.remove(k))
+                                    keys.decrementAndGet();
+
+                                break;
+
+                            case 3:
+                                c.query(new SqlFieldsQuery("select _val from Integer where _key between ? and ?")
+                                    .setArgs(k, k + 20).setLocal(true)).getAll();
+
+                                break;
+                        }
+                    }
+                    catch (CacheException e) {
+                        String msgStart = "Failed to get value for key:";
+
+                        for (Throwable th = e; th != null; th = th.getCause()) {
+                            String msg = th.getMessage();
+
+                            if (msg != null && msg.startsWith(msgStart)) {
+                                int dot = msg.indexOf('.', msgStart.length());
+
+                                assertTrue(dot != -1);
+
+                                final Integer failedKey = Integer.parseInt(msg.substring(msgStart.length(), dot).trim());
+
+                                X.println("___ failed key: " + failedKey);
+
+                                break;
+                            }
+                        }
+
+                        LT.warn(log, null, e.getMessage());
+
+                        return;
+                    }
+                }
+            }
+        }, THREADS_CNT);
+
+        try {
+            fut.get(60_000);
+
+            if (c.size(CachePeekMode.ALL) != 0)
+                fail("Not all keys removed.");
+
+            X.println("___ all keys removed");
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            X.println("___ timeout");
+            X.println("___ keys: " + keys.get());
+
+            keys.set(0);
+
+            fut.get();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/23512df8/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
index dee3078..b9205a9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteCacheQuerySelfTestSuite.java
@@ -63,6 +63,7 @@ public class IgniteCacheQuerySelfTestSuite extends TestSuite {
         suite.addTestSuite(IgniteCacheQueryEvictsMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryOffheapMultiThreadedSelfTest.class);
         // suite.addTestSuite(IgniteCacheQueryOffheapEvictsMultiThreadedSelfTest.class); TODO IGNITE-971.
+        suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class);
         suite.addTestSuite(IgniteCacheSqlQueryMultiThreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheOffheapTieredMultithreadedSelfTest.class);
         suite.addTestSuite(IgniteCacheQueryNodeRestartSelfTest.class);