You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/07/04 09:15:12 UTC

[1/6] ignite git commit: IGNITE-5613 - Fixed race on local sequence increment and distributed update

Repository: ignite
Updated Branches:
  refs/heads/ignite-gg-12389 160712cd2 -> ef13dfadd


IGNITE-5613 - Fixed race on local sequence increment and distributed update


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

Branch: refs/heads/ignite-gg-12389
Commit: 7d42dea8e3e50707c34e8e8d211cd54da1505210
Parents: c4ddda3
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jul 3 17:05:48 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jul 3 17:05:48 2017 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicSequenceImpl.java            | 55 ++++++++++++--------
 ...titionedAtomicSequenceMultiThreadedTest.java | 32 ++++++++++++
 2 files changed, 64 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/7d42dea8/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index 725e4aa..47fa49e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -382,39 +382,48 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
 
                     long newUpBound;
 
-                    curLocVal = locVal;
+                    // Even though we hold a transaction lock here, we must hold the local update lock here as well
+                    // because we mutate multipe variables (locVal and upBound).
+                    localUpdate.lock();
 
-                    // If local range was already reserved in another thread.
-                    if (curLocVal + l <= upBound) {
-                        locVal = curLocVal + l;
+                    try {
+                        curLocVal = locVal;
 
-                        return updated ? curLocVal + l : curLocVal;
-                    }
+                        // If local range was already reserved in another thread.
+                        if (curLocVal + l <= upBound) {
+                            locVal = curLocVal + l;
 
-                    long curGlobalVal = seq.get();
+                            return updated ? curLocVal + l : curLocVal;
+                        }
 
-                    long newLocVal;
+                        long curGlobalVal = seq.get();
 
-                    /* We should use offset because we already reserved left side of range.*/
-                    long off = batchSize > 1 ? batchSize - 1 : 1;
+                        long newLocVal;
 
-                    // Calculate new values for local counter, global counter and upper bound.
-                    if (curLocVal + l >= curGlobalVal) {
-                        newLocVal = curLocVal + l;
+                        /* We should use offset because we already reserved left side of range.*/
+                        long off = batchSize > 1 ? batchSize - 1 : 1;
 
-                        newUpBound = newLocVal + off;
-                    }
-                    else {
-                        newLocVal = curGlobalVal;
+                        // Calculate new values for local counter, global counter and upper bound.
+                        if (curLocVal + l >= curGlobalVal) {
+                            newLocVal = curLocVal + l;
 
-                        newUpBound = newLocVal + off;
-                    }
+                            newUpBound = newLocVal + off;
+                        }
+                        else {
+                            newLocVal = curGlobalVal;
 
-                    locVal = newLocVal;
-                    upBound = newUpBound;
+                            newUpBound = newLocVal + off;
+                        }
 
-                    if (updated)
-                        curLocVal = newLocVal;
+                        locVal = newLocVal;
+                        upBound = newUpBound;
+
+                        if (updated)
+                            curLocVal = newLocVal;
+                    }
+                    finally {
+                        localUpdate.unlock();
+                    }
 
                     // Global counter must be more than reserved upper bound.
                     seq.set(newUpBound + 1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/7d42dea8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
index 945650d..4db9bd3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedAtomicSequenceMultiThreadedTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.datastructures.partitioned;
 
 import java.util.Random;
 import java.util.UUID;
+import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteAtomicSequence;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cache.CacheMode;
@@ -26,6 +27,7 @@ import org.apache.ignite.configuration.AtomicConfiguration;
 import org.apache.ignite.internal.processors.cache.datastructures.IgniteAtomicsAbstractTest;
 import org.apache.ignite.internal.processors.datastructures.GridCacheAtomicSequenceImpl;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
 
 import static org.apache.ignite.cache.CacheMode.PARTITIONED;
 
@@ -281,6 +283,36 @@ public class GridCachePartitionedAtomicSequenceMultiThreadedTest extends IgniteA
     }
 
     /**
+     * @throws Exception if failed.
+     */
+    public void testMultipleSequences() throws Exception {
+        final int seqCnt = 5;
+        final int threadCnt = 5;
+        final int incCnt = 1_000;
+
+        final IgniteAtomicSequence[] seqs = new IgniteAtomicSequence[seqCnt];
+
+        String seqName = UUID.randomUUID().toString();
+
+        for (int i = 0; i < seqs.length; i++)
+            seqs[i] = grid(0).atomicSequence(seqName, 0, true);
+
+        GridTestUtils.runMultiThreaded(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                for (int i = 0; i < incCnt; i++) {
+                    for (IgniteAtomicSequence seq : seqs)
+                        seq.incrementAndGet();
+                }
+
+                return null;
+            }
+        }, threadCnt, "load");
+
+        for (IgniteAtomicSequence seq : seqs)
+            assertEquals(seqCnt * threadCnt * incCnt, seq.get());
+    }
+
+    /**
      * Executes given closure in a given number of threads given number of times.
      *
      * @param c Closure to execute.


[6/6] ignite git commit: ignite-gg-12389

Posted by sb...@apache.org.
ignite-gg-12389


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

Branch: refs/heads/ignite-gg-12389
Commit: ef13dfadd856d8e19a5bc7eeae47767f79b1c5e1
Parents: a9e1753
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 4 11:50:11 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 4 12:13:35 2017 +0300

----------------------------------------------------------------------
 .../cache/CacheAffinitySharedManager.java          |  2 +-
 .../processors/cache/ClusterCachesInfo.java        |  4 +++-
 .../cache/DynamicCacheChangeRequest.java           | 17 +++++++++++++++++
 .../cluster/GridClusterStateProcessor.java         | 14 +++++++-------
 .../ignite/spi/discovery/tcp/ClientImpl.java       | 13 +++++++------
 .../cache/IgniteClusterActivateDeactivateTest.java |  3 +++
 .../testframework/junits/GridAbstractTest.java     |  4 ++--
 7 files changed, 40 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
index 2bf7838..6f7f4c2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java
@@ -710,7 +710,7 @@ public class CacheAffinitySharedManager<K, V> extends GridCacheSharedManagerAdap
 
             NearCacheConfiguration nearCfg = null;
 
-            if (cctx.localNodeId().equals(req.initiatingNodeId()) && !exchActions.activate()) {
+            if (req.locallyConfigured() || (cctx.localNodeId().equals(req.initiatingNodeId()) && !exchActions.activate())) {
                 startCache = true;
 
                 nearCfg = req.nearCacheConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
index ab24814..a0a29d9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java
@@ -1083,7 +1083,7 @@ class ClusterCachesInfo {
 
                 DynamicCacheChangeRequest req = new DynamicCacheChangeRequest(msg.requestId(),
                     desc.cacheName(),
-                    locCfg != null ? ctx.localNodeId() : null);
+                    msg.initiatorNodeId());
 
                 req.startCacheConfiguration(desc.cacheConfiguration());
                 req.cacheType(desc.cacheType());
@@ -1093,6 +1093,8 @@ class ClusterCachesInfo {
                         req.startCacheConfiguration(locCfg.get1());
 
                     req.nearCacheConfiguration(locCfg.get2());
+
+                    req.locallyConfigured(true);
                 }
 
                 exchangeActions.addCacheToStart(req, desc);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
index f2fce18..2fd8780 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeRequest.java
@@ -89,6 +89,9 @@ public class DynamicCacheChangeRequest implements Serializable {
     /** Dynamic schema. */
     private QuerySchema schema;
 
