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 2016/01/13 07:48:34 UTC

[3/6] ignite git commit: Fixes: - allow 'committing' -> 'marked_rollback' tx state change only for thread committing transaction - fixed 'full_sync' mode for case when tx primary nodes fail - fixed race between statically configured cache start and Gr

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index d07a1e6..34872c6 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -85,7 +85,6 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
     }
 
     /** */
-    @SuppressWarnings({"deprecation"})
     private class TestListener implements CommunicationListener<Message> {
         /** */
         private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
index 7521f2e..b7c0deb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
@@ -49,11 +49,13 @@ import org.apache.ignite.testframework.GridTestNode;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.IgniteTestResources;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractTest;
+import org.apache.ignite.testframework.junits.spi.GridSpiTest;
 import org.eclipse.jetty.util.ConcurrentHashSet;
 
 /**
  *
  */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
 public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends CommunicationSpi>
     extends GridSpiAbstractTest<T> {
     /** */
@@ -87,7 +89,6 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
     }
 
     /** */
-    @SuppressWarnings({"deprecation"})
     private class TestListener implements CommunicationListener<Message> {
         /** */
         private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
@@ -151,6 +152,8 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
 
             int expMsgs = 0;
 
+            long totAcked = 0;
+
             for (int i = 0; i < 5; i++) {
                 info("Iteration: " + i);
 
@@ -172,6 +175,8 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
 
                 expMsgs += msgPerIter;
 
+                final long totAcked0 = totAcked;
+
                 for (TcpCommunicationSpi spi : spis) {
                     GridNioServer srv = U.field(spi, "nioSrvr");
 
@@ -189,6 +194,14 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
 
                             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                                 @Override public boolean apply() {
+                                    long acked = GridTestUtils.getFieldValue(recoveryDesc, "acked");
+
+                                    return acked > totAcked0;
+                                }
+                            }, 5000);
+
+                            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                                @Override public boolean apply() {
                                     return recoveryDesc.messagesFutures().isEmpty();
                                 }
                             }, 10_000);
@@ -218,6 +231,8 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
                 }
 
                 assertEquals(msgPerIter * 2, ackMsgs.get());
+
+                totAcked += msgPerIter;
             }
         }
         finally {
@@ -337,6 +352,8 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
                 return expMsgs == ackMsgs.get();
             }
         }, 5000);
+
+        assertEquals(expMsgs, ackMsgs.get());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 0df7da6..4f329e1 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -118,6 +118,8 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
         TcpDiscoverySpi spi = nodeSpi.get();
 
         if (spi == null) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index e0ffc60..949290e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -978,7 +978,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     /**
      * @param key Key.
      * @param cacheName Cache name.
-     * @return Ignite instance which has primary cache for given key.
+     * @return Ignite instance which has backup cache for given key.
      */
     protected Ignite backupNode(Object key, String cacheName) {
         List<Ignite> allGrids = Ignition.allGrids();
@@ -1001,8 +1001,38 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param key Key.
+     * @param cacheName Cache name.
+     * @return Ignite instances which has backup cache for given key.
+     */
+    protected List<Ignite> backupNodes(Object key, String cacheName) {
+        List<Ignite> allGrids = Ignition.allGrids();
+
+        assertFalse("There are no alive nodes.", F.isEmpty(allGrids));
+
+        Ignite ignite = allGrids.get(0);
+
+        Affinity<Object> aff = ignite.affinity(cacheName);
+
+        Collection<ClusterNode> nodes = aff.mapKeyToPrimaryAndBackups(key);
+
+        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes +']', nodes.size() > 1);
+
+        Iterator<ClusterNode> it = nodes.iterator();
+
+        it.next(); // Skip primary.
+
+        List<Ignite> backups = new ArrayList<>(nodes.size() - 1);
+
+        while (it.hasNext())
+            backups.add(grid(it.next()));
+
+        return backups;
+    }
+
+    /**
      * In ATOMIC cache with CLOCK mode if key is updated from different nodes at same time
-     * only one update wins others are ignored (can happen in test event when updates are executed from
+     * only one update wins others are ignored (can happen in test even when updates are executed from
      * different nodes sequentially), this delay is used to avoid lost updates.
      *
      * @param cache Cache.

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
index d81efd9..7363c7c 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java
@@ -21,6 +21,7 @@ import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxPessimisticOriginatingNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledTxOriginatingNodeFailureSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTxOriginatingNodeFailureSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheCommitDelayTxRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedPrimaryNodeFailureRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest;
@@ -39,6 +40,8 @@ public class IgniteCacheTxRecoverySelfTestSuite extends TestSuite {
     public static TestSuite suite() throws Exception {
         TestSuite suite = new TestSuite("Cache tx recovery test suite");
 
+        suite.addTestSuite(IgniteCacheCommitDelayTxRecoveryTest.class);
+
         suite.addTestSuite(IgniteCachePartitionedPrimaryNodeFailureRecoveryTest.class);
         suite.addTestSuite(IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest.class);
         suite.addTestSuite(IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/457a9ae4/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
----------------------------------------------------------------------
diff --git a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
index c19a8fc..e53f335 100644
--- a/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
+++ b/modules/zookeeper/src/test/java/org/apache/ignite/spi/discovery/tcp/ipfinder/zk/ZookeeperIpFinderTest.java
@@ -31,8 +31,10 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.configuration.IgniteConfiguration;
 import org.apache.ignite.events.Event;
 import org.apache.ignite.events.EventType;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 
 /**
@@ -41,7 +43,6 @@ import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
  * @author Raul Kripalani
  */
 public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
