You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2015/07/16 10:00:08 UTC

incubator-ignite git commit: # ignite-1124

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-1124 [created] 75eb33bba


# ignite-1124


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

Branch: refs/heads/ignite-1124
Commit: 75eb33bba990abe4220deb4070fad759955b47a7
Parents: 77da728
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jul 16 10:34:02 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jul 16 10:34:02 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      | 30 ++++++++++++++------
 .../distributed/dht/GridDhtTopologyFuture.java  | 14 ---------
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  2 +-
 .../GridDhtPartitionsExchangeFuture.java        | 11 -------
 ...acheAtomicReplicatedNodeRestartSelfTest.java |  5 ----
 5 files changed, 22 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75eb33bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index aec08c6..d13845a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -47,6 +47,7 @@ import org.apache.ignite.internal.processors.offheap.*;
 import org.apache.ignite.internal.processors.plugin.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.unsafe.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -765,26 +766,37 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @return Partition topology.
      */
     public GridDhtPartitionTopology topology() {
-        assert isNear() || isDht() || isColocated() || isDhtAtomic() : cache;
+        GridCacheAdapter<K, V> cache = this.cache;
 
-        return isNear() ? near().dht().topology() : dht().topology();
+        if (cache == null)
+            throw new IllegalStateException("Cache stopped: " + cacheName);
+
+        assert cache.isNear() || cache.isDht() || cache.isColocated() || cache.isDhtAtomic() : cache;
+
+        return topology(cache);
     }
 
     /**
      * @return Topology version future.
      */
     public GridDhtTopologyFuture topologyVersionFuture() {
-        assert isNear() || isDht() || isColocated() || isDhtAtomic() : cache;
+        GridCacheAdapter<K, V> cache = this.cache;
 
-        GridDhtTopologyFuture fut = null;
+        if (cache == null)
+            throw new IllegalStateException("Cache stopped: " + cacheName);
 
-        if (!isDhtAtomic()) {
-            GridDhtCacheAdapter<K, V> cache = isNear() ? near().dht() : colocated();
+        assert cache.isNear() || cache.isDht() || cache.isColocated() || cache.isDhtAtomic() : cache;
 
-            fut = cache.multiUpdateTopologyFuture();
-        }
+        return topology(cache).topologyVersionFuture();
+    }
 
-        return fut == null ? topology().topologyVersionFuture() : fut;
+    /**
+     * @param cache Cache.
+     * @return Partition topology.
+     */
+    private GridDhtPartitionTopology topology(GridCacheAdapter<K, V> cache) {
+        return cache.isNear() ? ((GridNearCacheAdapter<K, V>)cache).dht().topology() :
+            ((GridDhtCacheAdapter<K, V>)cache).topology();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75eb33bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
index 8a02ff2..57e3e33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
@@ -17,9 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 
@@ -33,18 +31,6 @@ import org.apache.ignite.internal.processors.cache.*;
  */
 public interface GridDhtTopologyFuture extends IgniteInternalFuture<AffinityTopologyVersion> {
     /**
-     * Gets a topology snapshot for the topology version represented by the future. Note that by the time
-     * partition exchange completes some nodes from the snapshot may leave the grid. One should use discovery
-     * service to check if the node is valid.
-     * <p/>
-     * This method will block until the topology future is ready.
-     *
-     * @return Topology snapshot for particular topology version.
-     * @throws IgniteCheckedException If topology future failed.
-     */
-    public GridDiscoveryTopologySnapshot topologySnapshot() throws IgniteCheckedException;
-
-    /**
      * Gets topology version of this future.
      *
      * @return Topology version.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75eb33bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 751c9ba..b855a1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -582,7 +582,7 @@ public class GridNearAtomicUpdateFuture extends GridFutureAdapter<Object>
                 return;
             }
 
-            GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
+            GridDhtTopologyFuture fut = cache.topology().topologyVersionFuture();
 
             if (fut.isDone()) {
                 if (!fut.isCacheTopologyValid(cctx)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75eb33bb/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 5701749..7d36768 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -244,17 +244,6 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /** {@inheritDoc} */
-    @Override public GridDiscoveryTopologySnapshot topologySnapshot() throws IgniteCheckedException {
-        get();
-
-        if (topSnapshot.get() == null)
-            topSnapshot.compareAndSet(null, new GridDiscoveryTopologySnapshot(discoEvt.topologyVersion(),
-                discoEvt.topologyNodes()));
-
-        return topSnapshot.get();
-    }
-
-    /** {@inheritDoc} */
     @Override public AffinityTopologyVersion topologyVersion() {
         return exchId.topologyVersion();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/75eb33bb/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
index 68c7fbb..f556023 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/IgniteCacheAtomicReplicatedNodeRestartSelfTest.java
@@ -26,11 +26,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.*;
  */
 public class IgniteCacheAtomicReplicatedNodeRestartSelfTest extends GridCacheReplicatedNodeRestartSelfTest {
     /** {@inheritDoc} */
-    @Override protected void beforeTestsStarted() throws Exception {
-        fail("https://issues.apache.org/jira/browse/IGNITE-1124");
-    }
-
-    /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return ATOMIC;
     }