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/12/14 14:22:00 UTC

ignite git commit: ignite-1905 Added test reproducing issue (IgniteStartCacheInTransactionSelfTest).

Repository: ignite
Updated Branches:
  refs/heads/ignite-1.5 87bddf40c -> a50e963b5


ignite-1905 Added test reproducing issue (IgniteStartCacheInTransactionSelfTest).


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

Branch: refs/heads/ignite-1.5
Commit: a50e963b5bc287d8d14753a1ba2f293645381b8c
Parents: 87bddf4
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 14 16:21:47 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 14 16:21:47 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/gridify/AbstractAopTest.java  |  32 ++---
 .../IgniteStartCacheInTransactionSelfTest.java  |   8 ++
 ...gniteTopologyValidatorAbstractCacheTest.java |  18 +--
 ...iteTopologyValidatorAbstractTxCacheTest.java |   1 -
 .../CacheTryLockMultithreadedTest.java          | 116 +++++++++++++++++++
 5 files changed, 152 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/a50e963b/modules/aop/src/test/java/org/apache/ignite/gridify/AbstractAopTest.java
----------------------------------------------------------------------
diff --git a/modules/aop/src/test/java/org/apache/ignite/gridify/AbstractAopTest.java b/modules/aop/src/test/java/org/apache/ignite/gridify/AbstractAopTest.java
index 4f0e2ea..c3f7b9b 100644
--- a/modules/aop/src/test/java/org/apache/ignite/gridify/AbstractAopTest.java
+++ b/modules/aop/src/test/java/org/apache/ignite/gridify/AbstractAopTest.java
@@ -29,6 +29,8 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.spi.deployment.local.LocalDeploymentSpi;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestClassLoader;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
@@ -41,8 +43,25 @@ import static org.apache.ignite.events.EventType.EVT_TASK_DEPLOYED;
 @SuppressWarnings( {"OverlyStrongTypeCast", "JUnitAbstractTestClassNamingConvention", "ProhibitedExceptionDeclared", "IfMayBeConditional"})
 public abstract class AbstractAopTest extends GridCommonAbstractTest {
     /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
     private DeploymentMode depMode = DeploymentMode.PRIVATE;
 
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDeploymentSpi(new LocalDeploymentSpi());
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setHeartbeatFrequency(500);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setDeploymentMode(depMode);
+
+        return cfg;
+    }
+
     /**
      * @throws Exception If test failed.
      */
@@ -676,19 +695,6 @@ public abstract class AbstractAopTest extends GridCommonAbstractTest {
         info("Executed @Gridify method gridifyNonDefaultNameResource(4) [result=" + res + ']');
     }
 
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        cfg.setDeploymentSpi(new LocalDeploymentSpi());
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setHeartbeatFrequency(500);
-
-        cfg.setDeploymentMode(depMode);
-
-        return cfg;
-    }
-
     /**
      * @return Test target.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a50e963b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
index 68a4da3..1e10a03 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteStartCacheInTransactionSelfTest.java
@@ -25,6 +25,9 @@ import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.configuration.NearCacheConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.transactions.Transaction;
@@ -37,10 +40,15 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  * Check starting cache in transaction.
  */
 public class IgniteStartCacheInTransactionSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
         CacheConfiguration ccfg = new CacheConfiguration();
 
         ccfg.setAtomicityMode(atomicityMode());

