You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by an...@apache.org on 2015/10/01 11:23:02 UTC

[01/20] ignite git commit: Disable shmem in failover tests to avoid ignite-1578.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1168 3a322c055 -> 6e0f41593


Disable shmem in failover tests to avoid ignite-1578.


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

Branch: refs/heads/ignite-1168
Commit: bfabab2d0ee22808eef8a597b4b82dd84e5a2c87
Parents: 7402918
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 30 09:19:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 30 09:26:21 2015 +0300

----------------------------------------------------------------------
 .../GridCacheAbstractFailoverSelfTest.java      |  3 ++
 .../GridCacheAbstractRemoveFailureTest.java     |  3 ++
 .../cache/IgniteCacheCreateRestartSelfTest.java |  3 ++
 ...acheAsyncOperationsFailoverAbstractTest.java | 11 ++++++++
 .../CachePutAllFailoverAbstractTest.java        | 11 ++++++++
 .../IgniteCachePutGetRestartAbstractTest.java   |  3 ++
 .../IgniteCacheSizeFailoverTest.java            |  3 ++
 ...gniteAtomicLongChangingTopologySelfTest.java | 29 +++++++-------------
 .../IgniteCacheCrossCacheTxFailoverTest.java    |  3 ++
 9 files changed, 50 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/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 e55803a..f4813ff 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
@@ -33,6 +33,7 @@ import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.TransactionConcurrency;
@@ -86,6 +87,8 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
         discoSpi.setHeartbeatFrequency(30_000);
         discoSpi.setReconnectCount(2);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index a3d9948..122910e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -41,6 +41,7 @@ import org.apache.ignite.internal.util.lang.GridTuple;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -92,6 +93,8 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCommonAbstr
 
         cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
index 0a2d778..6f32d36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheCreateRestartSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -47,6 +48,8 @@ public class IgniteCacheCreateRestartSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         cfg.setPeerClassLoadingEnabled(false);
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
index ecdfaac..25fb8de 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheAsyncOperationsFailoverAbstractTest.java
@@ -29,12 +29,14 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 
@@ -51,6 +53,15 @@ public abstract class CacheAsyncOperationsFailoverAbstractTest extends GridCache
     private static final long TEST_TIME = 60_000;
 
     /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return NODE_CNT;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
index f558ba0..86707f6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CachePutAllFailoverAbstractTest.java
@@ -28,12 +28,14 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.GridCacheAbstractSelfTest;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.jetbrains.annotations.NotNull;
 
