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/27 16:49:28 UTC

[37/38] incubator-ignite git commit: # ignite-23

# ignite-23


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

Branch: refs/heads/ignite-709_2
Commit: cf6c420ec30eb08cae9568e06b999ad8356f58b5
Parents: e35e4ae
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 27 14:57:18 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 27 15:59:01 2015 +0300

----------------------------------------------------------------------
 .../dht/GridDhtAssignmentFetchFuture.java       |  4 +-
 .../GridDhtPartitionsExchangeFuture.java        |  5 +-
 ...niteCacheClientNodeChangingTopologyTest.java | 75 ++++++++++++++------
 .../dht/IgniteCacheMultiTxLockSelfTest.java     | 47 ++++++++++--
 4 files changed, 100 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6c420e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
index 303d649..7bae7f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtAssignmentFetchFuture.java
@@ -96,12 +96,12 @@ public class GridDhtAssignmentFetchFuture extends GridFutureAdapter<List<List<Cl
 
     /**
      * @param node Node.
-     * @param res Reponse.
+     * @param res Response.
      */
     public void onResponse(ClusterNode node, GridDhtAffinityAssignmentResponse res) {
         if (!res.topologyVersion().equals(topVer)) {
             if (log.isDebugEnabled())
-                log.debug("Received affinity assignment for wrong topolgy version (will ignore) " +
+                log.debug("Received affinity assignment for wrong topology version (will ignore) " +
                     "[node=" + node + ", res=" + res + ", topVer=" + topVer + ']');
 
             return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6c420e/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 47bc1a3..162c7b3 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
@@ -345,8 +345,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     exchId + ']');
 
             // Fetch affinity assignment from remote node.
-            GridDhtAssignmentFetchFuture fetchFut =
-                new GridDhtAssignmentFetchFuture(cacheCtx, exchId.topologyVersion(), CU.affinityNodes(cacheCtx));
+            GridDhtAssignmentFetchFuture fetchFut = new GridDhtAssignmentFetchFuture(cacheCtx,
+                exchId.topologyVersion(),
+                CU.affinityNodes(cacheCtx, exchId.topologyVersion()));
 
             fetchFut.init();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6c420e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 45fa275..e13b283 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -1344,36 +1344,43 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
      * @throws Exception If failed.
      */
     public void testAtomicPrimaryPutAllMultinode() throws Exception {
-        putAllMultinode(PRIMARY, false);
+        multinode(PRIMARY, TestType.PUT_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testAtomicClockPutAllMultinode() throws Exception {
-        putAllMultinode(CLOCK ,false);
+        multinode(CLOCK, TestType.PUT_ALL);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testOptimisticTxPutAllMultinode() throws Exception {
-        putAllMultinode(null, false);
+        multinode(null, TestType.OPTIMISTIC_TX);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testPessimisticTxPutAllMultinode() throws Exception {
-        putAllMultinode(null, true);
+        multinode(null, TestType.PESSIMISTIC_TX);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLockAllMultinode() throws Exception {
+        multinode(null, TestType.LOCK);
     }
 
     /**
      * @param atomicWriteOrder Write order if test atomic cache.
-     * @param pessimisticTx {@code True} if use pessimistic tx.
+     * @param testType Test type.
      * @throws Exception If failed.
      */
-    private void putAllMultinode(final CacheAtomicWriteOrderMode atomicWriteOrder, final boolean pessimisticTx)
+    private void multinode(final CacheAtomicWriteOrderMode atomicWriteOrder, final TestType testType)
         throws Exception {
         ccfg = new CacheConfiguration();
 
@@ -1426,9 +1433,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
                     IgniteCache<Integer, Integer> cache = ignite.cache(null);
 
-                    boolean useTx = atomicWriteOrder == null;
+                    boolean useTx = testType == TestType.OPTIMISTIC_TX || testType == TestType.PESSIMISTIC_TX;
 
-                    if (useTx) {
+                    if (useTx || testType == TestType.LOCK) {
                         assertEquals(TRANSACTIONAL,
                             cache.getConfiguration(CacheConfiguration.class).getAtomicityMode());
                     }
@@ -1447,24 +1454,34 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                         }
 
                         try {
-                            if (useTx) {
-                                IgniteTransactions txs = ignite.transactions();
+                            if (testType == TestType.LOCK) {
+                                Lock lock = cache.lockAll(map.keySet());
 
-                                TransactionConcurrency concurrency = pessimisticTx ? PESSIMISTIC : OPTIMISTIC;
+                                lock.lock();
 
-                                try (Transaction tx = txs.txStart(concurrency, REPEATABLE_READ)) {
-                                    cache.putAll(map);
+                                lock.unlock();
+                            }
+                            else {
+                                if (useTx) {
+                                    IgniteTransactions txs = ignite.transactions();
 
-                                    tx.commit();
+                                    TransactionConcurrency concurrency =
+                                        testType == TestType.PESSIMISTIC_TX ? PESSIMISTIC : OPTIMISTIC;
+
+                                    try (Transaction tx = txs.txStart(concurrency, REPEATABLE_READ)) {
+                                        cache.putAll(map);
+
+                                        tx.commit();
+                                    }
                                 }
-                            }
-                            else
-                                cache.putAll(map);
+                                else
+                                    cache.putAll(map);
 
-                            putKeys.addAll(map.keySet());
+                                putKeys.addAll(map.keySet());
+                            }
                         }
                         catch (CacheException | IgniteException e) {
-                            log.info("Update failed, ignore: " + e);
+                            log.info("Operation failed, ignore: " + e);
                         }
 
                         if (++cntr % 100 == 0)
@@ -1572,7 +1589,8 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         fut.get(30_000);
 
-        checkData(null, putKeys, grid(SRV_CNT).cache(null), SRV_CNT + CLIENT_CNT);
+        if (testType != TestType.LOCK)
+            checkData(null, putKeys, grid(SRV_CNT).cache(null), SRV_CNT + CLIENT_CNT);
     }
 
     /**
@@ -1736,4 +1754,21 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
             }
         }
     }
+
+    /**
+     *
+     */
+    enum TestType {
+        /** */
+        PUT_ALL,
+
+        /** */
+        OPTIMISTIC_TX,
+
+        /** */
+        PESSIMISTIC_TX,
+
+        /** */
+        LOCK
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cf6c420e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 5983c1b..f33de76 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
@@ -33,6 +32,10 @@ import org.apache.ignite.testframework.junits.common.*;
 import java.util.*;
 import java.util.concurrent.*;
 
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
 /**
  * Tests explicit lock.
  */
@@ -46,6 +49,9 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
     /** */
     private volatile boolean run = true;
 
+    /** */
+    private boolean client;
+
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         stopAllGrids();
@@ -66,16 +72,18 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setName(CACHE_NAME);
-        ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
-        ccfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.PRIMARY_SYNC);
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
         ccfg.setBackups(2);
-        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setCacheMode(PARTITIONED);
         ccfg.setStartSize(100000);
         ccfg.setEvictionPolicy(new LruEvictionPolicy(100000));
         ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);
 
+        c.setClientMode(client);
+
         return c;
     }
 
@@ -83,33 +91,50 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testExplicitLockOneKey() throws Exception {
-        checkExplicitLock(1);
+        checkExplicitLock(1, false);
     }
 
     /**
      * @throws Exception If failed.
      */
     public void testExplicitLockManyKeys() throws Exception {
-        checkExplicitLock(4);
+        checkExplicitLock(4, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitLockManyKeysWithClient() throws Exception {
+        checkExplicitLock(4, true);
     }
 
     /**
+     * @param keys Number of keys.
+     * @param testClient If {@code true} uses one client node.
      * @throws Exception If failed.
      */
-    public void checkExplicitLock(int keys) throws Exception {
+    public void checkExplicitLock(int keys, boolean testClient) throws Exception {
         Collection<Thread> threads = new ArrayList<>();
 
         try {
             // Start grid 1.
             IgniteEx grid1 = startGrid(1);
 
+            assertFalse(grid1.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid1.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
 
+            client = testClient; // If test client start on node in client mode.
+
             // Start grid 2.
             IgniteEx grid2 = startGrid(2);
 
+            assertEquals((Object)testClient, grid2.configuration().isClientMode());
+
+            client = false;
+
             threads.add(runCacheOperations(grid2.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -117,6 +142,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 3.
             IgniteEx grid3 = startGrid(3);
 
+            assertFalse(grid3.configuration().isClientMode());
+
+            if (testClient)
+                log.info("Started client node: " + grid3.name());
+
             threads.add(runCacheOperations(grid3.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -124,6 +154,8 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
             // Start grid 4.
             IgniteEx grid4 = startGrid(4);
 
+            assertFalse(grid4.configuration().isClientMode());
+
             threads.add(runCacheOperations(grid4.cachex(CACHE_NAME), keys));
 
             TimeUnit.SECONDS.sleep(3L);
@@ -158,6 +190,7 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
 
     /**
      * @param cache Cache.
+     * @param keys Number of keys.
      * @return Running thread.
      */
     @SuppressWarnings("TypeMayBeWeakened")