+    /** */
+    private transient boolean locallyConfigured;
+
     /**
      * @param reqId Unique request ID.
      * @param cacheName Cache stop name.
@@ -387,6 +390,20 @@ public class DynamicCacheChangeRequest implements Serializable {
         this.schema = schema != null ? schema.copy() : null;
     }
 
+    /**
+     * @return Locally configured flag.
+     */
+    public boolean locallyConfigured() {
+        return locallyConfigured;
+    }
+
+    /**
+     * @param locallyConfigured Locally configured flag.
+     */
+    public void locallyConfigured(boolean locallyConfigured) {
+        this.locallyConfigured = locallyConfigured;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return "DynamicCacheChangeRequest [cacheName=" + cacheName() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
index 9e1ddfb..c262a3e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cluster/GridClusterStateProcessor.java
@@ -98,12 +98,13 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
 
             final GridChangeGlobalStateFuture f = stateChangeFut.get();
 
-            if (f != null)
+            if (f != null) {
                 f.initFut.listen(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> fut) {
-                        f.onDiscoveryEvent(e);
+                        f.onNodeLeft(e);
                     }
                 });
+            }
         }
     };
 
@@ -220,9 +221,6 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         }
         else {
             if (globalState.active() != msg.activate()) {
-//                if (!ctx.localNodeId().equals(msg.initiatorNodeId()))
-//                    stateChangeFut.compareAndSet(null, new GridChangeGlobalStateFuture(msg.requestId(), msg.activate(), ctx));
-// TODO GG-12389
                 Set<UUID> nodeIds = U.newHashSet(discoCache.allNodes().size());
 
                 for (ClusterNode node : discoCache.allNodes())
@@ -691,7 +689,9 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         private final IgniteLogger log;
 
         /**
-         *
+         * @param requestId State change request ID.
+         * @param activate New cluster state.
+         * @param ctx Context.
          */
         GridChangeGlobalStateFuture(UUID requestId, boolean activate, GridKernalContext ctx) {
             this.requestId = requestId;
@@ -704,7 +704,7 @@ public class GridClusterStateProcessor extends GridProcessorAdapter {
         /**
          * @param event Event.
          */
-        void onDiscoveryEvent(DiscoveryEvent event) {
+        void onNodeLeft(DiscoveryEvent event) {
             assert event != null;
 
             if (isDone())

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 0bc1239..5dbfe6e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -484,8 +484,6 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         long startTime = U.currentTimeMillis();
 
-        DiscoveryDataPacket discoveryData = recon ? null : spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
-
         while (true) {
             if (Thread.currentThread().isInterrupted())
                 throw new InterruptedException();
@@ -523,7 +521,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 InetSocketAddress addr = it.next();
 
-                T3<SocketStream, Integer, Boolean> sockAndRes = sendJoinRequest(recon, addr, discoveryData);
+                T3<SocketStream, Integer, Boolean> sockAndRes = sendJoinRequest(recon, addr);
 
                 if (sockAndRes == null) {
                     it.remove();
@@ -582,12 +580,10 @@ class ClientImpl extends TcpDiscoveryImpl {
     /**
      * @param recon {@code True} if reconnects.
      * @param addr Address.
-     * @param discoveryData Discovery data.
      * @return Socket, connect response and client acknowledge support flag.
      */
     @Nullable private T3<SocketStream, Integer, Boolean> sendJoinRequest(boolean recon,
-        InetSocketAddress addr,
-        DiscoveryDataPacket discoveryData) {
+        InetSocketAddress addr) {
         assert addr != null;
 
         if (log.isDebugEnabled())
@@ -608,6 +604,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(spi, true);
 
+        DiscoveryDataPacket discoveryData = null;
+
         while (true) {
             boolean openSock = false;
 
@@ -650,6 +648,9 @@ class ClientImpl extends TcpDiscoveryImpl {
                         marshalCredentials(node);
                     }
 
+                    if (discoveryData == null)
+                        discoveryData = spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
+
                     msg = new TcpDiscoveryJoinRequestMessage(node, discoveryData);
                 }
                 else

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
index 894bd3e..a9deeed 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -35,6 +35,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.TestRecordingCommunicationSpi;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -191,6 +192,8 @@ public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest
         for (int i = 0; i < srvs + clients; i++) {
             for (int c = 0; c < 2; c++)
                 checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+
+            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
         }
 
         checkCaches(srvs + clients, CACHES);

http://git-wip-us.apache.org/repos/asf/ignite/blob/ef13dfad/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index a9a870e..d6d241c 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -2099,13 +2099,13 @@ public abstract class GridAbstractTest extends TestCase {
         for (Ignite g : G.allGrids()) {
             final GridKernalContext ctx = ((IgniteKernal)g).context();
 
-            if (ctx.isStopping())
+            if (ctx.isStopping() || !g.active())
                 continue;
 
             AffinityTopologyVersion topVer = ctx.discovery().topologyVersionEx();
             AffinityTopologyVersion exchVer = ctx.cache().context().exchange().readyAffinityVersion();
 
-            if (! topVer.equals(exchVer)) {
+            if (!topVer.equals(exchVer)) {
                 info("Topology version mismatch [node="  + g.name() +
                     ", exchVer=" + exchVer +
                     ", topVer=" + topVer + ']');


[5/6] ignite git commit: ignite-gg-12389

Posted by sb...@apache.org.
ignite-gg-12389


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

Branch: refs/heads/ignite-gg-12389
Commit: a9e1753b6a8e4cafff43799ee94aa337748f39cc
Parents: c3f6344
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 4 10:38:52 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 4 10:38:52 2017 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCacheSharedContext.java      | 1 +
 .../cache/persistence/IgniteCacheSnapshotManager.java          | 6 ++++--
 .../java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java   | 2 +-
 3 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a9e1753b/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 32a2bce..eeec9c2 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
@@ -322,6 +322,7 @@ public class GridCacheSharedContext<K, V> {
     }
 
     /**
+     * @param active Active flag.
      * @throws IgniteCheckedException If failed.
      */
     void onReconnected(boolean active) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9e1753b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
index 7dd93f7..657e782 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/IgniteCacheSnapshotManager.java
@@ -116,7 +116,7 @@ public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter im
         FullPageId fullId,
         PageMemory pageMem
     ) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /**
@@ -127,13 +127,15 @@ public class IgniteCacheSnapshotManager extends GridCacheSharedManagerAdapter im
         ByteBuffer pageBuf,
         Integer tag
     ) throws IgniteCheckedException {
-
+        // No-op.
     }
 
+    /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
         // No-op.
     }
 
+    /** {@inheritDoc} */
     @Override public void onDeActivate(GridKernalContext kctx) {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/a9e1753b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index b469c07..0bc1239 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -484,7 +484,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         long startTime = U.currentTimeMillis();
 
-        DiscoveryDataPacket discoveryData = spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
+        DiscoveryDataPacket discoveryData = recon ? null : spi.collectExchangeData(new DiscoveryDataPacket(getLocalNodeId()));
 
         while (true) {
             if (Thread.currentThread().isInterrupted())


[3/6] ignite git commit: Merge remote-tracking branch 'professional/ignite-2.1.2' into ignite-2.1.2

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


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

Branch: refs/heads/ignite-gg-12389
Commit: ae5ec942710c9c70829288169474787d1e682b35
Parents: 64596ea 7d42dea
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon Jul 3 18:20:55 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Jul 3 18:20:55 2017 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicSequenceImpl.java            | 55 ++++++++++++--------
 ...titionedAtomicSequenceMultiThreadedTest.java | 32 ++++++++++++
 2 files changed, 64 insertions(+), 23 deletions(-)
----------------------------------------------------------------------



[2/6] ignite git commit: ignite-2.1.2 more assert info in wal manager

Posted by sb...@apache.org.
ignite-2.1.2 more assert info in wal manager


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

Branch: refs/heads/ignite-gg-12389
Commit: 64596ea8348105bf7b0e931af89e31190a2b53e5
Parents: c4ddda3
Author: Dmitriy Govorukhin <dm...@gmail.com>
Authored: Mon Jul 3 18:20:35 2017 +0300
Committer: Dmitriy Govorukhin <dm...@gmail.com>
Committed: Mon Jul 3 18:20:35 2017 +0300

----------------------------------------------------------------------
 .../cache/persistence/wal/FileWriteAheadLogManager.java        | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/64596ea8/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
index 225776d..5918141 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
@@ -1918,7 +1918,11 @@ public class FileWriteAheadLogManager extends GridCacheSharedManagerAdapter impl
             lock.lock();
 
             try {
-                assert head.get() instanceof FakeRecord: "head";
+                WALRecord rec = head.get();
+
+                assert rec instanceof FakeRecord : "Expected head FakeRecord, actual head "
+                    + (rec != null ? rec.getClass().getSimpleName() : "null");
+
                 assert written == lastFsyncPos || mode != WALMode.DEFAULT :
                     "fsync [written=" + written + ", lastFsync=" + lastFsyncPos + ']';
 


[4/6] ignite git commit: Merge remote-tracking branch 'remotes/community/ignite-2.1.2' into ignite-gg-12389

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/community/ignite-2.1.2' into ignite-gg-12389


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

Branch: refs/heads/ignite-gg-12389
Commit: c3f6344c4a9019246c1e75c4807cdd2cf0981305
Parents: 160712c ae5ec94
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jul 4 10:25:03 2017 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jul 4 10:25:03 2017 +0300

----------------------------------------------------------------------
 .../wal/FileWriteAheadLogManager.java           |  6 ++-
 .../GridCacheAtomicSequenceImpl.java            | 55 ++++++++++++--------
 ...titionedAtomicSequenceMultiThreadedTest.java | 32 ++++++++++++
 3 files changed, 69 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c3f6344c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/wal/FileWriteAheadLogManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/c3f6344c/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------