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

[1/6] incubator-ignite git commit: # IGNITE-709 Add TcpClientDiscoverySpiConfigSelfTest.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-836_2 4be04388e -> 9126679c3


# IGNITE-709 Add TcpClientDiscoverySpiConfigSelfTest.


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

Branch: refs/heads/ignite-836_2
Commit: 7449c6056b8c0963a2a807f4f45902df0c739e2b
Parents: 7625bdc
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 16:56:58 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 16:56:58 2015 +0300

----------------------------------------------------------------------
 .../TcpClientDiscoverySpiConfigSelfTest.java    | 38 ++++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |  1 +
 2 files changed, 39 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7449c605/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
new file mode 100644
index 0000000..06924b7
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.testframework.junits.spi.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpClientDiscoverySpi.class, group = "Discovery SPI")
+public class TcpClientDiscoverySpiConfigSelfTest extends GridSpiAbstractConfigTest<TcpDiscoverySpi> {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNegativeConfig() throws Exception {
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "ipFinder", null);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "networkTimeout", 0);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "socketTimeout", 0);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "ackTimeout", 0);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "heartbeatFrequency", 0);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "threadPriority", -1);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7449c605/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 09ba42b..bb0b97d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -52,6 +52,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpSpiForwardingSelfTest.class));
 
         suite.addTest(new TestSuite(TcpClientDiscoverySelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoverySpiConfigSelfTest.class));
 
         return suite;
     }


[2/6] incubator-ignite git commit: # IGNITE-709 Remove test code.

Posted by se...@apache.org.
# IGNITE-709 Remove test code.


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

Branch: refs/heads/ignite-836_2
Commit: 16694d3b2dd2b38ff8e63392de00f05aaaba284f
Parents: 7449c60
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 17:08:40 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 17:08:40 2015 +0300

----------------------------------------------------------------------
 .../spi/discovery/tcp/TcpClientDiscoverySpi.java       | 13 -------------
 1 file changed, 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/16694d3b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
index def9568..c319f9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
@@ -466,14 +466,10 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
 
                     msg.client(true);
 
-                    System.out.println("TcpClientDiscoverySpi.SocketReader: join write: " + msg);
-
                     writeToSocket(sock, msg);
 
                     int res = readReceipt(sock, ackTimeout);
 