@@ -50,6 +52,15 @@ public abstract class CachePutAllFailoverAbstractTest extends GridCacheAbstractS
     private static final long TEST_TIME = 60_000;
 
     /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
     @Override protected int gridCount() {
         return NODE_CNT;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePutGetRestartAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePutGetRestartAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePutGetRestartAbstractTest.java
index 1d92750..1d4370a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePutGetRestartAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePutGetRestartAbstractTest.java
@@ -32,6 +32,7 @@ import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractTest;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 
@@ -81,6 +82,8 @@ public abstract class IgniteCachePutGetRestartAbstractTest extends IgniteCacheAb
 
         cfg.setPeerClassLoadingEnabled(false);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
index 2b31f50..00b7c0f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSizeFailoverTest.java
@@ -25,6 +25,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -48,6 +49,8 @@ public class IgniteCacheSizeFailoverTest extends GridCommonAbstractTest {
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setCacheMode(PARTITIONED);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteAtomicLongChangingTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteAtomicLongChangingTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteAtomicLongChangingTopologySelfTest.java
index 1f2e035..337334e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteAtomicLongChangingTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteAtomicLongChangingTopologySelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -36,32 +37,22 @@ import java.util.concurrent.atomic.*;
  *
  */
 public class IgniteAtomicLongChangingTopologySelfTest extends GridCommonAbstractTest {
-    /**
-     * Grid count.
-     */
+    /** Grid count. */
     private static final int GRID_CNT = 5;
 
-    /**
-     * Restart count.
-     */
+    /** Restart cound. */
     private static final int RESTART_CNT = 15;
 
-    /**
-     * Atomic long name.
-     */
+    /** Atomic long name. */
     private static final String ATOMIC_LONG_NAME = "test-atomic-long";
 
-    /**
-     * Queue.
-     */
+    /** Queue. */
     private final Queue<Long> queue = new ConcurrentLinkedQueue<>();
 
     /** */
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
-    /**
-     * {@inheritDoc}
-     */
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
@@ -77,12 +68,12 @@ public class IgniteAtomicLongChangingTopologySelfTest extends GridCommonAbstract
 
         cfg.setAtomicConfiguration(atomicCfg);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         return cfg;
     }
 
-    /**
-     * {@inheritDoc}
-     */
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         stopAllGrids();
 
@@ -90,7 +81,7 @@ public class IgniteAtomicLongChangingTopologySelfTest extends GridCommonAbstract
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testQueueCreateNodesJoin() throws Exception {
         CountDownLatch startLatch = new CountDownLatch(GRID_CNT);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bfabab2d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
index 7425e23..8500e97 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCrossCacheTxFailoverTest.java
@@ -37,6 +37,7 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
@@ -91,6 +92,8 @@ public class IgniteCacheCrossCacheTxFailoverTest extends GridCommonAbstractTest
 
         cfg.setSwapSpaceSpi(new FileSwapSpaceSpi());
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         return cfg;
     }
 


[05/20] ignite git commit: ignite-1329 Removed outdated code from GridCacheIoManager

Posted by an...@apache.org.
ignite-1329 Removed outdated code from GridCacheIoManager


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

Branch: refs/heads/ignite-1168
Commit: cd439672847d764838075b7eafae504ce27d0563
Parents: c6ec7e1
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 30 13:43:41 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 30 13:43:41 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheIoManager.java    | 76 +-------------------
 .../processors/cache/GridCacheMessage.java      |  7 --
 .../cache/GridCacheSharedContext.java           | 32 ---------
 .../processors/cache/GridCacheUtils.java        |  8 ---
 .../dht/GridDhtAffinityAssignmentRequest.java   |  5 --
 .../dht/GridDhtAffinityAssignmentResponse.java  |  5 --
 .../distributed/dht/GridDhtLockRequest.java     |  5 --
 .../distributed/dht/GridDhtTxFinishRequest.java |  5 --
 .../dht/GridDhtTxPrepareRequest.java            |  7 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |  5 --
 .../dht/preloader/GridDhtForceKeysResponse.java |  5 --
 .../GridDhtPartitionDemandMessage.java          |  5 --
 .../GridDhtPartitionSupplyMessage.java          |  5 --
 .../GridDhtPartitionsAbstractMessage.java       |  5 --
 14 files changed, 4 insertions(+), 171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/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 421ec82..c81dae5 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
@@ -270,62 +270,8 @@ public class GridCacheIoManager extends GridCacheSharedManagerAdapter {
 
             if (cacheMsg.classError() != null)
                 processFailedMessage(nodeId, cacheMsg);
-            else {
-                if (cacheMsg.allowForStartup())
-                    processMessage(nodeId, cacheMsg, c);
-                else {
-                    IgniteInternalFuture<?> startFut = startFuture(cacheMsg);
-
-                    if (startFut.isDone())
-                        processMessage(nodeId, cacheMsg, c);
-                    else {
-                        if (log.isDebugEnabled())
-                            log.debug("Waiting for start future to complete for message [nodeId=" + nodeId +
-                                ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
-
-                        // Don't hold this thread waiting for preloading to complete.
-                        startFut.listen(new CI1<IgniteInternalFuture<?>>() {
-                            @Override public void apply(final IgniteInternalFuture<?> f) {
-                                cctx.kernalContext().closure().runLocalSafe(
-                                    new GridPlainRunnable() {
-                                        @Override public void run() {
-                                            rw.readLock();
-
-                                            try {
-                                                if (stopping) {
-                                                    if (log.isDebugEnabled())
-                                                        log.debug("Received cache communication message while stopping " +
-                                                            "(will ignore) [nodeId=" + nodeId + ", msg=" + cacheMsg + ']');
-
-                                                    return;
-                                                }
-
-                                                f.get();
-
-                                                if (log.isDebugEnabled())
-                                                    log.debug("Start future completed for message [nodeId=" + nodeId +
-                                                        ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
-
-                                                processMessage(nodeId, cacheMsg, c);
-                                            }
-                                            catch (IgniteCheckedException e) {
-                                                // Log once.
-                                                if (startErr.compareAndSet(false, true))
-                                                    U.error(log, "Failed to complete preload start future " +
-                                                        "(will ignore message) " +
-                                                        "[fut=" + f + ", nodeId=" + nodeId + ", msg=" + cacheMsg + ']', e);
-                                            }
-                                            finally {
-                                                rw.readUnlock();
-                                            }
-                                        }
-                                    }
-                                );
-                            }
-                        });
-                    }
-                }
-            }
+            else
+                processMessage(nodeId, cacheMsg, c);
         }
         catch (Throwable e) {
             U.error(log, "Failed to process message [senderId=" + nodeId + ", messageType=" + cacheMsg.getClass() + ']', e);
@@ -549,24 +495,6 @@ 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();
-
-        if (cacheId != 0)
-            return cctx.cacheContext(cacheId).preloader().startFuture();
-        else {
-            if (F.eq(cacheMsg.topologyVersion(), AffinityTopologyVersion.NONE))
-                return new GridFinishedFuture<>();
-
-            return cctx.preloadersStartFuture(cacheMsg.topologyVersion());
-        }
-    }
-
-    /**
      * @param nodeId Node ID.
      * @param msg Message.
      * @param c Closure.

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 55688e4..4b700e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -93,13 +93,6 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
-     * @return {@code True} if this message is preloader message.
-     */
-    public boolean allowForStartup() {
-        return false;
-    }
-
-    /**
      * @return {@code True} if this message is partition exchange message.
      */
     public boolean partitionExchangeMessage() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 13e390a..c5d4162 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -333,38 +333,6 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
-     * @return Compound preloaders start future.
-     */
-    public IgniteInternalFuture<Object> preloadersStartFuture(AffinityTopologyVersion topVer) {
-        if (preloadersStartFut == null) {
-            GridCompoundFuture<Object, Object> compound = null;
-
-            for (GridCacheContext<?, ?> cacheCtx : cacheContexts()) {
-                if (cacheCtx.startTopologyVersion() != null && cacheCtx.startTopologyVersion().compareTo(topVer) <= 0) {
-                    IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
-
-                    if (!startFut.isDone()) {
-                        if (compound == null)
-                            compound = new GridCompoundFuture<>();
-
-                        compound.add(startFut);
-                    }
-                }
-            }
-
-            if (compound != null) {
-                compound.markInitialized();
-
-                return preloadersStartFut = compound;
-            }
-            else
-                return preloadersStartFut = new GridFinishedFuture<>();
-        }
-        else
-            return preloadersStartFut;
-    }
-
-    /**
      * @return Transactional metrics adapter.
      */
     public TransactionMetricsAdapter txMetrics() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 2d5698a..7854c93 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -286,14 +286,6 @@ public class GridCacheUtils {
     }
 
     /**
-     * @param msg Message to check.
-     * @return {@code True} if preloader message.
-     */
-    public static boolean allowForStartup(Object msg) {
-        return ((GridCacheMessage)msg).allowForStartup();
-    }
-
-    /**
      * Writes {@link GridCacheVersion} to output stream. This method is meant to be used by
      * implementations of {@link Externalizable} interface.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
index 6381073..e6a5c9a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentRequest.java
@@ -52,11 +52,6 @@ public class GridDhtAffinityAssignmentRequest extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean partitionExchangeMessage() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
index 139100f..2798d7c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAffinityAssignmentResponse.java
@@ -70,11 +70,6 @@ public class GridDhtAffinityAssignmentResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean partitionExchangeMessage() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
index cf4b581..87aa256 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockRequest.java
@@ -171,11 +171,6 @@ public class GridDhtLockRequest extends GridDistributedLockRequest {
         this.accessTtl = accessTtl;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @return Near node ID.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
index f859314..be59a95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishRequest.java
@@ -159,11 +159,6 @@ public class GridDhtTxFinishRequest extends GridDistributedTxFinishRequest {
         this.taskNameHash = taskNameHash;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @return Mini ID.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
index 5f9443b..29d5a70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareRequest.java
@@ -164,11 +164,6 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
         return nearXidVer;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @return Near node ID.
      */
@@ -215,6 +210,8 @@ public class GridDhtTxPrepareRequest extends GridDistributedTxPrepareRequest {
 
     /**
      * Marks last added key for preloading.
+     *
+     * @param idx Key index.
      */
     public void markKeyForPreload(int idx) {
         if (preloadKeys == null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
index 3268e26..fac8b9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysRequest.java
@@ -90,11 +90,6 @@ public class GridDhtForceKeysRequest extends GridCacheMessage implements GridCac
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @param keys Collection of keys.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
index 93e39ce..9418887 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysResponse.java
@@ -103,11 +103,6 @@ public class GridDhtForceKeysResponse extends GridCacheMessage implements GridCa
         return err;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @return Keys.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
index b588372..b211d27 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandMessage.java
@@ -99,11 +99,6 @@ public class GridDhtPartitionDemandMessage extends GridCacheMessage {
         // No-op.
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
     /**
      * @param p Partition.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
index 3ccc5ae..67c08bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyMessage.java
@@ -96,11 +96,6 @@ public class GridDhtPartitionSupplyMessage extends GridCacheMessage implements G
     }
 
     /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean ignoreClassErrors() {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/cd439672/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
index cc7d439..041d5f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsAbstractMessage.java
@@ -56,11 +56,6 @@ abstract class GridDhtPartitionsAbstractMessage extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean allowForStartup() {
-        return true;
-    }
-
-    /** {@inheritDoc} */
     @Override public boolean partitionExchangeMessage() {
         return true;
     }


[19/20] ignite git commit: Test Failed

Posted by an...@apache.org.
Test Failed


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

Branch: refs/heads/ignite-1168
Commit: b08e0b20f46fdfb4bb86b6c495e407d77873e077
Parents: b962a91
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 19:02:05 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 19:02:05 2015 +0300

----------------------------------------------------------------------
 .../partitioned/GridCachePartitionedSetFailoverSelfTest.java     | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b08e0b20/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetFailoverSelfTest.java
index ea1bb9b..755ac91 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedSetFailoverSelfTest.java
@@ -37,4 +37,8 @@ public class GridCachePartitionedSetFailoverSelfTest extends GridCacheSetFailove
     @Override protected CacheMemoryMode collectionMemoryMode() {
         return ONHEAP_TIERED;
     }
+
+    @Override public void testNodeRestart(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1593");
+    }
 }
\ No newline at end of file


[17/20] ignite git commit: Test Failed

Posted by an...@apache.org.
Test Failed


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

Branch: refs/heads/ignite-1168
Commit: aa9d23e80a08d0cfb4c3fc13cb7c59bc2ceb3bee
Parents: cdcaf9a
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 18:52:33 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 18:52:33 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java  | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/aa9d23e8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java
index caeb9b6..cc24d7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest.java
@@ -37,4 +37,8 @@ public class GridCachePartitionedAtomicQueueCreateMultiNodeSelfTest
     @Override protected CacheAtomicityMode collectionCacheAtomicityMode() {
         return ATOMIC;
     }
+
+    @Override public void testTx(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1591");
+    }
 }
\ No newline at end of file


[10/20] ignite git commit: Test fix

Posted by an...@apache.org.
Test fix


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

Branch: refs/heads/ignite-1168
Commit: 7571f211ea85854854f08b9d2cecaff9850e126d
Parents: 1282f8b
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 15:43:40 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 15:43:40 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/dht/IgniteCrossCacheTxSelfTest.java      | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7571f211/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCrossCacheTxSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCrossCacheTxSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCrossCacheTxSelfTest.java
index bf9b1c6..dc0293e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCrossCacheTxSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCrossCacheTxSelfTest.java
@@ -148,11 +148,11 @@ public class IgniteCrossCacheTxSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < TX_CNT; i++) {
             int grid = ThreadLocalRandom8.current().nextInt(nodeCount());
 
+            IgniteCache<Integer, String> first = grid(grid).cache(FIRST_CACHE);
+            IgniteCache<Integer, String> second = grid(grid).cache(SECOND_CACHE);
+
             try (Transaction tx = grid(grid).transactions().txStart(concurrency, isolation)) {
                 try {
-                    IgniteCache<Integer, String> first = grid(grid).cache(FIRST_CACHE);
-                    IgniteCache<Integer, String> second = grid(grid).cache(SECOND_CACHE);
-
                     int size = ThreadLocalRandom8.current().nextInt(24) + 1;
 
                     for (int k = 0; k < size; k++) {


[16/20] ignite git commit: https://issues.apache.org/jira/browse/IGNITE-1584

Posted by an...@apache.org.
https://issues.apache.org/jira/browse/IGNITE-1584


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

Branch: refs/heads/ignite-1168
Commit: cdcaf9aa760581c80732c75311c2e0a57d29ccb4
Parents: d3f04ff
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 17:40:19 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 17:40:19 2015 +0300

----------------------------------------------------------------------
 .../tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java      | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cdcaf9aa/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java
index f5e77e5..bf0cbd5 100644
--- a/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java
+++ b/modules/cloud/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/cloud/TcpDiscoveryCloudIpFinderSelfTest.java
@@ -68,6 +68,8 @@ public class TcpDiscoveryCloudIpFinderSelfTest extends
      * @throws Exception If any error occurs.
      */
     public void testGoogleComputeEngine() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1585");
+
         testCloudProvider("google-compute-engine");
     }
 


[08/20] ignite git commit: # master - changed "ssl" to "tls/ssl"

Posted by an...@apache.org.
# master - changed "ssl" to "tls/ssl"


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

Branch: refs/heads/ignite-1168
Commit: f3808831a8df1b67b81a92f50cd2b23c3358e604
Parents: c6ec7e1
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Wed Sep 30 14:19:13 2015 +0200
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Wed Sep 30 14:19:13 2015 +0200

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/f3808831/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 68a0383..c02dc59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2177,7 +2177,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         assert log != null;
 
         U.quietAndInfo(log, "Security status [authentication=" + onOff(ctx.security().enabled())
-            + ", ssl=" + onOff(ctx.config().getSslContextFactory() != null) + ']');
+            + ", tls/ssl=" + onOff(ctx.config().getSslContextFactory() != null) + ']');
     }
 
     /**


[03/20] ignite git commit: IGNITE-1515: Fixed delete.

Posted by an...@apache.org.
IGNITE-1515: Fixed delete.


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

Branch: refs/heads/ignite-1168
Commit: 54bb7d760d9c1261a64fcf3d0be858be7b14444e
Parents: be906e3
Author: iveselovskiy <iv...@gridgain.com>
Authored: Wed Sep 30 10:05:19 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Sep 30 10:05:19 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/igfs/IgfsPath.java   |   2 +-
 .../internal/processors/igfs/IgfsFileInfo.java  |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      |  57 +----
 .../processors/igfs/IgfsMetaManager.java        | 251 +++++++++++++++----
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   2 +-
 .../processors/igfs/IgfsAbstractSelfTest.java   |  80 ++++--
 .../igfs/IgfsMetaManagerSelfTest.java           |   6 -
 .../processors/igfs/IgfsMetricsSelfTest.java    |   2 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |  29 +--
 .../igfs/UniversalFileSystemAdapter.java        |   1 -
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |  36 +++
 ...oopFileSystemUniversalFileSystemAdapter.java |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   7 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   5 +-
 14 files changed, 345 insertions(+), 139 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
index a99c1ee..fb0621c 100644
--- a/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
+++ b/modules/core/src/main/java/org/apache/ignite/igfs/IgfsPath.java
@@ -50,7 +50,7 @@ public final class IgfsPath implements Comparable<IgfsPath>, Externalizable {
     private static final char SLASH_CHAR = '/';
 
     /** The directory separator. */
-    private static final String SLASH = "/";
+    public static final String SLASH = "/";
 
     /** URI representing this path. Should never change after object creation or de-serialization. */
     private String path;

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
index 116d585..8564500 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFileInfo.java
@@ -216,7 +216,7 @@ public final class IgfsFileInfo implements Externalizable {
      * @param listing New directory listing.
      * @param old Old file info.
      */
-    IgfsFileInfo(Map<String, IgfsListingEntry> listing, IgfsFileInfo old) {
+    IgfsFileInfo(@Nullable Map<String, IgfsListingEntry> listing, IgfsFileInfo old) {
         this(old.isDirectory(), old.id, old.blockSize, old.len, old.affKey, listing, old.props, old.fileMap(),
             old.lockId, false, old.accessTime, old.modificationTime, old.evictExclude());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 0dd0307..d5ba95f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -53,7 +53,6 @@ import org.apache.ignite.events.DiscoveryEvent;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.IgfsEvent;
 import org.apache.ignite.igfs.IgfsBlockLocation;
-import org.apache.ignite.igfs.IgfsDirectoryNotEmptyException;
 import org.apache.ignite.igfs.IgfsFile;
 import org.apache.ignite.igfs.IgfsInvalidPathException;
 import org.apache.ignite.igfs.IgfsMetrics;
@@ -712,6 +711,9 @@ public final class IgfsImpl implements IgfsEx {
                 if (log.isDebugEnabled())
                     log.debug("Deleting file [path=" + path + ", recursive=" + recursive + ']');
 
+                if (IgfsPath.SLASH.equals(path.toString()))
+                    return false;
+
                 IgfsMode mode = resolveMode(path);
 
                 Set<IgfsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
@@ -721,8 +723,11 @@ public final class IgfsImpl implements IgfsEx {
                 FileDescriptor desc = getFileDescriptor(path);
 
                 if (childrenModes.contains(PRIMARY)) {
-                    if (desc != null)
-                        res = delete0(desc, path.parent(), recursive);
+                    if (desc != null) {
+                        IgniteUuid deletedId = meta.softDelete(path, recursive);
+
+                        res = deletedId != null;
+                    }
                     else if (mode == PRIMARY)
                         checkConflictWithPrimary(path);
                 }
@@ -750,48 +755,6 @@ public final class IgfsImpl implements IgfsEx {
         });
     }
 
-    /**
-     * Internal procedure for (optionally) recursive file and directory deletion.
-     *
-     * @param desc File descriptor of file or directory to delete.
-     * @param parentPath Parent path. If specified, events will be fired for each deleted file
-     *      or directory. If not specified, events will not be fired.
-     * @param recursive Recursive deletion flag.
-     * @return {@code True} if file was successfully deleted. If directory is not empty and
-     *      {@code recursive} flag is false, will return {@code false}.
-     * @throws IgniteCheckedException In case of error.
-     */
-    private boolean delete0(FileDescriptor desc, @Nullable IgfsPath parentPath, boolean recursive)
-        throws IgniteCheckedException {
-        IgfsPath curPath = parentPath == null ? new IgfsPath() : new IgfsPath(parentPath, desc.fileName);
-
-        if (desc.isFile) {
-            deleteFile(curPath, desc, true);
-
-            return true;
-        }
-        else {
-            if (recursive) {
-                meta.softDelete(desc.parentId, desc.fileName, desc.fileId);
-
-                return true;
-            }
-            else {
-                Map<String, IgfsListingEntry> infoMap = meta.directoryListing(desc.fileId);
-
-                if (F.isEmpty(infoMap)) {
-                    deleteFile(curPath, desc, true);
-
-                    return true;
-                }
-                else
-                    // Throw exception if not empty and not recursive.
-                    throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not empty " +
-                        "and recursive flag is not set)");
-            }
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public void mkdirs(IgfsPath path) {
         mkdirs(path, null);
@@ -1454,7 +1417,7 @@ public final class IgfsImpl implements IgfsEx {
      */
     IgniteInternalFuture<?> formatAsync() {
         try {
-            IgniteUuid id = meta.softDelete(null, null, ROOT_ID);
+            IgniteUuid id = meta.format();
 
             if (id == null)
                 return new GridFinishedFuture<Object>();
@@ -1526,6 +1489,8 @@ public final class IgfsImpl implements IgfsEx {
      * @throws IgniteCheckedException If failed.
      */
     @Nullable private FileDescriptor getFileDescriptor(IgfsPath path) throws IgniteCheckedException {
+        assert path != null;
+
         List<IgniteUuid> ids = meta.fileIds(path);
 
         IgfsFileInfo fileInfo = meta.info(ids.get(ids.size() - 1));

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
index d283b64..bb6404c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManager.java
@@ -34,6 +34,7 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.SortedSet;
 import java.util.TreeMap;
 import java.util.TreeSet;
 import java.util.concurrent.CountDownLatch;
@@ -95,6 +96,20 @@ import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_REA
  */
 @SuppressWarnings("all")
 public class IgfsMetaManager extends IgfsManager {
+    /** Comparator for Id sorting. */
+    private static final Comparator<IgniteUuid> PATH_ID_SORTING_COMPARATOR
+            = new Comparator<IgniteUuid>() {
+        @Override public int compare(IgniteUuid u1, IgniteUuid u2) {
+            if (u1 == u2)
+                return 0;
+
+            if (u1 == null)
+                return -1;
+
+            return u1.compareTo(u2);
+        }
+    };
+
     /** IGFS configuration. */
     private FileSystemConfiguration cfg;
 
@@ -376,6 +391,7 @@ public class IgfsMetaManager extends IgfsManager {
 
     /**
      * Gets file info by its ID.
+     * NB: this method is used both in Tx and out of Tx.
      *
      * @param fileId File ID to get details for.
      * @return File info.
@@ -593,12 +609,39 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Answers if the collection is sorted.
+     *
+     * @param col The collection to check.
+     * @param <T> The type of the collection elements.
+     * @return If the collection sorted.
+     */
+    private static <T extends Comparable<T>> boolean isSorted(Collection<T> col) {
+        T prev = null;
+
+        for (T t: col) {
+            if (t == null)
+                throw new NullPointerException("Collections should not contain nulls");
+
+            if (prev != null && prev.compareTo(t) > 0)
+                return false; // disordered.
+
+            prev = t;
+        }
+
+        return true;
+    }
+
+    /**
      * Lock file IDs.
+     *
      * @param fileIds File IDs (sorted).
      * @return Map with lock info.
      * @throws IgniteCheckedException If failed.
      */
     private Map<IgniteUuid, IgfsFileInfo> lockIds(Collection<IgniteUuid> fileIds) throws IgniteCheckedException {
+        assert isSorted(fileIds);
+        assert validTxState(true);
+
         if (log.isDebugEnabled())
             log.debug("Locking file ids: " + fileIds);
 
@@ -608,19 +651,34 @@ public class IgfsMetaManager extends IgfsManager {
         if (log.isDebugEnabled())
             log.debug("Locked file ids: " + fileIds);
 
-        // Force root ID always exist in cache.
-        if (fileIds.contains(ROOT_ID) && !map.containsKey(ROOT_ID)) {
-            IgfsFileInfo info = new IgfsFileInfo();
+        // Force root & trash IDs always exist in cache.
+        addInfoIfNeeded(fileIds, map, ROOT_ID);
+        addInfoIfNeeded(fileIds, map, TRASH_ID);
 
-            id2InfoPrj.putIfAbsent(ROOT_ID, info);
+        // Returns detail's map for locked IDs.
+        return map;
+    }
 
-            map = new GridLeanMap<>(map);
+    /**
+     * Adds FileInfo into the cache if it is requested in fileIds and is not present in the map.
+     *
+     * @param fileIds A list that may contain the id.
+     * @param map The map that may not contain the id.
+     * @param id The id to check.
+     * @throws IgniteCheckedException On error.
+     */
+    private void addInfoIfNeeded(Collection<IgniteUuid> fileIds, Map<IgniteUuid, IgfsFileInfo> map, IgniteUuid id) throws IgniteCheckedException {
+        assert validTxState(true);
 
-            map.put(ROOT_ID, info);
-        }
+        if (fileIds.contains(id) && !map.containsKey(id)) {
+            IgfsFileInfo info = new IgfsFileInfo(id);
 
-        // Returns detail's map for locked IDs.
-        return map;
+            assert info.listing() != null;
+
+            id2InfoPrj.putIfAbsent(id, info);
+
+            map.put(id, info);
+        }
     }
 
     /**
@@ -779,13 +837,11 @@ public class IgfsMetaManager extends IgfsManager {
             log.debug("Locking parent id [parentId=" + parentId + ", fileName=" + fileName + ", newFileInfo=" +
                 newFileInfo + ']');
 
-        validTxState(true);
+        assert validTxState(true);
 
         // Lock only parent file ID.
         IgfsFileInfo parentInfo = info(parentId);
 
-        assert validTxState(true);
-
         if (parentInfo == null)
             throw fsException(new IgfsPathNotFoundException("Failed to lock parent directory (not found): " + parentId));
 
@@ -798,8 +854,6 @@ public class IgfsMetaManager extends IgfsManager {
 
         IgfsListingEntry entry = parentListing.get(fileName);
 
-        assert validTxState(true);
-
         if (entry != null)
             return entry.fileId();
 
@@ -832,18 +886,7 @@ public class IgfsMetaManager extends IgfsManager {
                 List<IgniteUuid> srcPathIds = fileIds(srcPath);
                 List<IgniteUuid> dstPathIds = fileIds(dstPath);
 
-                final Set<IgniteUuid> allIds = new TreeSet<>(new Comparator<IgniteUuid>() {
-                    @Override
-                    public int compare(IgniteUuid u1, IgniteUuid u2) {
-                        if (u1 == u2)
-                            return 0;
-
-                        if (u1 == null)
-                            return -1;
-
-                        return u1.compareTo(u2);
-                    }
-                });
+                final Set<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
 
                 allIds.addAll(srcPathIds);
 
@@ -1009,6 +1052,7 @@ public class IgfsMetaManager extends IgfsManager {
     private void moveNonTx(IgniteUuid fileId, @Nullable String srcFileName, IgniteUuid srcParentId, String destFileName,
         IgniteUuid destParentId) throws IgniteCheckedException {
         assert validTxState(true);
+
         assert fileId != null;
         assert srcFileName != null;
         assert srcParentId != null;
@@ -1026,8 +1070,6 @@ public class IgfsMetaManager extends IgfsManager {
         // Lock file ID and parent IDs for this transaction.
         Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(srcParentId, fileId, destParentId);
 
-        validTxState(true);
-
         IgfsFileInfo srcInfo = infoMap.get(srcParentId);
 
         if (srcInfo == null)
@@ -1196,6 +1238,66 @@ public class IgfsMetaManager extends IgfsManager {
     }
 
     /**
+     * Deletes (moves to TRASH) all elements under the root folder.
+     *
+     * @return The new Id if the artificially created folder containing all former root
+     * elements moved to TRASH folder.
+     * @throws IgniteCheckedException On error.
+     */
+    IgniteUuid format() throws IgniteCheckedException {
+        if (busyLock.enterBusy()) {
+            try {
+                assert validTxState(false);
+
+                final IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+
+                try {
+                    // NB: We may lock root because its id is less than any other id:
+                    final IgfsFileInfo rootInfo = lockIds(ROOT_ID, TRASH_ID).get(ROOT_ID);
+
+                    assert rootInfo != null;
+
+                    Map<String, IgfsListingEntry> rootListingMap = rootInfo.listing();
+
+                    assert rootListingMap != null;
+
+                    if (rootListingMap.isEmpty())
+                        return null; // Root is empty, nothing to do.
+
+                    // Construct new info and move locked entries from root to it.
+                    Map<String, IgfsListingEntry> transferListing = new HashMap<>(rootListingMap);
+
+                    IgfsFileInfo newInfo = new IgfsFileInfo(transferListing);
+
+                    id2InfoPrj.put(newInfo.id(), newInfo);
+
+                    // Add new info to trash listing.
+                    id2InfoPrj.invoke(TRASH_ID, new UpdateListing(newInfo.id().toString(),
+                        new IgfsListingEntry(newInfo), false));
+
+                    // Remove listing entries from root.
+                    // Note that root directory properties and other attributes are preserved:
+                    id2InfoPrj.put(ROOT_ID, new IgfsFileInfo(null/*listing*/, rootInfo));
+
+                    tx.commit();
+
+                    delWorker.signal();
+
+                    return newInfo.id();
+                }
+                finally {
+                    tx.close();
+                }
+            }
+            finally {
+                busyLock.leaveBusy();
+            }
+        }
+        else
+            throw new IllegalStateException("Failed to perform format because Grid is stopping.");
+    }
+
+    /**
      * Move path to the trash directory.
      *
      * @param parentId Parent ID.
@@ -1204,26 +1306,86 @@ public class IgfsMetaManager extends IgfsManager {
      * @return ID of an entry located directly under the trash directory.
      * @throws IgniteCheckedException If failed.
      */
-    IgniteUuid softDelete(@Nullable IgniteUuid parentId, @Nullable String pathName, IgniteUuid pathId) throws IgniteCheckedException {
+    IgniteUuid softDelete(final IgfsPath path, final boolean recursive) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 assert validTxState(false);
 
-                IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
+                final SortedSet<IgniteUuid> allIds = new TreeSet<>(PATH_ID_SORTING_COMPARATOR);
+
+                List<IgniteUuid> pathIdList = fileIds(path);
+
+                assert pathIdList.size() > 1;
+
+                final IgniteUuid victimId = pathIdList.get(pathIdList.size() - 1);
+
+                assert !TRASH_ID.equals(victimId) : "TRASH does not have path, it cannot ever be deletion victim.";
+                assert !ROOT_ID.equals(victimId); // root deletion is prevented in earlier stages.
+
+                allIds.addAll(pathIdList);
+
+                if (allIds.remove(null))
+                    return null; // A fragment of the path no longer exists.
+
+                boolean added = allIds.add(TRASH_ID);
+                assert added;
+
+                final IgniteInternalTx tx = metaCache.txStartEx(PESSIMISTIC, REPEATABLE_READ);
 
                 try {
-                    if (parentId == null)
-                        lockIds(pathId, TRASH_ID);
-                    else
-                        lockIds(parentId, pathId, TRASH_ID);
+                    final Map<IgniteUuid, IgfsFileInfo> infoMap = lockIds(allIds);
+
+                    // Directory starure was changed concurrently, so the original path no longer exists:
+                    if (!verifyPathIntegrity(path, pathIdList, infoMap))
+                        return null;
+
+                    final IgfsFileInfo victimInfo = infoMap.get(victimId);
+
+                    if (!recursive && victimInfo.isDirectory() && !victimInfo.listing().isEmpty())
+                        // Throw exception if not empty and not recursive.
+                        throw new IgfsDirectoryNotEmptyException("Failed to remove directory (directory is not " +
+                            "empty and recursive flag is not set).");
+
+                    IgfsFileInfo destInfo = infoMap.get(TRASH_ID);
 
-                    IgniteUuid resId = softDeleteNonTx(parentId, pathName, pathId);
+                    assert destInfo != null;
+
+                    final String srcFileName = path.name();
+
+                    final String destFileName = victimId.toString();
+
+                    assert destInfo.listing().get(destFileName) == null : "Failed to add file name into the " +
+                        "destination directory (file already exists) [destName=" + destFileName + ']';
+
+                    IgfsFileInfo srcParentInfo = infoMap.get(pathIdList.get(pathIdList.size() - 2));
+
+                    assert srcParentInfo != null;
+
+                    IgniteUuid srcParentId = srcParentInfo.id();
+                    assert srcParentId.equals(pathIdList.get(pathIdList.size() - 2));
+
+                    IgfsListingEntry srcEntry = srcParentInfo.listing().get(srcFileName);
+
+                    assert srcEntry != null : "Deletion victim not found in parent listing [path=" + path +
+                        ", name=" + srcFileName + ", listing=" + srcParentInfo.listing() + ']';
+
+                    assert victimId.equals(srcEntry.fileId());
+
+                    id2InfoPrj.invoke(srcParentId, new UpdateListing(srcFileName, srcEntry, true));
+
+                    // Add listing entry into the destination parent listing.
+                    id2InfoPrj.invoke(TRASH_ID, new UpdateListing(destFileName, srcEntry, false));
+
+                    if (victimInfo.isFile())
+                        // Update a file info of the removed file with a file path,
+                        // which will be used by delete worker for event notifications.
+                        id2InfoPrj.invoke(victimId, new UpdatePath(path));
 
                     tx.commit();
 
                     delWorker.signal();
 
-                    return resId;
+                    return victimId;
                 }
                 finally {
                     tx.close();
@@ -1234,8 +1396,8 @@ public class IgfsMetaManager extends IgfsManager {
             }
         }
         else
-            throw new IllegalStateException("Failed to perform soft delete because Grid is stopping [parentId=" +
-                parentId + ", pathName=" + pathName + ", pathId=" + pathId + ']');
+            throw new IllegalStateException("Failed to perform soft delete because Grid is " +
+                "stopping [path=" + path + ']');
     }
 
     /**
@@ -1316,6 +1478,7 @@ public class IgfsMetaManager extends IgfsManager {
 
     /**
      * Remove listing entries of the given parent.
+     * This operation actually deletes directories from TRASH, is used solely by IgfsDeleteWorker.
      *
      * @param parentId Parent ID.
      * @param listing Listing entries.
@@ -1403,6 +1566,7 @@ public class IgfsMetaManager extends IgfsManager {
 
     /**
      * Remove entry from the metadata listing.
+     * Used solely by IgfsDeleteWorker.
      *
      * @param parentId Parent ID.
      * @param name Name.
@@ -2394,17 +2558,18 @@ public class IgfsMetaManager extends IgfsManager {
         if (busyLock.enterBusy()) {
             try {
                 SynchronizationTask<IgfsFileInfo> task = new SynchronizationTask<IgfsFileInfo>() {
-                    @Override public IgfsFileInfo onSuccess(Map<IgfsPath, IgfsFileInfo> infos)
-                        throws Exception {
+                    @Override public IgfsFileInfo onSuccess(Map<IgfsPath, IgfsFileInfo> infos) throws Exception {
                         if (infos.get(path) == null)
                             return null;
 
                         fs.update(path, props);
 
-                        assert path.parent() == null || infos.get(path.parent()) != null;
+                        IgfsFileInfo parentInfo = infos.get(path.parent());
+
+                        assert path.parent() == null || parentInfo != null;
 
-                        return updatePropertiesNonTx(infos.get(path.parent()).id(), infos.get(path).id(), path.name(),
-                            props);
+                        return updatePropertiesNonTx(parentInfo == null ? null : parentInfo.id(),
+                            infos.get(path).id(), path.name(), props);
                     }
 
                     @Override public IgfsFileInfo onFailure(@Nullable Exception err) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
index ebf91e0..fd4ec17 100644
--- a/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/igfs/IgfsFragmentizerSelfTest.java
@@ -237,7 +237,7 @@ public class IgfsFragmentizerSelfTest extends IgfsFragmentizerAbstractSelfTest {
             U.sleep(200);
         }
 
-        igfs.delete(new IgfsPath("/"), true);
+        igfs.format();
 
         igfs.awaitDeletesAsync().get();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
index cfa99ff..7e73859 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsAbstractSelfTest.java
@@ -101,10 +101,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     protected static final long BLOCK_SIZE = 32 * 1024 * 1024;
 
     /** Default repeat count. */
-    protected static final int REPEAT_CNT = 5;
+    protected static final int REPEAT_CNT = 5; // Diagnostic: ~100; Regression: 5
 
     /** Concurrent operations count. */
-    protected static final int OPS_CNT = 16;
+    protected static final int OPS_CNT = 16; // Diagnostic: ~160; Regression: 16
 
     /** Seed. */
     protected static final long SEED = System.currentTimeMillis();
@@ -252,6 +252,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         clear(igfs, igfsSecondary);
+
+        assert igfs.listFiles(new IgfsPath("/")).isEmpty();
     }
 
     /** {@inheritDoc} */
@@ -923,6 +925,23 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Tests that root directory properties persist afetr the #format() operation.
+     *
+     * @throws Exception If failed.
+     */
+    public void testRootPropertiesPersistAfterFormat() throws Exception {
+        igfs.update(new IgfsPath("/"), Collections.singletonMap("foo", "moo"));
+
+        igfs.format();
+
+        IgfsFile file = igfs.info(new IgfsPath("/"));
+
+        Map<String,String> props = file.properties();
+
+        assertEquals("moo", props.get("foo"));
+    }
+
+    /**
      * Test regular file open.
      *
      * @throws Exception If failed.
@@ -1600,6 +1619,8 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testConcurrentMkdirsDelete() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
+
         for (int i = 0; i < REPEAT_CNT; i++) {
             final CyclicBarrier barrier = new CyclicBarrier(2);
 
@@ -1881,18 +1902,10 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     public void testDeadlocksRename() throws Exception {
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
-                info(">>>>>> Start deadlock test.");
-
                 checkDeadlocks(5, 2, 2, 2, OPS_CNT, 0, 0, 0, 0);
-
-                info(">>>>>> End deadlock test.");
             }
             finally {
-                info(">>>>>> Start cleanup.");
-
                 clear(igfs, igfsSecondary);
-
-                info(">>>>>> End cleanup.");
             }
         }
     }
@@ -1903,8 +1916,6 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocksDelete() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1515");
-
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
                 checkDeadlocks(5, 2, 2, 2, 0, OPS_CNT, 0, 0, 0);
@@ -1948,6 +1959,42 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
     }
 
     /**
+     * Ensure that deadlocks do not occur during concurrent delete & rename operations.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksDeleteRename() throws Exception {
+        for (int i = 0; i < REPEAT_CNT; i++) {
+            try {
+                checkDeadlocks(5, 2, 2, 2,
+                    OPS_CNT, OPS_CNT, 0, 0, 0);
+            }
+            finally {
+                clear(igfs, igfsSecondary);
+            }
+        }
+    }
+
+    /**
+     * Ensure that deadlocks do not occur during concurrent delete & rename operations.
+     *
+     * @throws Exception If failed.
+     */
+    public void testDeadlocksDeleteMkdirs() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
+
+        for (int i = 0; i < REPEAT_CNT; i++) {
+            try {
+                checkDeadlocks(5, 2, 2, 2,
+                     0, OPS_CNT, 0, OPS_CNT, 0);
+            }
+            finally {
+                clear(igfs, igfsSecondary);
+            }
+        }
+    }
+
+    /**
      * Ensure that deadlocks do not occur during concurrent file creation operations.
      *
      * @throws Exception If failed.
@@ -1969,11 +2016,16 @@ public abstract class IgfsAbstractSelfTest extends IgfsCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDeadlocks() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1515");
+        fail("https://issues.apache.org/jira/browse/IGNITE-1541");
 
         for (int i = 0; i < REPEAT_CNT; i++) {
             try {
-                checkDeadlocks(5, 2, 2, 2, OPS_CNT, OPS_CNT, OPS_CNT, OPS_CNT, OPS_CNT);
+                checkDeadlocks(5, 2, 2, 2,
+                    OPS_CNT, // rename
+                    OPS_CNT, // delete
+                    OPS_CNT, // update
+                    OPS_CNT, // mkdirs
+                    OPS_CNT); // create
             }
             finally {
                 clear(igfs, igfsSecondary);

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
index 206c9fe..4072636 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetaManagerSelfTest.java
@@ -300,25 +300,19 @@ public class IgfsMetaManagerSelfTest extends IgfsCommonAbstractTest {
         System.out.println("b: " + mgr.directoryListing(b.id()));
         System.out.println("f3: " + mgr.directoryListing(f3.id()));
 
-        //mgr.move(a.id(), "a", ROOT_ID, "a2", ROOT_ID);
         mgr.move(path("/a"), path("/a2"));
-        //mgr.move(b.id(), "b", a.id(), "b2", a.id());
         mgr.move(path("/a2/b"), path("/a2/b2"));
 
         assertNotNull(mgr.info(b.id()));
 
-        //mgr.move(f3.id(), "f3", b.id(), "f3-2", a.id());
         mgr.move(path("/a2/b2/f3"), path("/a2/b2/f3-2"));
 
         assertNotNull(mgr.info(b.id()));
 
-        //mgr.move(f3.id(), "f3-2", a.id(), "f3", b.id());
         mgr.move(path("/a2/b2/f3-2"), path("/a2/b2/f3"));
 
-        //mgr.move(b.id(), "b2", a.id(), "b", a.id());
         mgr.move(path("/a2/b2"), path("/a2/b"));
 
-        //mgr.move(a.id(), "a2", ROOT_ID, "a", ROOT_ID);
         mgr.move(path("/a2"), path("/a"));
 
         // Validate 'remove' operation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
index 8a2e5bf..fb1d6f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsMetricsSelfTest.java
@@ -348,7 +348,7 @@ public class IgfsMetricsSelfTest extends IgfsCommonAbstractTest {
         assertEquals(0, m.filesOpenedForRead());
         assertEquals(0, m.filesOpenedForWrite());
 
-        fs.delete(new IgfsPath("/"), true);
+        fs.format();
 
         m = fs.metrics();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
index cb134f8..9c4d832 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsProcessorSelfTest.java
@@ -389,15 +389,6 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
         assert paths.size() == 3 : "Unexpected paths: " + paths;
 
-        // Delete.
-        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                igfs.delete(path("/"), false);
-
-                return null;
-            }
-        }, IgfsException.class, null);
-
         igfs.delete(path("/A1/B1/C1"), false);
         assertNull(igfs.info(path("/A1/B1/C1")));
 
@@ -416,19 +407,19 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
 
         assertEquals(Arrays.asList(path("/A"), path("/A1"), path("/A2")), sorted(igfs.listPaths(path("/"))));
 
-        GridTestUtils.assertThrowsInherited(log, new Callable<Object>() {
-            @Override public Object call() throws Exception {
-                igfs.delete(path("/"), false);
-
-                return null;
-            }
-        }, IgfsException.class, null);
-        assertEquals(Arrays.asList(path("/A"), path("/A1"), path("/A2")), sorted(igfs.listPaths(path("/"))));
-
+        // Delete root when it is not empty:
         igfs.delete(path("/"), true);
+        igfs.delete(path("/"), false);
+
+        igfs.delete(path("/A"), true);
+        igfs.delete(path("/A1"), true);
+        igfs.delete(path("/A2"), true);
         assertEquals(Collections.<IgfsPath>emptyList(), igfs.listPaths(path("/")));
 
+        // Delete root when it is empty:
         igfs.delete(path("/"), false);
+        igfs.delete(path("/"), true);
+
         assertEquals(Collections.<IgfsPath>emptyList(), igfs.listPaths(path("/")));
 
         for (Cache.Entry<Object, Object> e : metaCache)
@@ -608,7 +599,7 @@ public class IgfsProcessorSelfTest extends IgfsCommonAbstractTest {
         assertEquals(text, read("/b"));
 
         // Cleanup.
-        igfs.delete(root, true);
+        igfs.format();
 
         assertEquals(Collections.<IgfsPath>emptyList(), igfs.listPaths(root));
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
index 3bf70e2..ba8c164 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/UniversalFileSystemAdapter.java
@@ -28,7 +28,6 @@ import java.util.Map;
  * To be used solely in tests.
  */
 public interface UniversalFileSystemAdapter {
-
     /**
      * Gets name of the FS.
      * @return name of this file system.

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java
index 3913cbd..fa5cbc5 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/igfs/HadoopIgfsUtils.java
@@ -20,7 +20,11 @@ package org.apache.ignite.internal.processors.hadoop.igfs;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.AbstractFileSystem;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.ParentNotDirectoryException;
+import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathExistsException;
 import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException;
 import org.apache.ignite.IgniteCheckedException;
@@ -130,6 +134,38 @@ public class HadoopIgfsUtils {
     }
 
     /**
+     * Deletes all files from the given file system.
+     *
+     * @param fs The file system to clean up.
+     * @throws IOException On error.
+     */
+    public static void clear(FileSystem fs) throws IOException {
+        // Delete root contents:
+        FileStatus[] statuses = fs.listStatus(new Path("/"));
+
+        if (statuses != null) {
+            for (FileStatus stat: statuses)
+                fs.delete(stat.getPath(), true);
+        }
+    }
+
+    /**
+     * Deletes all files from the given file system.
+     *
+     * @param fs The file system to clean up.
+     * @throws IOException On error.
+     */
+    public static void clear(AbstractFileSystem fs) throws IOException {
+        // Delete root contents:
+        FileStatus[] statuses = fs.listStatus(new Path("/"));
+
+        if (statuses != null) {
+            for (FileStatus stat: statuses)
+                fs.delete(stat.getPath(), true);
+        }
+    }
+
+    /**
      * Constructor.
      */
     private HadoopIgfsUtils() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
index 03f0066..608bd25 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopFileSystemUniversalFileSystemAdapter.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsEx;
 import org.apache.ignite.internal.processors.igfs.UniversalFileSystemAdapter;
 
@@ -33,7 +34,6 @@ import org.apache.ignite.internal.processors.igfs.UniversalFileSystemAdapter;
  * Universal adapter wrapping {@link org.apache.hadoop.fs.FileSystem} instance.
  */
 public class HadoopFileSystemUniversalFileSystemAdapter implements UniversalFileSystemAdapter {
-
     /** The wrapped filesystem. */
     private final FileSystem fileSys;
 
@@ -70,7 +70,7 @@ public class HadoopFileSystemUniversalFileSystemAdapter implements UniversalFile
 
     /** {@inheritDoc} */
     @Override public void format() throws IOException {
-        fileSys.delete(new Path("/"), true);
+        HadoopIgfsUtils.clear(fileSys);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
index 1235786..c938571 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/HadoopIgfs20FileSystemAbstractSelfTest.java
@@ -59,6 +59,7 @@ import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.FileSystemConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.hadoop.fs.IgniteHadoopIgfsSecondaryFileSystem;
+import org.apache.ignite.internal.processors.hadoop.igfs.HadoopIgfsUtils;
 import org.apache.ignite.internal.processors.igfs.IgfsCommonAbstractTest;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.typedef.F;
@@ -352,7 +353,7 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         try {
-            fs.delete(new Path("/"), true);
+            HadoopIgfsUtils.clear(fs);
         }
         catch (Exception ignore) {
             // No-op.
@@ -627,7 +628,9 @@ public abstract class HadoopIgfs20FileSystemAbstractSelfTest extends IgfsCommonA
 
         Path root = new Path(fsHome, "/");
 
-        assertTrue(fs.delete(root, true));
+        assertFalse(fs.delete(root, true));
+
+        assertTrue(fs.delete(new Path(fsHome, "/someDir1"), true));
 
         assertPathDoesNotExist(fs, someDir3);
         assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1/someDir2"));

http://git-wip-us.apache.org/repos/asf/ignite/blob/54bb7d76/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
index 5ea841e..2626ebb 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgniteHadoopFileSystemAbstractSelfTest.java
@@ -300,7 +300,7 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         try {
-            fs.delete(new Path("/"), true);
+            HadoopIgfsUtils.clear(fs);
         }
         catch (Exception ignore) {
             // No-op.
@@ -783,7 +783,8 @@ public abstract class IgniteHadoopFileSystemAbstractSelfTest extends IgfsCommonA
 
         Path root = new Path(fsHome, "/");
 
-        assertTrue(fs.delete(root, true));
+        assertFalse(fs.delete(root, true));
+        assertTrue(fs.delete(new Path("/someDir1"), true));
 
         assertPathDoesNotExist(fs, someDir3);
         assertPathDoesNotExist(fs, new Path(fsHome, "/someDir1/someDir2"));


[14/20] ignite git commit: https://issues.apache.org/jira/browse/IGNITE-1584

Posted by an...@apache.org.
https://issues.apache.org/jira/browse/IGNITE-1584


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

Branch: refs/heads/ignite-1168
Commit: 3de0e475ce7d889fd565f9be2c18a72cafc86430
Parents: bada5c7
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 17:25:46 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 17:25:46 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheAtomicPutAllFailoverSelfTest.java           | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3de0e475/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicPutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicPutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicPutAllFailoverSelfTest.java
index 1feafe4..b3464b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicPutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicPutAllFailoverSelfTest.java
@@ -29,4 +29,8 @@ public class IgniteCacheAtomicPutAllFailoverSelfTest extends GridCachePutAllFail
     @Override protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }
+
+    @Override public void testPutAllFailoverColocatedNearEnabledTwoBackupsOffheapTieredSwap(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1584");
+    }
 }
\ No newline at end of file


[18/20] ignite git commit: Test Failed

Posted by an...@apache.org.
Test Failed


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

Branch: refs/heads/ignite-1168
Commit: b962a91582f6d6459fad24a9529bbe692eba20ca
Parents: aa9d23e
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 18:58:33 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 18:58:33 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedOffHeapValuesQueueApiSelfTest.java       | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b962a915/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java
index 78e0214..d46f684 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedOffHeapValuesQueueApiSelfTest.java
@@ -28,4 +28,8 @@ public class GridCachePartitionedOffHeapValuesQueueApiSelfTest extends GridCache
     @Override protected CacheMemoryMode collectionMemoryMode() {
         return OFFHEAP_VALUES;
     }
+
+    @Override public void testQueueRemoveMultithreadBounded(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1592");
+    }
 }
\ No newline at end of file


[20/20] ignite git commit: Merge branches 'ignite-1168' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1168

Posted by an...@apache.org.
Merge branches 'ignite-1168' and 'master' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-1168


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

Branch: refs/heads/ignite-1168
Commit: 6e0f4159358993323bbf5c90fb8d24aa012cc211
Parents: 3a322c0 b08e0b2
Author: Andrey <an...@gridgain.com>
Authored: Thu Oct 1 16:22:40 2015 +0700
Committer: Andrey <an...@gridgain.com>
Committed: Thu Oct 1 16:22:40 2015 +0700

----------------------------------------------------------------------
 .../TcpDiscoveryCloudIpFinderSelfTest.java      |   2 +
 .../java/org/apache/ignite/igfs/IgfsPath.java   |   2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  70 +++---
 .../discovery/GridDiscoveryManager.java         |  22 +-
 .../processors/cache/GridCacheEventManager.java |  12 +-
 .../processors/cache/GridCacheIoManager.java    |  76 +-----
 .../processors/cache/GridCacheMessage.java      |   7 -
 .../processors/cache/GridCacheProcessor.java    |  68 +++--
 .../cache/GridCacheSharedContext.java           |  32 ---
 .../processors/cache/GridCacheUtils.java        |   8 -
 .../dht/GridDhtAffinityAssignmentRequest.java   |   5 -
 .../dht/GridDhtAffinityAssignmentResponse.java  |   5 -
 .../distributed/dht/GridDhtLockRequest.java     |   5 -
 .../distributed/dht/GridDhtTxFinishRequest.java |   5 -
 .../dht/GridDhtTxPrepareRequest.java            |   7 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |   5 -
 .../dht/preloader/GridDhtForceKeysResponse.java |   5 -
 .../GridDhtPartitionDemandMessage.java          |   5 -
 .../GridDhtPartitionSupplyMessage.java          |   5 -
 .../GridDhtPartitionsAbstractMessage.java       |   5 -
 .../dht/preloader/GridDhtPreloader.java         |   6 +
 .../internal/processors/igfs/IgfsFileInfo.java  |   2 +-
 .../internal/processors/igfs/IgfsImpl.java      |  57 +----
 .../processors/igfs/IgfsMetaManager.java        | 251 +++++++++++++++----
 .../util/nio/GridNioRecoveryDescriptor.java     |   4 +-
 .../ignite/igfs/IgfsFragmentizerSelfTest.java   |   2 +-
 .../IgniteClientReconnectAbstractTest.java      |  35 ++-
 .../IgniteClientReconnectCacheTest.java         | 154 ++++++++++++
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +
 .../GridCacheAbstractFailoverSelfTest.java      |   3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   3 +
 ...IgniteCacheAtomicPutAllFailoverSelfTest.java |   4 +
 .../cache/IgniteCacheCreateRestartSelfTest.java |   3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   7 +
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  12 +-
 ...ionedAtomicQueueCreateMultiNodeSelfTest.java |   4 +
 ...artitionedOffHeapValuesQueueApiSelfTest.java |   4 +
 ...GridCachePartitionedSetFailoverSelfTest.java |   4 +
 ...acheAsyncOperationsFailoverAbstractTest.java |  11 +
 .../CachePutAllFailoverAbstractTest.java        |  11 +
 .../IgniteCacheAtomicNodeRestartTest.java       |   8 +
 .../IgniteCachePutGetRestartAbstractTest.java   |   3 +
 .../IgniteCacheSizeFailoverTest.java            |   3 +
 ...gniteAtomicLongChangingTopologySelfTest.java |  29 +--
 .../IgniteCacheCrossCacheTxFailoverTest.java    |   3 +
 .../dht/IgniteCrossCacheTxSelfTest.java         |   6 +-
 ...ledFairAffinityMultiNodeFullApiSelfTest.java |   4 +
 .../processors/igfs/IgfsAbstractSelfTest.java   |  80 ++++--
 .../igfs/IgfsMetaManagerSelfTest.java           |   6 -
 .../processors/igfs/IgfsMetricsSelfTest.java    |   2 +-
 .../processors/igfs/IgfsProcessorSelfTest.java  |  29 +--
 .../igfs/UniversalFileSystemAdapter.java        |   1 -
 .../processors/hadoop/igfs/HadoopIgfsUtils.java |  36 +++
 ...oopFileSystemUniversalFileSystemAdapter.java |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |   7 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |   5 +-
 56 files changed, 733 insertions(+), 423 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6e0f4159/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/6e0f4159/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------


[12/20] ignite git commit: https://issues.apache.org/jira/browse/IGNITE-1583

Posted by an...@apache.org.
https://issues.apache.org/jira/browse/IGNITE-1583


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

Branch: refs/heads/ignite-1168
Commit: 67d027e518ea4bb64b7a2bc51f1327fe204a2b50
Parents: 52600f3
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 17:19:49 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 17:19:49 2015 +0300

----------------------------------------------------------------------
 .../managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/67d027e5/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index e35333b..50bcc41 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -176,6 +176,8 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
      * @throws Exception If failed.
      */
     public void testAlivesClient() throws Exception {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1583");
+
         clientMode = true;
 
         doTestAlive();


[13/20] ignite git commit: https://issues.apache.org/jira/browse/IGNITE-1582

Posted by an...@apache.org.
https://issues.apache.org/jira/browse/IGNITE-1582


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

Branch: refs/heads/ignite-1168
Commit: bada5c78c16c1ee7b48fa04198e4266b72392311
Parents: 67d027e
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 17:23:12 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 17:23:12 2015 +0300

----------------------------------------------------------------------
 ...cheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java | 4 ++++
 1 file changed, 4 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bada5c78/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
index 484de6e..8e47134 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest.java
@@ -33,4 +33,8 @@ public class GridCacheAtomicNearEnabledFairAffinityMultiNodeFullApiSelfTest
 
         return cfg;
     }
+
+    @Override public void testWithSkipStore(){
+        fail("https://issues.apache.org/jira/browse/IGNITE-1582");
+    }
 }
\ No newline at end of file


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

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


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

Branch: refs/heads/ignite-1168
Commit: de39fbf269ad6f834ab01444558ea2dced1c2035
Parents: f380883 1282f8b
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Wed Sep 30 14:19:28 2015 +0200
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Wed Sep 30 14:19:28 2015 +0200

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  22 +--
 .../processors/cache/GridCacheEventManager.java |  12 +-
 .../processors/cache/GridCacheIoManager.java    |  76 +--------
 .../processors/cache/GridCacheMessage.java      |   7 -
 .../processors/cache/GridCacheProcessor.java    |  68 +++++---
 .../cache/GridCacheSharedContext.java           |  32 ----
 .../processors/cache/GridCacheUtils.java        |   8 -
 .../dht/GridDhtAffinityAssignmentRequest.java   |   5 -
 .../dht/GridDhtAffinityAssignmentResponse.java  |   5 -
 .../distributed/dht/GridDhtLockRequest.java     |   5 -
 .../distributed/dht/GridDhtTxFinishRequest.java |   5 -
 .../dht/GridDhtTxPrepareRequest.java            |   7 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |   5 -
 .../dht/preloader/GridDhtForceKeysResponse.java |   5 -
 .../GridDhtPartitionDemandMessage.java          |   5 -
 .../GridDhtPartitionSupplyMessage.java          |   5 -
 .../GridDhtPartitionsAbstractMessage.java       |   5 -
 .../dht/preloader/GridDhtPreloader.java         |   6 +
 .../IgniteClientReconnectAbstractTest.java      |  35 ++++-
 .../IgniteClientReconnectCacheTest.java         | 154 +++++++++++++++++++
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   7 +
 ...CacheP2pUnmarshallingRebalanceErrorTest.java |  12 +-
 22 files changed, 266 insertions(+), 225 deletions(-)
----------------------------------------------------------------------



[06/20] ignite git commit: ignite-1564 Fixed client cache reconnect issues

Posted by an...@apache.org.
ignite-1564 Fixed client cache reconnect issues


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

Branch: refs/heads/ignite-1168
Commit: 273f291d9fac0919d57b9ed732564b323a956f90
Parents: cd43967
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 30 13:48:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 30 13:48:48 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManager.java         |  22 +--
 .../processors/cache/GridCacheEventManager.java |  12 +-
 .../processors/cache/GridCacheProcessor.java    |  68 +++++---
 .../dht/preloader/GridDhtPreloader.java         |   6 +
 .../IgniteClientReconnectAbstractTest.java      |  35 ++++-
 .../IgniteClientReconnectCacheTest.java         | 154 +++++++++++++++++++
 6 files changed, 249 insertions(+), 48 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/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 785613d..aec36a2 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
@@ -78,7 +78,7 @@ import org.apache.ignite.internal.processors.jobmetrics.GridJobMetrics;
 import org.apache.ignite.internal.processors.security.SecurityContext;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.F0;
-import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashMap;
+import org.apache.ignite.internal.util.GridBoundedConcurrentOrderedMap;
 import org.apache.ignite.internal.util.GridSpinBusyLock;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -208,7 +208,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
     /** Topology cache history. */
     private final Map<AffinityTopologyVersion, DiscoCache> discoCacheHist =
-        new GridBoundedConcurrentLinkedHashMap<>(DISCOVERY_HISTORY_SIZE, DISCOVERY_HISTORY_SIZE, 0.7f, 1);
+        new GridBoundedConcurrentOrderedMap<>(DISCOVERY_HISTORY_SIZE);
 
     /** Topology snapshots history. */
     private volatile Map<Long, Collection<ClusterNode>> topHist = new HashMap<>();
@@ -465,14 +465,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 final Map<Long, Collection<ClusterNode>> snapshots,
                 @Nullable DiscoverySpiCustomMessage spiCustomMsg
             ) {
-                if (type == EVT_NODE_JOINED && node.isLocal() && ctx.clientDisconnected()) {
-                    discoCacheHist.clear();
-
-                    topHist.clear();
-
-                    topSnap.set(new Snapshot(AffinityTopologyVersion.ZERO, null));
-                }
-
                 DiscoveryCustomMessage customMsg = spiCustomMsg == null ? null
                     : ((CustomMessageWrapper)spiCustomMsg).delegate();
 
@@ -593,6 +585,13 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     locJoinEvt = new GridFutureAdapter<>();
 
                     registeredCaches.clear();
+
+                    discoCacheHist.clear();
+
+                    topHist.clear();
+
+                    topSnap.set(new Snapshot(AffinityTopologyVersion.ZERO,
+                        new DiscoCache(locNode, Collections.<ClusterNode>emptySet())));
                 }
                 else if (type == EVT_CLIENT_NODE_RECONNECTED) {
                     assert locNode.isClient() : locNode;
@@ -620,7 +619,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     return;
                 }
 
-                discoWrk.addEvent(type, nextTopVer, node, topSnapshot, customMsg);
+                if (type == EVT_CLIENT_NODE_DISCONNECTED || type == EVT_NODE_SEGMENTED || !ctx.clientDisconnected())
+                    discoWrk.addEvent(type, nextTopVer, node, topSnapshot, customMsg);
             }
         });
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index c2f8f3f..751c316 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -40,14 +40,6 @@ import static org.apache.ignite.events.EventType.EVT_CACHE_STOPPED;
  * Cache event manager.
  */
 public class GridCacheEventManager extends GridCacheManagerAdapter {
-    /** Local node ID. */
-    private UUID locNodeId;
-
-    /** {@inheritDoc} */
-    @Override public void start0() {
-        locNodeId = cctx.localNodeId();
-    }
-
     /**
      * Adds local event listener.
      *
@@ -96,7 +88,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
     {
         addEvent(part,
             key,
-            locNodeId,
+            cctx.localNodeId(),
             tx,
             owner,
             type,
@@ -116,7 +108,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         addEvent(
             0,
             null,
-            locNodeId,
+            cctx.localNodeId(),
             (IgniteUuid)null,
             null,
             type,

http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index c86dfd9..6c13399 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -26,6 +26,7 @@ import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.IdentityHashMap;
+import java.util.LinkedHashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.ListIterator;
@@ -197,6 +198,9 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** */
     private Map<String, DynamicCacheDescriptor> cachesOnDisconnect;
 
+    /** */
+    private Map<UUID, DynamicCacheChangeBatch> clientReconnectReqs;
+
     /**
      * @param ctx Kernal context.
      */
@@ -1050,6 +1054,13 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         }
 
+        if (clientReconnectReqs != null) {
+            for (Map.Entry<UUID, DynamicCacheChangeBatch> e : clientReconnectReqs.entrySet())
+                processClientReconnectData(e.getKey(), e.getValue());
+
+            clientReconnectReqs = null;
+        }
+
         sharedCtx.onReconnected();
 
         for (GridCacheAdapter cache : reconnected)
@@ -1881,28 +1892,16 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             DynamicCacheChangeBatch batch = (DynamicCacheChangeBatch)data;
 
             if (batch.clientReconnect()) {
-                for (DynamicCacheChangeRequest req : batch.requests()) {
-                    assert !req.template() : req;
-
-                    String name = req.cacheName();
-
-                    boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+                if (ctx.clientDisconnected()) {
+                    if (clientReconnectReqs == null)
+                        clientReconnectReqs = new LinkedHashMap<>();
 
-                    if (!sysCache) {
-                        DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+                    clientReconnectReqs.put(joiningNodeId, batch);
 
-                        if (desc != null && desc.deploymentId().equals(req.deploymentId())) {
-                            Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
-
-                            assert nodes != null : req;
-                            assert nodes.containsKey(joiningNodeId) : nodes;
-
-                            ctx.discovery().addClientNode(req.cacheName(), joiningNodeId, nodes.get(joiningNodeId));
-                        }
-                    }
-                    else
-                        ctx.discovery().addClientNode(req.cacheName(), joiningNodeId, false);
+                    return;
                 }
+
+                processClientReconnectData(joiningNodeId, batch);
             }
             else {
                 for (DynamicCacheChangeRequest req : batch.requests()) {
@@ -1983,6 +1982,37 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param clientNodeId Client node ID.
+     * @param batch Cache change batch.
+     */
+    private void processClientReconnectData(UUID clientNodeId, DynamicCacheChangeBatch batch) {
+        assert batch.clientReconnect() : batch;
+
+        for (DynamicCacheChangeRequest req : batch.requests()) {
+            assert !req.template() : req;
+
+            String name = req.cacheName();
+
+            boolean sysCache = CU.isMarshallerCache(name) || CU.isUtilityCache(name) || CU.isAtomicsCache(name);
+
+            if (!sysCache) {
+                DynamicCacheDescriptor desc = registeredCaches.get(maskNull(req.cacheName()));
+
+                if (desc != null && desc.deploymentId().equals(req.deploymentId())) {
+                    Map<UUID, Boolean> nodes = batch.clientNodes().get(name);
+
+                    assert nodes != null : req;
+                    assert nodes.containsKey(clientNodeId) : nodes;
+
+                    ctx.discovery().addClientNode(req.cacheName(), clientNodeId, nodes.get(clientNodeId));
+                }
+            }
+            else
+                ctx.discovery().addClientNode(req.cacheName(), clientNodeId, false);
+        }
+    }
+
+    /**
      * Dynamically starts cache using template configuration.
      *
      * @param cacheName Cache name.

http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 9d5fdca..19b461e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -260,6 +260,12 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     /** {@inheritDoc} */
     @Override public void onReconnected() {
         startFut = new GridFutureAdapter<>();
+
+        long topVer0 = cctx.kernalContext().discovery().topologyVersion();
+
+        assert topVer0 > 0 : topVer0;
+
+        topVer.set(topVer0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
index 3a6d04f..0c1df7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectAbstractTest.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal;
 
 import java.io.IOException;
 import java.net.Socket;
+import java.util.Collections;
+import java.util.List;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -199,15 +201,28 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
      */
     protected void reconnectClientNode(Ignite client, Ignite srv, @Nullable Runnable disconnectedC)
         throws Exception {
-        final TestTcpDiscoverySpi clientSpi = spi(client);
+        reconnectClientNodes(Collections.singletonList(client), srv, disconnectedC);
+    }
+
+    /**
+     * Reconnect client node.
+     *
+     * @param clients Clients.
+     * @param srv Server.
+     * @param disconnectedC Closure which will be run when client node disconnected.
+     * @throws Exception If failed.
+     */
+    protected void reconnectClientNodes(List<Ignite> clients, Ignite srv, @Nullable Runnable disconnectedC)
+        throws Exception {
         final TestTcpDiscoverySpi srvSpi = spi(srv);
 
-        final CountDownLatch disconnectLatch = new CountDownLatch(1);
-        final CountDownLatch reconnectLatch = new CountDownLatch(1);
+        final CountDownLatch disconnectLatch = new CountDownLatch(clients.size());
+        final CountDownLatch reconnectLatch = new CountDownLatch(clients.size());
 
         log.info("Block reconnect.");
 
-        clientSpi.writeLatch = new CountDownLatch(1);
+        for (Ignite client : clients)
+            spi(client).writeLatch = new CountDownLatch(1);
 
         IgnitePredicate<Event> p = new IgnitePredicate<Event>() {
             @Override public boolean apply(Event evt) {
@@ -226,9 +241,11 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
             }
         };
 
-        client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+        for (Ignite client : clients)
+            client.events().localListen(p, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
 
-        srvSpi.failNode(client.cluster().localNode().id(), null);
+        for (Ignite client : clients)
+            srvSpi.failNode(client.cluster().localNode().id(), null);
 
         waitReconnectEvent(disconnectLatch);
 
@@ -237,11 +254,13 @@ public abstract class IgniteClientReconnectAbstractTest extends GridCommonAbstra
 
         log.info("Allow reconnect.");
 
-        clientSpi.writeLatch.countDown();
+        for (Ignite client : clients)
+            spi(client).writeLatch.countDown();
 
         waitReconnectEvent(reconnectLatch);
 
-        client.events().stopLocalListen(p);
+        for (Ignite client : clients)
+            client.events().stopLocalListen(p);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/273f291d/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
index 807027c..edd95e9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientReconnectCacheTest.java
@@ -40,6 +40,7 @@ import org.apache.ignite.Ignition;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.cluster.ClusterGroup;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -998,6 +999,159 @@ public class IgniteClientReconnectCacheTest extends IgniteClientReconnectAbstrac
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectClusterRestartMultinode() throws Exception {
+        clientMode = true;
+
+        final int CLIENTS = 5;
+
+        CountDownLatch disconnectLatch = new CountDownLatch(CLIENTS);
+        CountDownLatch reconnectLatch = new CountDownLatch(CLIENTS);
+
+        List<IgniteCache> caches = new ArrayList<>();
+
+        for (int i = 0; i < CLIENTS; i++) {
+            Ignite client = startGrid(SRV_CNT + i);
+
+            addListener(client, disconnectLatch, reconnectLatch);
+
+            IgniteCache cache = client.getOrCreateCache(new CacheConfiguration<>());
+
+            assertNotNull(cache);
+
+            caches.add(cache);
+        }
+
+        for (int i = 0; i < SRV_CNT; i++)
+            stopGrid(i);
+
+        assertTrue(disconnectLatch.await(30_000, MILLISECONDS));
+
+        log.info("Restart servers.");
+
+        clientMode = false;
+
+        startGridsMultiThreaded(0, SRV_CNT);
+
+        assertTrue(reconnectLatch.await(30_000, MILLISECONDS));
+
+        for (final IgniteCache clientCache : caches) {
+            GridTestUtils.assertThrows(log, new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    return clientCache.get(1);
+                }
+            }, IllegalStateException.class, null);
+        }
+
+        for (int i = 0; i < SRV_CNT + CLIENTS; i++) {
+            Ignite ignite = grid(i);
+
+            ClusterGroup grp = ignite.cluster().forCacheNodes(null);
+
+            assertEquals(0, grp.nodes().size());
+
+            grp = ignite.cluster().forClientNodes(null);
+
+            assertEquals(0, grp.nodes().size());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testReconnectMultinode() throws Exception {
+        grid(0).createCache(new CacheConfiguration<>());
+
+        clientMode = true;
+
+        final int CLIENTS = 2;
+
+        List<Ignite> clients = new ArrayList<>();
+
+        for (int i = 0; i < CLIENTS; i++) {
+            Ignite client = startGrid(SRV_CNT + i);
+
+            assertNotNull(client.getOrCreateCache(new CacheConfiguration<>()));
+
+            clients.add(client);
+        }
+
+        int nodes = SRV_CNT + CLIENTS;
+        int srvNodes = SRV_CNT;
+
+        for (int iter = 0; iter < 3; iter++) {
+            log.info("Iteration: " + iter);
+
+            reconnectClientNodes(clients, grid(0), null);
+
+            for (Ignite client : clients) {
+                IgniteCache<Object, Object> cache = client.cache(null);
+
+                assertNotNull(cache);
+
+                cache.put(client.name(), 1);
+
+                assertEquals(1, cache.get(client.name()));
+
+                ClusterGroup grp = client.cluster().forCacheNodes(null);
+
+                assertEquals(CLIENTS + srvNodes, grp.nodes().size());
+
+                grp = client.cluster().forClientNodes(null);
+
+                assertEquals(CLIENTS, grp.nodes().size());
+            }
+
+            for (int i = 0; i < nodes; i++) {
+                Ignite ignite = grid(i);
+
+                ClusterGroup grp = ignite.cluster().forCacheNodes(null);
+
+                assertEquals(CLIENTS + srvNodes, grp.nodes().size());
+
+                grp = ignite.cluster().forClientNodes(null);
+
+                assertEquals(CLIENTS, grp.nodes().size());
+            }
+
+            clientMode = false;
+
+            startGrid(nodes++);
+
+            srvNodes++;
+
+            clientMode = true;
+
+            startGrid(nodes++);
+        }
+    }
+
+    /**
+     * @param client Client.
+     * @param disconnectLatch Disconnect event latch.
+     * @param reconnectLatch Reconnect event latch.
+     */
+    private void addListener(Ignite client, final CountDownLatch disconnectLatch, final CountDownLatch reconnectLatch) {
+        client.events().localListen(new IgnitePredicate<Event>() {
+            @Override public boolean apply(Event evt) {
+                if (evt.type() == EVT_CLIENT_NODE_DISCONNECTED) {
+                    info("Disconnected: " + evt);
+
+                    disconnectLatch.countDown();
+                }
+                else if (evt.type() == EVT_CLIENT_NODE_RECONNECTED) {
+                    info("Reconnected: " + evt);
+
+                    reconnectLatch.countDown();
+                }
+
+                return true;
+            }
+        }, EVT_CLIENT_NODE_DISCONNECTED, EVT_CLIENT_NODE_RECONNECTED);
+    }
+
+    /**
      *
      */
     static class TestClass1 implements Serializable {}


[04/20] ignite git commit: # master - changed "communication encryption" to "ssl"

Posted by an...@apache.org.
# master - changed "communication encryption" to "ssl"


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

Branch: refs/heads/ignite-1168
Commit: c6ec7e1165017d4331199bdf667b81a6f2ed91c8
Parents: 54bb7d7
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Wed Sep 30 10:52:15 2015 +0200
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Wed Sep 30 10:52:15 2015 +0200

----------------------------------------------------------------------
 .../apache/ignite/internal/IgniteKernal.java    | 70 ++++++++++----------
 1 file changed, 35 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c6ec7e11/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 60725e4..68a0383 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -17,38 +17,6 @@
 
 package org.apache.ignite.internal;
 
-import java.io.Externalizable;
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InvalidObjectException;
-import java.io.ObjectInput;
-import java.io.ObjectOutput;
-import java.io.ObjectStreamException;
-import java.io.Serializable;
-import java.lang.management.ManagementFactory;
-import java.lang.management.RuntimeMXBean;
-import java.lang.ref.WeakReference;
-import java.lang.reflect.Constructor;
-import java.text.DateFormat;
-import java.text.DecimalFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Date;
-import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Timer;
-import java.util.UUID;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-import javax.management.JMException;
-import javax.management.ObjectName;
 import org.apache.ignite.IgniteAtomicLong;
 import org.apache.ignite.IgniteAtomicReference;
 import org.apache.ignite.IgniteAtomicSequence;
@@ -64,7 +32,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
-import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.IgniteQueue;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteServices;
@@ -94,6 +61,7 @@ import org.apache.ignite.internal.managers.failover.GridFailoverManager;
 import org.apache.ignite.internal.managers.indexing.GridIndexingManager;
 import org.apache.ignite.internal.managers.loadbalancer.GridLoadBalancerManager;
 import org.apache.ignite.internal.managers.swapspace.GridSwapSpaceManager;
+import org.apache.ignite.internal.portable.api.IgnitePortables;
 import org.apache.ignite.internal.processors.GridProcessor;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
@@ -157,7 +125,6 @@ import org.apache.ignite.lifecycle.LifecycleBean;
 import org.apache.ignite.lifecycle.LifecycleEventType;
 import org.apache.ignite.marshaller.MarshallerExclusions;
 import org.apache.ignite.marshaller.optimized.OptimizedMarshaller;
-import org.apache.ignite.internal.portable.api.PortableMarshaller;
 import org.apache.ignite.mxbean.ClusterLocalNodeMetricsMXBean;
 import org.apache.ignite.mxbean.IgniteMXBean;
 import org.apache.ignite.mxbean.ThreadPoolMXBean;
@@ -168,6 +135,39 @@ import org.apache.ignite.spi.IgniteSpi;
 import org.apache.ignite.spi.IgniteSpiVersionCheckException;
 import org.jetbrains.annotations.Nullable;
 
+import javax.management.JMException;
+import javax.management.ObjectName;
+import java.io.Externalizable;
+import java.io.File;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InvalidObjectException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.io.ObjectStreamException;
+import java.io.Serializable;
+import java.lang.management.ManagementFactory;
+import java.lang.management.RuntimeMXBean;
+import java.lang.ref.WeakReference;
+import java.lang.reflect.Constructor;
+import java.text.DateFormat;
+import java.text.DecimalFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Date;
+import java.util.List;
+import java.util.ListIterator;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Timer;
+import java.util.UUID;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONFIG_URL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII;
@@ -2177,7 +2177,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         assert log != null;
 
         U.quietAndInfo(log, "Security status [authentication=" + onOff(ctx.security().enabled())
-            + ", communication encryption=" + onOff(ctx.config().getSslContextFactory() != null) + ']');
+            + ", ssl=" + onOff(ctx.config().getSslContextFactory() != null) + ']');
     }
 
     /**


[02/20] ignite git commit: More info in assert.

Posted by an...@apache.org.
More info in assert.


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

Branch: refs/heads/ignite-1168
Commit: be906e32f68d35e01f0b0e5f610937a27277f71a
Parents: bfabab2
Author: sboikov <sb...@gridgain.com>
Authored: Wed Sep 30 09:29:38 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Sep 30 09:29:38 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/util/nio/GridNioRecoveryDescriptor.java      | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be906e32/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 42656c4..88837de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -259,8 +259,8 @@ public class GridNioRecoveryDescriptor {
      */
     public void connected() {
         synchronized (this) {
-            assert reserved;
-            assert !connected;
+            assert reserved : this;
+            assert !connected : this;
 
             connected = true;
 


[07/20] ignite git commit: Test fix

Posted by an...@apache.org.
Test fix


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

Branch: refs/heads/ignite-1168
Commit: 1282f8b1ddc248e562ecb937bce84acabb8198f7
Parents: 273f291
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 15:16:31 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 15:16:31 2015 +0300

----------------------------------------------------------------------
 .../cache/IgniteCacheP2pUnmarshallingErrorTest.java     |  7 +++++++
 .../IgniteCacheP2pUnmarshallingRebalanceErrorTest.java  | 12 ++++++------
 2 files changed, 13 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1282f8b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
index b62cc48..1b2b84d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingErrorTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.util.typedef.X;
@@ -76,6 +77,12 @@ public class IgniteCacheP2pUnmarshallingErrorTest extends IgniteCacheAbstractTes
             cfg.setCacheConfiguration();
         }
 
+        if (getTestGridName(10).equals(gridName)) {
+
+            CacheConfiguration cc = cfg.getCacheConfiguration()[0];
+            cc.setRebalanceDelay(-1);
+        }
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1282f8b1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
index 9faeac8..73388fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheP2pUnmarshallingRebalanceErrorTest.java
@@ -52,20 +52,20 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
 
         readCnt.set(Integer.MAX_VALUE);
 
-        for (int i = 0; i <= 1000; i++)
+        for (int i = 0; i <= 100; i++)
             jcache(0).put(new TestKey(String.valueOf(++key)), "");
 
-        startGrid(3);
+        startGrid(10); //custom rebalanceDelay set at cfg.
 
-        Affinity<Object> aff = affinity(grid(3).cache(null));
+        Affinity<Object> aff = affinity(grid(10).cache(null));
 
-        while (!aff.isPrimary(grid(3).localNode(), new TestKey(String.valueOf(key))))
+        while (!aff.isPrimary(grid(10).localNode(), new TestKey(String.valueOf(key))))
             --key;
 
         readCnt.set(1);
 
         try {
-            jcache(3).get(new TestKey(String.valueOf(key)));
+            jcache(10).get(new TestKey(String.valueOf(key)));
 
             assert false : "p2p marshalling failed, but error response was not sent";
         }
@@ -73,4 +73,4 @@ public class IgniteCacheP2pUnmarshallingRebalanceErrorTest extends IgniteCacheP2
             assert X.hasCause(e, IOException.class);
         }
     }
-}
\ No newline at end of file
+}


[15/20] ignite git commit: https://issues.apache.org/jira/browse/IGNITE-1584

Posted by an...@apache.org.
https://issues.apache.org/jira/browse/IGNITE-1584


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

Branch: refs/heads/ignite-1168
Commit: d3f04ff8a20949125d907fbc781c539bc81c924c
Parents: 3de0e47
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 17:28:28 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 17:28:28 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/IgniteCacheAtomicNodeRestartTest.java  | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/d3f04ff8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
index dc83ab9..327db0e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicNodeRestartTest.java
@@ -30,4 +30,12 @@ public class IgniteCacheAtomicNodeRestartTest extends GridCachePartitionedNodeRe
     @Override protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }
+
+    @Override public void testRestartWithPutFourNodesNoBackups() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1587");
+    }
+
+    @Override public void testRestartWithPutFourNodesOneBackupsOffheapTiered() {
+        fail("https://issues.apache.org/jira/browse/IGNITE-1587");
+    }
 }
\ No newline at end of file


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

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


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

Branch: refs/heads/ignite-1168
Commit: 52600f30dca8397f417a53611231d1ad3e66454b
Parents: 7571f21 de39fbf
Author: Anton Vinogradov <av...@apache.org>
Authored: Wed Sep 30 15:44:07 2015 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Wed Sep 30 15:44:07 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------