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/26 18:28:45 UTC

[1/7] ignite git commit: ignite-db-x fix DynamicProxySerializationMultiJvmSelfTest

Repository: ignite
Updated Branches:
  refs/heads/ignite-3477 788d9bb74 -> ad061021b


ignite-db-x fix DynamicProxySerializationMultiJvmSelfTest


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

Branch: refs/heads/ignite-3477
Commit: cce91f1fcb09aefe2e11cfde9f4ab3efcd8eb60a
Parents: 3587ac6
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Tue Jan 24 15:55:16 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Tue Jan 24 15:55:16 2017 +0300

----------------------------------------------------------------------
 .../deployment/GridDeploymentMessageCountSelfTest.java       | 8 ++++++++
 1 file changed, 8 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/cce91f1f/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
index b02dce0..fdcb67a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.compute.ComputeTaskFuture;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.spi.IgniteSpiException;
@@ -57,6 +58,13 @@ public class GridDeploymentMessageCountSelfTest extends GridCommonAbstractTest {
     private Map<String, MessageCountingCommunicationSpi> commSpis = new ConcurrentHashMap8<>();
 
     /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        assert G.allGrids().isEmpty();
+    }
+
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 


[5/7] ignite git commit: ignite-db-x fix assign own partition state

Posted by ag...@apache.org.
ignite-db-x fix assign own partition state


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

Branch: refs/heads/ignite-3477
Commit: 4b1d21faee58865fe1a6e9f3db62cddbfeed17da
Parents: 196c0ac
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Thu Jan 26 21:23:34 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Thu Jan 26 21:23:34 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 40 +++++++-------------
 1 file changed, 14 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/4b1d21fa/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index f83f93a..5be1d86 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -1065,9 +1065,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public GridDhtPartitionMap2 update(
+        @Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionFullMap partMap,
-        @Nullable Map<Integer, T2<Long, Long>> cntrMap) {
+        @Nullable Map<Integer, T2<Long, Long>> cntrMap
+    ) {
         if (log.isDebugEnabled())
             log.debug("Updating full partition map [exchId=" + exchId + ", parts=" + fullMapString() + ']');
 
@@ -1080,6 +1082,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 return null;
 
             if (cntrMap != null) {
+                // update local map partition counters
                 for (Map.Entry<Integer, T2<Long, Long>> e : cntrMap.entrySet()) {
                     T2<Long, Long> cntr = this.cntrMap.get(e.getKey());
 
@@ -1087,6 +1090,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         this.cntrMap.put(e.getKey(), e.getValue());
                 }
 
+                // update local counters in partitions
                 for (int i = 0; i < locParts.length(); i++) {
                     GridDhtLocalPartition part = locParts.get(i);
 
@@ -1100,6 +1104,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 }
             }
 
+            //if need skip
             if (exchId != null && lastExchangeId != null && lastExchangeId.compareTo(exchId) >= 0) {
                 if (log.isDebugEnabled())
                     log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" +
@@ -1141,6 +1146,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     }
                 }
 