-                    System.out.println("TcpClientDiscoverySpi.SocketReader: join res: " + (res == RES_OK ? "OK" : "" + res));
-
                     switch (res) {
                         case RES_OK:
                             return sock;
@@ -671,9 +667,6 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
 
                             msg.senderNodeId(rmtNodeId);
 
-                            if (!(msg instanceof TcpDiscoveryHeartbeatMessage))
-                                System.out.println("TcpClientDiscoverySpi.SocketReader: read: " + msg);
-
                             if (log.isDebugEnabled())
                                 log.debug("Message has been received: " + msg);
 
@@ -784,9 +777,6 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
                 }
 
                 try {
-                    if (!(msg instanceof TcpDiscoveryHeartbeatMessage))
-                        System.out.println("TcpClientDiscoverySpi.SocketReader: write: " + msg);
-
                     writeToSocket(sock, msg);
 
                     msg = null;
@@ -934,9 +924,6 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
                 while (true) {
                     Object msg = queue.take();
 
-                    if (!(msg instanceof TcpDiscoveryHeartbeatMessage))
-                        System.out.println("TcpClientDiscoverySpi.MessageWorker: process: " + msg);
-
                     if (msg == JOIN_TIMEOUT) {
                         if (joinLatch.getCount() > 0) {
                             joinErr = new IgniteSpiException("Join process timed out [sock=" + sock +


[6/6] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-836_2

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-836_2


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

Branch: refs/heads/ignite-836_2
Commit: 9126679c313bb53a119faf15f6b43a9ab94cda26
Parents: 4be0438 796234f
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 17:37:01 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 17:37:01 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpClientDiscoverySpi.java    |  73 +++++---
 ...pClientDiscoveryMarshallerCheckSelfTest.java |  99 +++++++++++
 .../tcp/TcpClientDiscoverySelfTest.java         | 178 ++++++++++++++++++-
 .../TcpClientDiscoverySpiConfigSelfTest.java    |  39 ++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   2 +
 5 files changed, 362 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9126679c/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9126679c/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------


[4/6] incubator-ignite git commit: # IGNITE-709 Bug fix: don't wait for leaving topology if SPI was not started properly.

Posted by se...@apache.org.
# IGNITE-709 Bug fix: don't wait for leaving topology if SPI was not started properly.


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

Branch: refs/heads/ignite-836_2
Commit: 07e5561bac54c3171a9a86843fc6272eba2af2c0
Parents: 0ac6a73
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 17:15:55 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 17:15:55 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java   | 10 ++++------
 1 file changed, 4 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07e5561b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
index c319f9e..9d437d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
@@ -303,14 +303,12 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
     @Override public void spiStop() throws IgniteSpiException {
         timer.cancel();
 
-        if (msgWorker.isAlive()) { // Should always be alive
+        if (msgWorker != null && msgWorker.isAlive()) { // Should always be alive
             msgWorker.addMessage(SPI_STOP);
 
             try {
-                if (!leaveLatch.await(netTimeout, MILLISECONDS)) {
-                    if (log.isDebugEnabled())
-                        U.error(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
-                }
+                if (!leaveLatch.await(netTimeout, MILLISECONDS))
+                    U.warn(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
             }
             catch (InterruptedException ignored) {
 
@@ -1000,7 +998,7 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
 
                                 joinLatch.countDown();
 
-                                continue;
+                                break;
                             }
                         }
 


[5/6] incubator-ignite git commit: # IGNITE-709 Add test for connection timeout.

Posted by se...@apache.org.
# IGNITE-709 Add test for connection timeout.


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

Branch: refs/heads/ignite-836_2
Commit: 796234f12178413d42eb94fc8080d03747a2df0a
Parents: 07e5561
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 17:36:05 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 17:36:05 2015 +0300

----------------------------------------------------------------------
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 50 ++++++++++++++++++--
 .../tcp/TcpClientDiscoverySelfTest.java         | 43 ++++++++++++++---
 .../TcpClientDiscoverySpiConfigSelfTest.java    |  1 +
 3 files changed, 85 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/796234f1/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
index 9d437d1..e0b67d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
@@ -60,6 +60,9 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
     /** Default disconnect check interval. */
     public static final long DFLT_DISCONNECT_CHECK_INT = 2000;
 
+    /** Default open connection. */
+    public static final long DFLT_OPEN_CONN_TIMEOUT = 5000;
+
     /** */
     private static final Object JOIN_TIMEOUT = "JOIN_TIMEOUT";
 
@@ -106,6 +109,9 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
     private final Timer timer = new Timer("TcpClientDiscoverySpi.timer");
 
     /** */
+    private long openConnTimeout = DFLT_OPEN_CONN_TIMEOUT;
+
+    /** */
     private MessageWorker msgWorker;
 
     /** {@inheritDoc} */
@@ -138,6 +144,20 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
         return netTimeout;
     }
 
+    /**
+     * @return Timeout for opening socket.
+     */
+    public long getOpenConnectionTimeout() {
+        return openConnTimeout;
+    }
+
+    /**
+     * @param openConnTimeout Timeout for opening socket
+     */
+    public void setOpenConnectionTimeout(long openConnTimeout) {
+        this.openConnTimeout = openConnTimeout;
+    }
+
     /** {@inheritDoc} */
     @Override public int getThreadPriority() {
         return threadPri;
@@ -213,6 +233,7 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
         assertParameter(ackTimeout > 0, "ackTimeout > 0");
         assertParameter(hbFreq > 0, "heartbeatFreq > 0");
         assertParameter(threadPri > 0, "threadPri > 0");
+        assertParameter(openConnTimeout > 0, "openConnectionTimeout > 0");
 
         try {
             locHost = U.resolveLocalHost(locAddr);
@@ -408,12 +429,15 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
     }
 
     /**
-     *
+     * @return Opened socket or {@code null} if timeout.
+     * @see #openConnTimeout
      */
-    @NotNull
-    private Socket joinTopology(boolean recon) throws IgniteSpiException, InterruptedException {
+    @SuppressWarnings("BusyWait")
+    @Nullable private Socket joinTopology(boolean recon) throws IgniteSpiException, InterruptedException {
         Collection<InetSocketAddress> addrs = null;
 
+        long startTime = U.currentTimeMillis();
+
         while (true) {
             if (Thread.currentThread().isInterrupted())
                 throw new InterruptedException();
@@ -428,6 +452,9 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
                 else {
                     U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + ipFinder);
 
+                    if ((U.currentTimeMillis() - startTime) > openConnTimeout)
+                        return null;
+
                     Thread.sleep(2000);
                 }
             }
@@ -499,6 +526,9 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
                 U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
                     "in 2000ms): " + addrs0);
 
+                if ((U.currentTimeMillis() - startTime) > openConnTimeout)
+                    return null;
+
                 Thread.sleep(2000);
             }
         }
@@ -831,6 +861,12 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
             try {
                 sock = joinTopology(true);
 
+                if (sock == null) {
+                    log.error("Failed to reconnect to cluster: timeout.");
+
+                    return;
+                }
+
                 if (isInterrupted())
                     throw new InterruptedException();
 
@@ -906,6 +942,14 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
             try {
                 final Socket sock = joinTopology(false);
 
+                if (sock == null) {
+                    joinErr = new IgniteSpiException("Join process timed out");
+
+                    joinLatch.countDown();
+
+                    return;
+                }
+
                 currSock = sock;
 
                 sockWriter.setSocket(sock);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/796234f1/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
index 3e8d5fb..15d9c64 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
@@ -91,6 +91,9 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     /** */
     private UUID nodeId;
 
+    /** */
+    private TcpDiscoveryVmIpFinder clientIpFinder;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -107,15 +110,21 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
         else if (gridName.startsWith("client")) {
             TcpClientDiscoverySpi disco = new TestTcpClientDiscovery();
 
-            TcpDiscoveryVmIpFinder ipFinder = new TcpDiscoveryVmIpFinder();
+            TcpDiscoveryVmIpFinder ipFinder;
+
+            if (clientIpFinder != null)
+                ipFinder = clientIpFinder;
+            else {
+                ipFinder = new TcpDiscoveryVmIpFinder();
 
-            String addr = new ArrayList<>(IP_FINDER.getRegisteredAddresses()).
-                get((clientIdx.get() - 1) / clientsPerSrv).toString();
+                String addr = new ArrayList<>(IP_FINDER.getRegisteredAddresses()).
+                    get((clientIdx.get() - 1) / clientsPerSrv).toString();
 
-            if (addr.startsWith("/"))
-                addr = addr.substring(1);
+                if (addr.startsWith("/"))
+                    addr = addr.substring(1);
 
-            ipFinder.setAddresses(Arrays.asList(addr));
+                ipFinder.setAddresses(Arrays.asList(addr));
+            }
 
             disco.setIpFinder(ipFinder);
 
@@ -156,11 +165,33 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
         stopAllServers(true);
 
         nodeId = null;
+        clientIpFinder = null;
 
         assert G.allGrids().isEmpty();
     }
 
     /**
+     *
+     * @throws Exception
+     */
+    public void testNodeJoinedTimeout() throws Exception {
+        clientIpFinder = new TcpDiscoveryVmIpFinder();
+
+        try {
+            startClientNodes(1);
+
+            fail("Client cannot be start because no server nodes run");
+        }
+        catch (IgniteCheckedException e) {
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+
+            assert spiEx.getMessage().contains("Join process timed out") : spiEx.getMessage();
+        }
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testClientNodeJoin() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/796234f1/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
index 06924b7..6dbd34e 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
@@ -34,5 +34,6 @@ public class TcpClientDiscoverySpiConfigSelfTest extends GridSpiAbstractConfigTe
         checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "ackTimeout", 0);
         checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "heartbeatFrequency", 0);
         checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "threadPriority", -1);
+        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "openConnectionTimeout", 0);
     }
 }


[3/6] incubator-ignite git commit: # IGNITE-709 Add additional tests.

Posted by se...@apache.org.
# IGNITE-709 Add additional tests.


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

Branch: refs/heads/ignite-836_2
Commit: 0ac6a73a4e2484e67531048d7d3727d98f3ad654
Parents: 16694d3
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 7 17:11:46 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 7 17:11:46 2015 +0300

----------------------------------------------------------------------
 ...pClientDiscoveryMarshallerCheckSelfTest.java |  99 ++++++++++++++
 .../tcp/TcpClientDiscoverySelfTest.java         | 135 ++++++++++++++++++-
 .../IgniteSpiDiscoverySelfTestSuite.java        |   1 +
 3 files changed, 234 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac6a73a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
new file mode 100644
index 0000000..fa63e3c
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
@@ -0,0 +1,99 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.marshaller.jdk.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+
+/**
+ * Test for {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi}.
+ */
+public class TcpClientDiscoveryMarshallerCheckSelfTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg =  super.getConfiguration(gridName);
+
+        if (gridName.endsWith("0")) {
+            TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+            discoSpi.setIpFinder(ipFinder);
+
+            cfg.setDiscoverySpi(discoSpi);
+
+            cfg.setLocalHost("127.0.0.1");
+
+            cfg.setMarshaller(new JdkMarshaller());
+        }
+        else {
+            TcpClientDiscoverySpi disco = new TcpClientDiscoverySpi();
+
+            TcpDiscoveryVmIpFinder clientIpFinder = new TcpDiscoveryVmIpFinder();
+
+            String addr = F.first(ipFinder.getRegisteredAddresses()).toString();
+
+            if (addr.startsWith("/"))
+                addr = addr.substring(1);
+
+            clientIpFinder.setAddresses(Collections.singleton(addr));
+
+            disco.setIpFinder(clientIpFinder);
+
+            cfg.setDiscoverySpi(disco);
+
+            cfg.setMarshaller(new OptimizedMarshaller());
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMarshallerInConsistency() throws Exception {
+        startGrid(0);
+
+        try {
+            startGrid(1);
+
+            fail("Expected SPI exception was not thrown.");
+        }
+        catch (IgniteCheckedException e) {
+            Throwable ex = e.getCause().getCause();
+
+            assertTrue(ex instanceof IgniteSpiException);
+            assertTrue(ex.getMessage().contains("Local node's marshaller differs from remote node's marshaller"));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac6a73a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
index 05fb52b..3e8d5fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySelfTest.java
@@ -21,12 +21,14 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
+import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.discovery.tcp.messages.*;
@@ -86,6 +88,9 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     /** */
     private static CountDownLatch msgLatch;
 
+    /** */
+    private UUID nodeId;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -123,6 +128,9 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
             cfg.setNodeId(UUID.fromString(nodeId));
         }
 
+        if (nodeId != null)
+            cfg.setNodeId(nodeId);
+
         return cfg;
     }
 
@@ -147,6 +155,8 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
         stopAllClients(true);
         stopAllServers(true);
 
+        nodeId = null;
+
         assert G.allGrids().isEmpty();
     }
 
@@ -287,7 +297,25 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void testClientReconnect() throws Exception {
+    public void testPing() throws Exception {
+        startServerNodes(2);
+        startClientNodes(1);
+
+        Ignite srv0 = G.ignite("server-0");
+        Ignite srv1 = G.ignite("server-1");
+        Ignite client = G.ignite("client-0");
+
+        assert ((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
+        assert ((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
+
+        assert ((IgniteEx)client).context().discovery().pingNode(srv0.cluster().localNode().id());
+        assert ((IgniteEx)client).context().discovery().pingNode(srv1.cluster().localNode().id());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectOnRouterFail() throws Exception {
         clientsPerSrv = 1;
 
         startServerNodes(3);
@@ -313,6 +341,31 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
+    public void testClientReconnectOnNetworkProblem() throws Exception {
+        clientsPerSrv = 1;
+
+        startServerNodes(3);
+        startClientNodes(3);
+
+        checkNodes(3, 3);
+
+        setClientRouter(2, 0);
+
+        srvFailedLatch = new CountDownLatch(2);
+        clientFailedLatch = new CountDownLatch(3);
+
+        attachListeners(2, 3);
+
+        ((TcpClientDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi()).brokeConnection();
+
+        G.ignite("client-2").message().remoteListen(null, new MessageListener()); // Send some discovery message.
+
+        checkNodes(3, 3);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testGetMissedMessagesOnReconnect() throws Exception {
         clientsPerSrv = 1;
 
@@ -563,6 +616,86 @@ public class TcpClientDiscoverySelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @throws Exception If any error occurs.
+     */
+    public void testDuplicateId() throws Exception {
+        startServerNodes(2);
+
+        nodeId = G.ignite("server-1").cluster().localNode().id();
+
+        try {
+            startGrid("client-0");
+
+            assert false;
+        }
+        catch (IgniteCheckedException e) {
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+            assert spiEx.getMessage().contains("same ID") : spiEx.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testJoinTimeout() throws Exception {
+        startServerNodes(2);
+
+        final CountDownLatch cnt = new CountDownLatch(1);
+
+        ((TcpDiscoverySpi)G.ignite("server-1").configuration().getDiscoverySpi()).addSendMessageListener(
+            new IgniteInClosure<TcpDiscoveryAbstractMessage>() {
+                @Override public void apply(TcpDiscoveryAbstractMessage msg) {
+                    try {
+                        cnt.await(10, MINUTES);
+                    }
+                    catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+
+                        throw new IgniteInterruptedException(e);
+                    }
+                }
+            });
+
+        try {
+            startGrid("client-0");
+
+            assert false;
+        }
+        catch (IgniteCheckedException e) {
+            cnt.countDown();
+
+            IgniteSpiException spiEx = e.getCause(IgniteSpiException.class);
+
+            assert spiEx != null : e;
+            assert spiEx.getMessage().contains("Join process timed out") : spiEx.getMessage();
+        }
+    }
+
+    /**
+     * @throws Exception If any error occurs.
+     */
+    public void testGridStartTime() throws Exception {
+        startServerNodes(2);
+
+        startClientNodes(2);
+
+        long startTime = -1;
+
+        for (Ignite g : G.allGrids()) {
+            IgniteEx kernal = (IgniteKernal)g;
+
+            assertTrue(kernal.context().discovery().gridStartTime() > 0);
+
+            if (startTime == -1)
+                startTime = kernal.context().discovery().gridStartTime();
+            else
+                assertEquals(startTime, kernal.context().discovery().gridStartTime());
+        }
+    }
+
+    /**
      * @param clientIdx Index.
      * @throws Exception In case of error.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0ac6a73a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index bb0b97d..ebc7111 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -53,6 +53,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
 
         suite.addTest(new TestSuite(TcpClientDiscoverySelfTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoverySpiConfigSelfTest.class));
+        suite.addTest(new TestSuite(TcpClientDiscoveryMarshallerCheckSelfTest.class));
 
         return suite;
     }