You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2017/01/23 18:17:21 UTC

[1/5] ignite git commit: ignite-db-x fix rebalance npe, index tree npe

Repository: ignite
Updated Branches:
  refs/heads/ignite-3477 eda245b86 -> c875221a4


ignite-db-x  fix rebalance npe, index tree npe


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

Branch: refs/heads/ignite-3477
Commit: 26000caa83b92d43ce14fd53f7802e9802a7670c
Parents: b9826ef
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Mon Jan 23 18:33:14 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Mon Jan 23 18:33:14 2017 +0300

----------------------------------------------------------------------
 .../atomic/GridNearAtomicSingleUpdateFuture.java  |  2 +-
 .../dht/preloader/GridDhtPartitionDemander.java   | 18 +++++-------------
 .../cache/transactions/IgniteTxHandler.java       |  9 ++++++---
 .../processors/query/h2/database/H2TreeIndex.java |  2 +-
 4 files changed, 13 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/26000caa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index 1f421c9..5261e34 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -208,7 +208,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             if (!res.futureVersion().equals(futVer))
                 return;
 
-            if (!this.req.nodeId().equals(nodeId))
+            if (this.req == null || !this.req.nodeId().equals(nodeId))
                 return;
 
             req = this.req;

http://git-wip-us.apache.org/repos/asf/ignite/blob/26000caa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index d2efd24..204e4da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -452,23 +452,15 @@ public class GridDhtPartitionDemander {
         for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
             final ClusterNode node = e.getKey();
 
-            GridDhtPartitionDemandMessage d = e.getValue();
+            final GridDhtPartitionDemandMessage d = e.getValue();
 
-            fut.appendPartitions(node.id(), d.partitions()); //Future preparation.
-        }
-
-        for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
-            final ClusterNode node = e.getKey();
-
-            final CacheConfiguration cfg = cctx.config();
-
-            final Collection<Integer> parts = fut.remaining.get(node.id()).get2();
+            final Collection<Integer> parts = d.partitions();
 