-
     /** ZK Cluster size. */
     private static final int ZK_CLUSTER_SIZE = 3;
 
@@ -79,7 +80,6 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         // start the Curator client so we can perform assertions on the ZK state later
         zkCurator = CuratorFrameworkFactory.newClient(zkCluster.getConnectString(), new RetryNTimes(10, 1000));
         zkCurator.start();
-
     }
 
     /**
@@ -98,22 +98,21 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         }
 
         stopAllGrids();
-
     }
 
     /**
      * Enhances the default configuration with the {#TcpDiscoveryZookeeperIpFinder}.
      *
      * @param gridName Grid name.
-     * @return
-     * @throws Exception
+     * @return Ignite configuration.
+     * @throws Exception If failed.
      */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration configuration = super.getConfiguration(gridName);
 
         TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi) configuration.getDiscoverySpi();
         TcpDiscoveryZookeeperIpFinder zkIpFinder = new TcpDiscoveryZookeeperIpFinder();
-        zkIpFinder.setAllowDuplicateRegistrations(isAllowDuplicateRegistrations());
+        zkIpFinder.setAllowDuplicateRegistrations(allowDuplicateRegistrations);
 
         // first node => configure with zkUrl; second node => configure with CuratorFramework; third and subsequent
         // shall be configured through system property
@@ -126,11 +125,12 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         }
 
         tcpDisco.setIpFinder(zkIpFinder);
+
         return configuration;
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testOneIgniteNodeIsAlone() throws Exception {
         startGrid(0);
@@ -141,7 +141,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testTwoIgniteNodesFindEachOther() throws Exception {
         // start one node
@@ -164,7 +164,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testThreeNodesWithThreeDifferentConfigMethods() throws Exception {
         // start one node
@@ -195,7 +195,7 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testFourNodesStartingAndStopping() throws Exception {
         // start one node
@@ -242,10 +242,10 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testFourNodesWithDuplicateRegistrations() throws Exception {
-        setAllowDuplicateRegistrations(true);
+        allowDuplicateRegistrations = true;
 
         // start 4 nodes
         System.setProperty(TcpDiscoveryZookeeperIpFinder.PROP_ZK_CONNECTION_STRING, zkCluster.getConnectString());
@@ -265,10 +265,10 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testFourNodesWithNoDuplicateRegistrations() throws Exception {
-        setAllowDuplicateRegistrations(false);
+        allowDuplicateRegistrations = false;
 
         // start 4 nodes
         System.setProperty(TcpDiscoveryZookeeperIpFinder.PROP_ZK_CONNECTION_STRING, zkCluster.getConnectString());
@@ -288,10 +288,10 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testFourNodesRestartLastSeveralTimes() throws Exception {
-        setAllowDuplicateRegistrations(false);
+        allowDuplicateRegistrations = false;
 
         // start 4 nodes
         System.setProperty(TcpDiscoveryZookeeperIpFinder.PROP_ZK_CONNECTION_STRING, zkCluster.getConnectString());
@@ -321,14 +321,13 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
         stopAllGrids();
 
         assertEquals(0, zkCurator.getChildren().forPath(SERVICES_IGNITE_ZK_PATH).size());
-
     }
 
     /**
-     * @throws Exception
+     * @throws Exception If failed.
      */
     public void testFourNodesKillRestartZookeeper() throws Exception {
-        setAllowDuplicateRegistrations(false);
+        allowDuplicateRegistrations = false;
 
         // start 4 nodes
         System.setProperty(TcpDiscoveryZookeeperIpFinder.PROP_ZK_CONNECTION_STRING, zkCluster.getConnectString());
@@ -357,14 +356,28 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
         // stop all grids
         stopAllGrids();
-        Thread.sleep(2000);
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                try {
+                    return zkCurator.getChildren().forPath(SERVICES_IGNITE_ZK_PATH).size() == 0;
+                }
+                catch (Exception e) {
+                    fail("Unexpected error: ");
+
+                    return true;
+                }
+            }
+        }, 5000);
 
         // check that all nodes are gone in ZK
         assertEquals(0, zkCurator.getChildren().forPath(SERVICES_IGNITE_ZK_PATH).size());
     }
 
     /**
-     * @throws Exception
+     * @param ignite Node.
+     * @param joinEventCount Expected events number.
+     * @return Events latch.
      */
     private CountDownLatch expectJoinEvents(Ignite ignite, int joinEventCount) {
         final CountDownLatch latch = new CountDownLatch(joinEventCount);
@@ -378,18 +391,4 @@ public class ZookeeperIpFinderTest extends GridCommonAbstractTest {
 
         return latch;
     }
-
-    /**
-     * @throws Exception
-     */
-    public void setAllowDuplicateRegistrations(boolean allowDuplicateRegistrations) {
-        this.allowDuplicateRegistrations = allowDuplicateRegistrations;
-    }
-
-    /**
-     * @throws Exception
-     */
-    public boolean isAllowDuplicateRegistrations() {
-        return allowDuplicateRegistrations;
-    }
 }
\ No newline at end of file