+                //remove entry if node left
                 for (Iterator<UUID> it = partMap.keySet().iterator(); it.hasNext(); ) {
                     UUID nodeId = it.next();
 
@@ -1179,32 +1185,12 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             GridDhtPartitionMap2 nodeMap = partMap.get(cctx.localNodeId());
 
-            if (nodeMap != null) {
+            if (nodeMap != null && cctx.shared().database().persistenceEnabled()) {
                 for (Map.Entry<Integer, GridDhtPartitionState> e : nodeMap.entrySet()) {
                     int p = e.getKey();
                     GridDhtPartitionState state = e.getValue();
 
-                    if (state == OWNING) {
-                        GridDhtLocalPartition locPart = locParts.get(p);
-
-                        assert locPart != null;
-
-                        if (cntrMap != null) {
-                            T2<Long, Long> cntr = cntrMap.get(p);
-
-                            if (cntr != null && cntr.get2() > locPart.updateCounter())
-                                locPart.updateCounter(cntr.get2());
-                        }
-
-                        if (locPart.state() != OWNING) {
-                            boolean success = locPart.own();
-
-                            assert success : locPart;
-
-                            changed |= success;
-                        }
-                    }
-                    else if (state == MOVING) {
+                   if (state == MOVING) {
                         GridDhtLocalPartition locPart = locParts.get(p);
 
                         assert locPart != null;
@@ -1250,9 +1236,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public GridDhtPartitionMap2 update(
+        @Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionMap2 parts,
-        @Nullable Map<Integer, T2<Long, Long>> cntrMap) {
+        @Nullable Map<Integer, T2<Long, Long>> cntrMap
+    ) {
         if (log.isDebugEnabled())
             log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
 


[2/7] ignite git commit: ignite-db-x fix GridCacheDhtPreloadDisabledSelfTest.testDisabledPreloader

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


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

Branch: refs/heads/ignite-3477
Commit: 0cc13dab42743a5a24f8347586f8f796bdfac021
Parents: cce91f1
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Wed Jan 25 15:14:29 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Wed Jan 25 15:14:29 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/service/GridServiceProcessor.java | 4 ++++
 .../distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java     | 3 +--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0cc13dab/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index ab04e7e..914c3a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1593,6 +1593,10 @@ public class GridServiceProcessor extends GridProcessorAdapter implements Ignite
             if (!busyLock.enterBusy())
                 return;
 
+            //Must check that threadpool was not shutdown.
+            if (depExe.isShutdown())
+                return;
+
             try {
                 final AffinityTopologyVersion topVer;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/0cc13dab/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
index 61c7d37..f26eb60 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDisabledSelfTest.java
@@ -25,7 +25,6 @@ import java.util.List;
 import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
-import org.apache.ignite.cache.CachePeekMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
 import org.apache.ignite.cluster.ClusterNode;
@@ -186,7 +185,7 @@ public class GridCacheDhtPreloadDisabledSelfTest extends GridCommonAbstractTest
 
             for (int i = 0; i < keyCnt; i++) {
                 assertNull(near(cache1).peekEx(i));
-                assertNotNull((dht(cache1).peekEx(i)));
+                assertNotNull((dht(cache1).localPeek(i, null, null)));
 
                 assertEquals(Integer.toString(i), cache1.localPeek(i));
             }


[4/7] ignite git commit: ignite-db-x mute unswap test

Posted by ag...@apache.org.
ignite-db-x mute unswap test


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

Branch: refs/heads/ignite-3477
Commit: 196c0ac371f8e6d40dee15e6906372649522c852
Parents: a7f1ef4
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Thu Jan 26 12:39:09 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Thu Jan 26 12:39:09 2017 +0300

----------------------------------------------------------------------
 .../replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java     | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/196c0ac3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
index 006cb87..f820593 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedUnswapAdvancedSelfTest.java
@@ -69,8 +69,7 @@ public class GridCacheReplicatedUnswapAdvancedSelfTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testUnswapAdvanced() throws Exception {
-        //TODO GG-11141
-        fail();
+        fail("https://issues.apache.org/jira/browse/IGNITE-4551");
 
         Ignite g1 = startGrid(1);
         Ignite g2 = startGrid(2);


[3/7] ignite git commit: ignite-db-x mute p2p test

Posted by ag...@apache.org.
ignite-db-x mute p2p test


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

Branch: refs/heads/ignite-3477
Commit: a7f1ef4617dea79584d25efbfda61ba078cd085a
Parents: 0cc13da
Author: Dmitriy Govorukhin <dg...@gridgain.com>
Authored: Thu Jan 26 12:28:07 2017 +0300
Committer: Dmitriy Govorukhin <dg...@gridgain.com>
Committed: Thu Jan 26 12:28:07 2017 +0300

----------------------------------------------------------------------
 .../managers/deployment/GridDeploymentMessageCountSelfTest.java   | 3 +--
 1 file changed, 1 insertion(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a7f1ef46/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
index fdcb67a..759179f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/deployment/GridDeploymentMessageCountSelfTest.java
@@ -134,8 +134,7 @@ public class GridDeploymentMessageCountSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCacheValueDeploymentOnPut() throws Exception {
-        //TODO GG-11141
-        fail();
+        fail("https://issues.apache.org/jira/browse/IGNITE-4551");
 
         ClassLoader ldr = getExternalClassLoader();
 


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

Branch: refs/heads/ignite-3477
Commit: b7d24dc274f9af8ee8c17cf0ad44dc467f198c93
Parents: c875221 4b1d21f
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jan 26 21:25:44 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 26 21:25:44 2017 +0300

----------------------------------------------------------------------
 .../dht/GridDhtPartitionTopologyImpl.java       | 40 +++++++-------------
 .../service/GridServiceProcessor.java           |  4 ++
 .../GridDeploymentMessageCountSelfTest.java     | 11 +++++-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |  3 +-
 ...idCacheReplicatedUnswapAdvancedSelfTest.java |  3 +-
 5 files changed, 29 insertions(+), 32 deletions(-)
----------------------------------------------------------------------



[7/7] ignite git commit: Merge branch 'ignite-3477' of https://git-wip-us.apache.org/repos/asf/ignite into ignite-db-x

Posted by ag...@apache.org.
Merge branch 'ignite-3477' of https://git-wip-us.apache.org/repos/asf/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/ad061021
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/ad061021
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/ad061021

Branch: refs/heads/ignite-3477
Commit: ad061021be40569de7402bf1f480ab6bc2fe1ea5
Parents: b7d24dc 788d9bb
Author: Alexey Goncharuk <al...@gmail.com>
Authored: Thu Jan 26 21:26:25 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Jan 26 21:26:25 2017 +0300

----------------------------------------------------------------------
 .../ignite/internal/pagemem/PageMemory.java     |  13 +-
 .../pagemem/impl/PageMemoryNoStoreImpl.java     |  13 +-
 .../internal/pagemem/impl/PageNoStoreImpl.java  |   2 +
 .../internal/pagemem/wal/record/WALRecord.java  |   5 +-
 .../wal/record/delta/DataPageUpdateRecord.java  |  79 +++++++++
 .../processors/cache/CacheLazyEntry.java        |   2 +-
 .../processors/cache/GridCacheMapEntry.java     |  64 +++++---
 .../cache/IgniteCacheOffheapManager.java        |   8 +-
 .../cache/IgniteCacheOffheapManagerImpl.java    | 160 +++++++++++++++----
 .../cache/database/CacheDataRowAdapter.java     |  70 +++++---
 .../processors/cache/database/RowStore.java     |  12 ++
 .../cache/database/freelist/FreeList.java       |   8 +
 .../cache/database/freelist/FreeListImpl.java   |  69 ++++++--
 .../cache/database/tree/BPlusTree.java          |  44 ++++-
 .../cache/database/tree/io/DataPageIO.java      |  57 ++++++-
 .../cache/database/tree/io/IOVersions.java      |   7 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   4 +-
 .../distributed/dht/GridDhtLocalPartition.java  |  14 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   9 +-
 .../distributed/near/GridNearCacheEntry.java    |   5 +-
 .../apache/ignite/internal/util/GridUnsafe.java |   3 +
 .../database/IgniteDbPutGetAbstractTest.java    |  77 +++++++++
 .../query/h2/database/H2RowFactory.java         |   2 +-
 23 files changed, 586 insertions(+), 141 deletions(-)
----------------------------------------------------------------------