-            GridDhtPartitionDemandMessage d = e.getValue();
+            fut.appendPartitions(node.id(), parts);
 
             //Check remote node rebalancing API version.
             if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) {
-                U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() +
+                U.log(log, "Starting rebalancing [mode=" + cctx.config().getRebalanceMode() +
                     ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
                     ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
 
@@ -513,7 +505,7 @@ public class GridDhtPartitionDemander {
             }
             else {
                 U.log(log, "Starting rebalancing (old api) [cache=" + cctx.name() +
-                    ", mode=" + cfg.getRebalanceMode() +
+                    ", mode=" + cctx.config().getRebalanceMode() +
                     ", fromNode=" + node.id() +
                     ", partitionsCount=" + parts.size() +
                     ", topology=" + fut.topologyVersion() +

http://git-wip-us.apache.org/repos/asf/ignite/blob/26000caa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index 240e9c9..ba2ab3c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -675,15 +675,18 @@ public class IgniteTxHandler {
      * @param req Request.
      * @return Future.
      */
-    @Nullable private IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId,
-        GridNearTxFinishRequest req) {
+    @Nullable private IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(
+        UUID nodeId,
+        GridNearTxFinishRequest req
+    ) {
         if (txFinishMsgLog.isDebugEnabled())
             txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId + ']');
 
         IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
 
         assert req.txState() != null || (fut != null && fut.error() != null) ||
-            (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null);
+            (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null) :
+            "[req=" + req + ", fut=" + fut + "]";
 
         return fut;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/26000caa/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
index 19cbbf9..f82746f 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
@@ -199,7 +199,7 @@ public class H2TreeIndex extends GridH2IndexBase {
     /** {@inheritDoc} */
     @Override public void destroy() {
         try {
-            if (!cctx.kernalContext().clientNode()) {
+            if (cctx.affinityNode()) {
                 tree.destroy();
 
                 cctx.offheap().dropRootPageForIndex(tree.getName());


[3/5] ignite git commit: ignite-db-x minor update

Posted by ag...@apache.org.
ignite-db-x minor update


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

Branch: refs/heads/ignite-3477
Commit: 742aa94c258791ff39365fa55b15d9beb8b1766b
Parents: 26000ca
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Mon Jan 23 19:20:47 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Mon Jan 23 19:20:47 2017 +0300

----------------------------------------------------------------------
 .../distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/742aa94c/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
index 5261e34..f69fb4f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateFuture.java
@@ -208,7 +208,7 @@ public class GridNearAtomicSingleUpdateFuture extends GridNearAtomicAbstractUpda
             if (!res.futureVersion().equals(futVer))
                 return;
 
-            if (this.req == null || !this.req.nodeId().equals(nodeId))
+            if ((nodeErr && this.req == null) || !this.req.nodeId().equals(nodeId))
                 return;
 
             req = this.req;


[5/5] ignite git commit: Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x

Posted by ag...@apache.org.
Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x


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

Branch: refs/heads/ignite-3477
Commit: c875221a4c11de7d8933f19f4b9d46a148dd8c05
Parents: 68ffca0 3587ac6
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jan 23 21:16:20 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jan 23 21:16:20 2017 +0300

----------------------------------------------------------------------
 .../GridNearAtomicSingleUpdateFuture.java       |  2 +-
 ...eContinuousQueryAsyncFilterListenerTest.java | 10 ++++----
 .../GridServiceProcessorProxySelfTest.java      | 24 +++++++++++++++++---
 3 files changed, 28 insertions(+), 8 deletions(-)
----------------------------------------------------------------------



[2/5] ignite git commit: Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x

Posted by ag...@apache.org.
Merge branch 'ignite-db-x' of https://github.com/gridgain/apache-ignite into ignite-db-x


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

Branch: refs/heads/ignite-3477
Commit: 68ffca03e5457ed00013903a0083de03a11254e7
Parents: eda245b 26000ca
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Mon Jan 23 19:12:56 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Mon Jan 23 19:12:56 2017 +0300

----------------------------------------------------------------------
 .../atomic/GridNearAtomicSingleUpdateFuture.java  |  2 +-
 .../dht/preloader/GridDhtPartitionDemander.java   | 18 +++++-------------
 .../cache/transactions/IgniteTxHandler.java       |  9 ++++++---
 .../processors/query/h2/database/H2TreeIndex.java |  2 +-
 4 files changed, 13 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/68ffca03/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/database/H2TreeIndex.java
----------------------------------------------------------------------


[4/5] ignite git commit: ignite-db-x fix testLocalProxyInvocation

Posted by ag...@apache.org.
ignite-db-x fix testLocalProxyInvocation


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

Branch: refs/heads/ignite-3477
Commit: 3587ac673658fffaadaa19fea16a575dfb077d30
Parents: 742aa94
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Mon Jan 23 20:35:02 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Mon Jan 23 20:35:02 2017 +0300

----------------------------------------------------------------------
 ...eContinuousQueryAsyncFilterListenerTest.java | 10 ++++----
 .../GridServiceProcessorProxySelfTest.java      | 24 +++++++++++++++++---
 2 files changed, 27 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/3587ac67/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
index 0605bc8..47e96b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryAsyncFilterListenerTest.java
@@ -445,7 +445,8 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
 
                 int nodeIdx = i % NODES;
 
-                final IgniteCache cache = grid(nodeIdx).cache(ccfg.getName());
+                final String cacheName = ccfg.getName();
+                final IgniteCache cache = grid(nodeIdx).cache(cacheName);
 
                 final QueryTestKey key = NODES - 1 != nodeIdx ? affinityKey(cache) : new QueryTestKey(1);
 
@@ -473,7 +474,7 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
                     new IgniteBiInClosure<Ignite, CacheEntryEvent<? extends QueryTestKey, ? extends QueryTestValue>>() {
                         @Override public void apply(Ignite ignite, CacheEntryEvent<? extends QueryTestKey,
                             ? extends QueryTestValue> e) {
-                            IgniteCache<Object, Object> cache0 = ignite.cache(cache.getName());
+                            IgniteCache<Object, Object> cache0 = ignite.cache(cacheName);
 
                             QueryTestValue val = e.getValue();
 
@@ -600,7 +601,8 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
 
                 int nodeIdx = i % NODES;
 
-                final IgniteCache cache = grid(nodeIdx).cache(ccfg.getName());
+                final String cacheName = ccfg.getName();
+                final IgniteCache cache = grid(nodeIdx).cache(cacheName);
 
                 final QueryTestKey key = NODES - 1 != nodeIdx ? affinityKey(cache) : new QueryTestKey(1);
 
@@ -622,7 +624,7 @@ public class CacheContinuousQueryAsyncFilterListenerTest extends GridCommonAbstr
                                     Thread.currentThread().getName().contains("callback-"));
                             }
 
-                            IgniteCache<Object, Object> cache0 = ignite.cache(cache.getName());
+                            IgniteCache<Object, Object> cache0 = ignite.cache(cacheName);
 
                             QueryTestValue val = e.getValue();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/3587ac67/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
index 7b5abf5..9fd2d2c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
@@ -19,11 +19,14 @@ package org.apache.ignite.internal.processors.service;
 
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicReference;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.util.typedef.PA;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.services.Service;
 import org.apache.ignite.services.ServiceContext;
+import org.apache.ignite.testframework.GridTestUtils;
 
 /**
  * Service proxy test.
@@ -212,12 +215,27 @@ public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstr
         ignite.services().deployNodeSingleton(name, new MapServiceImpl<String, Integer>());
 
         for (int i = 0; i < nodeCount(); i++) {
-            MapService<Integer, String> svc =  grid(i).services().serviceProxy(name, MapService.class, false);
+            final int idx = i;
+
+            final AtomicReference< MapService<Integer, String>> ref = new AtomicReference<>();
+
+            //wait because after deployNodeSingleton we don't have guarantees what service was deploy.
+            boolean wait = GridTestUtils.waitForCondition(new PA() {
+                @Override public boolean apply() {
+                    MapService<Integer, String> svc = grid(idx)
+                        .services()
+                        .serviceProxy(name, MapService.class, false);
+
+                    ref.set(svc);
+
+                    return svc instanceof Service;
+                }
+            }, 2000);
 
             // Make sure service is a local instance.
-            assertTrue("Invalid service instance [srv=" + svc + ", node=" + i + ']', svc instanceof Service);
+            assertTrue("Invalid service instance [srv=" + ref.get() + ", node=" + i + ']', wait);
 
-            svc.put(i, Integer.toString(i));
+            ref.get().put(i, Integer.toString(i));
         }
 
         MapService<Integer, String> map = ignite.services().serviceProxy(name, MapService.class, false);