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/05/13 17:43:21 UTC

incubator-ignite git commit: #ignite-373: RemoveAll does not check owning for None rebalance mode.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-373 e371d620d -> 21b7598f4


#ignite-373: RemoveAll does not check owning for None rebalance mode.


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

Branch: refs/heads/ignite-373
Commit: 21b7598f4257bb73e8322e89e95681caefae10fd
Parents: e371d62
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed May 13 18:43:05 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed May 13 18:43:05 2015 +0300

----------------------------------------------------------------------
 .../GridDistributedCacheAdapter.java            |  2 +-
 .../near/NoneRebalanceModeSelfTest.java         | 65 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  1 +
 3 files changed, 67 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21b7598f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 56f8e1f..c5ef22f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -362,7 +362,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
                     for (int part : ctx.affinity().primaryPartitions(ctx.localNodeId(), topVer)) {
                         GridDhtLocalPartition locPart = dht.topology().localPartition(part, topVer, false);
 
-                        if (locPart == null || locPart.state() != OWNING || !locPart.reserve())
+                        if (locPart == null || (ctx.rebalanceEnabled() && locPart.state() != OWNING) || !locPart.reserve())
                             return false;
 
                         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21b7598f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
new file mode 100644
index 0000000..8a7fd5f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/NoneRebalanceModeSelfTest.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ * Test none rebalance mode.
+ */
+public class NoneRebalanceModeSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @SuppressWarnings({"ConstantConditions"})
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration c = super.getConfiguration(gridName);
+
+        CacheConfiguration cc = defaultCacheConfiguration();
+
+        cc.setRebalanceMode(NONE);
+
+        c.setCacheConfiguration(cc);
+
+        return c;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid(0);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRemoveAll() throws Exception {
+        GridNearTransactionalCache cache = (GridNearTransactionalCache)((IgniteKernal)grid(0)).internalCache(null);
+
+        assert cache.dht().topology().localPartitions().get(0).state() == GridDhtPartitionState.MOVING;
+
+        grid(0).cache(null).removeAll();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/21b7598f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index dc3a2c0..5738778 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -58,6 +58,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTestSuite(GridCachePartitionedGetSelfTest.class);
         suite.addTest(new TestSuite(GridCachePartitionedBasicApiTest.class));
         suite.addTest(new TestSuite(GridCacheNearMultiGetSelfTest.class));
+        suite.addTest(new TestSuite(NoneRebalanceModeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearJobExecutionSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearOneNodeSelfTest.class));
         suite.addTest(new TestSuite(GridCacheNearMultiNodeSelfTest.class));