http://git-wip-us.apache.org/repos/asf/ignite/blob/a50e963b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
index 65f4694..7ac68db 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractCacheTest.java
@@ -33,7 +33,7 @@ import org.apache.ignite.transactions.Transaction;
  */
 public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCacheAbstractTest implements Serializable {
     /** key-value used at test. */
-    protected static String KEY_VALUE = "1";
+    protected static String KEY_VAL = "1";
 
     /** cache name 1. */
     protected static String CACHE_NAME_1 = "cache1";
@@ -86,7 +86,7 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void putInvalid(String cacheName) {
         try {
-            grid(0).cache(cacheName).put(KEY_VALUE, KEY_VALUE);
+            grid(0).cache(cacheName).put(KEY_VAL, KEY_VAL);
 
             assert false : "topology validation broken";
         }
@@ -103,9 +103,9 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void putValid(String cacheName) {
         try {
-            grid(0).cache(cacheName).put(KEY_VALUE, KEY_VALUE);
+            grid(0).cache(cacheName).put(KEY_VAL, KEY_VAL);
 
-            assert grid(0).cache(cacheName).get(KEY_VALUE).equals(KEY_VALUE);
+            assert grid(0).cache(cacheName).get(KEY_VAL).equals(KEY_VAL);
         }
         catch (CacheException ex) {
             assert false : "topology validation broken";
@@ -119,7 +119,7 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void getInvalid(String cacheName) {
         try {
-            assert grid(0).cache(cacheName).get(KEY_VALUE).equals(KEY_VALUE);
+            assert grid(0).cache(cacheName).get(KEY_VAL).equals(KEY_VAL);
         }
         catch (CacheException ex) {
             assert false : "topology validation broken";
@@ -133,7 +133,7 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      */
     protected void removeInvalid(String cacheName) {
         try {
-            grid(0).cache(cacheName).remove(KEY_VALUE);
+            grid(0).cache(cacheName).remove(KEY_VAL);
 
             assert false : "topology validation broken";
         }
@@ -164,9 +164,9 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      * @param cacheName cache name.
      */
     public void remove(String cacheName) {
-        assert grid(0).cache(cacheName).get(KEY_VALUE) != null;
+        assert grid(0).cache(cacheName).get(KEY_VAL) != null;
 
-        grid(0).cache(cacheName).remove(KEY_VALUE);
+        grid(0).cache(cacheName).remove(KEY_VAL);
     }
 
     /**
@@ -175,7 +175,7 @@ public abstract class IgniteTopologyValidatorAbstractCacheTest extends IgniteCac
      * @param cacheName cache name.
      */
     public void assertEmpty(String cacheName) {
-        assert grid(0).cache(cacheName).get(KEY_VALUE) == null;
+        assert grid(0).cache(cacheName).get(KEY_VAL) == null;
     }
 
     /** topology validator test. */

http://git-wip-us.apache.org/repos/asf/ignite/blob/a50e963b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java
index e3e8fec..fd386bb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTopologyValidatorAbstractTxCacheTest.java
@@ -29,7 +29,6 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
  * Topology validator test
  */
 public abstract class IgniteTopologyValidatorAbstractTxCacheTest extends IgniteTopologyValidatorAbstractCacheTest {
-
     /** {@inheritDoc} */
     @Override protected CacheAtomicityMode atomicityMode() {
         return TRANSACTIONAL;

http://git-wip-us.apache.org/repos/asf/ignite/blob/a50e963b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheTryLockMultithreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheTryLockMultithreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheTryLockMultithreadedTest.java
new file mode 100644
index 0000000..11e0d51
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheTryLockMultithreadedTest.java
@@ -0,0 +1,116 @@
+/*
+ * 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;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.locks.Lock;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static java.util.concurrent.TimeUnit.MILLISECONDS;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class CacheTryLockMultithreadedTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int SRVS = 2;
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration<Integer, Integer> ccfg = new CacheConfiguration<>();
+
+        ccfg.setAtomicityMode(TRANSACTIONAL);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setCacheMode(REPLICATED);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(SRVS);
+
+        client = true;
+
+        startGrid(SRVS);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTryLock() throws Exception {
+        Ignite client = grid(SRVS);
+
+        final Integer key = 1;
+
+        final IgniteCache<Integer, Integer> cache = client.cache(null);
+
+        final long stopTime = System.currentTimeMillis() + 30_000;
+
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Lock lock = cache.lock(key);
+
+                while (System.currentTimeMillis() < stopTime) {
+                    for (int i = 0; i < 1000; i++) {
+                        boolean locked = lock.tryLock(100, MILLISECONDS);
+
+                        if (locked)
+                            lock.unlock();
+                    }
+                }
+
+                return null;
+            }
+        }, 20, "lock-thread");
+    }
+}