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/25 15:28:03 UTC

incubator-ignite git commit: # IGNITE-709 Create and use org.apache.ignite.testframework.junits.ConfigurationPatcher class.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-709_tests [created] 056bb33ec


# IGNITE-709 Create and use org.apache.ignite.testframework.junits.ConfigurationPatcher class.


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

Branch: refs/heads/ignite-709_tests
Commit: 056bb33ec3f8a808e85690eee863116ff9baf42d
Parents: 97e6bd3
Author: sevdokimov <se...@gridgain.com>
Authored: Mon May 25 16:27:44 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon May 25 16:27:44 2015 +0300

----------------------------------------------------------------------
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |  58 ++++-------
 .../junits/ConfigurationPatcher.java            |  30 ++++++
 .../junits/ConfigurationPatchers.java           | 104 +++++++++++++++++++
 .../testframework/junits/GridAbstractTest.java  |  74 ++++++++++++-
 4 files changed, 230 insertions(+), 36 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/056bb33e/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index 8157d59..0d9d8a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -34,6 +34,7 @@ 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.*;
 import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -90,20 +91,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     private static CountDownLatch msgLatch;
 
     /** */
-    private UUID nodeId;
-
-    /** */
     private TcpDiscoveryVmIpFinder clientIpFinder;
 
-    /** */
-    private long joinTimeout = TcpClientDiscoverySpi.DFLT_JOIN_TIMEOUT;
-
-    /** */
-    private long netTimeout = TcpDiscoverySpiAdapter.DFLT_NETWORK_TIMEOUT;
-
-    /** */
-    private boolean longSockTimeouts;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -147,19 +136,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         else
             throw new IllegalArgumentException();
 
-        if (longSockTimeouts) {
-            disco.setAckTimeout(2000);
-            disco.setSocketTimeout(2000);
-        }
-
-        disco.setJoinTimeout(joinTimeout);
-        disco.setNetworkTimeout(netTimeout);
-
         cfg.setDiscoverySpi(disco);
 
-        if (nodeId != null)
-            cfg.setNodeId(nodeId);
-
         return cfg;
     }
 
@@ -184,11 +162,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         stopAllClients(true);
         stopAllServers(true);
 
-        nodeId = null;
         clientIpFinder = null;
-        joinTimeout = TcpClientDiscoverySpi.DFLT_JOIN_TIMEOUT;
-        netTimeout = TcpClientDiscoverySpi.DFLT_NETWORK_TIMEOUT;
-        longSockTimeouts = false;
 
         assert G.allGrids().isEmpty();
     }
@@ -199,10 +173,13 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      */
     public void testJoinTimeout() throws Exception {
         clientIpFinder = new TcpDiscoveryVmIpFinder();
-        joinTimeout = 1000;
 
         try {
-            startClientNodes(1);
+            startGrid("client-0", new ConfigurationPatchers.DiscoveryPatcher() {
+                @Override protected void patch(TcpDiscoverySpiAdapter disco) {
+                    disco.setJoinTimeout(500);
+                }
+            });
 
             fail("Client cannot be start because no server nodes run");
         }
@@ -792,10 +769,14 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     public void testDuplicateId() throws Exception {
         startServerNodes(2);
 
-        nodeId = G.ignite("server-1").cluster().localNode().id();
+        final UUID nodeId = G.ignite("server-1").cluster().localNode().id();
 
         try {
-            startGrid("client-0");
+            startGrid("client-0", new ConfigurationPatcher() {
+                @Override public void patch(IgniteConfiguration cfg) {
+                    cfg.setNodeId(nodeId);
+                }
+            });
 
             assert false;
         }
@@ -811,7 +792,12 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      * @throws Exception If any error occurs.
      */
     public void testTimeoutWaitingNodeAddedMessage() throws Exception {
-        longSockTimeouts = true;
+        addPatchers(new ConfigurationPatchers.DiscoveryPatcher() {
+            @Override public void patch(TcpDiscoverySpiAdapter disco) {
+                disco.setAckTimeout(2000);
+                disco.setSocketTimeout(2000);
+            }
+        });
 
         startServerNodes(2);
 
@@ -832,9 +818,11 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             });
 
         try {
-            netTimeout = 500;
-
-            startGrid("client-0");
+            startGrid("client-0", new ConfigurationPatchers.DiscoveryPatcher() {
+                @Override protected void patch(TcpDiscoverySpiAdapter disco) {
+                    disco.setNetworkTimeout(500);
+                }
+            });
 
             assert false;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/056bb33e/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatcher.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatcher.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatcher.java
new file mode 100644
index 0000000..e4d20fd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatcher.java
@@ -0,0 +1,30 @@
+/*
+ * 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.testframework.junits;
+
+import org.apache.ignite.configuration.*;
+
+/**
+ * Class allow to modify Ignite configuration.
+ */
+public interface ConfigurationPatcher {
+    /**
+     * @param cfg Configuration to patch.
+     */
+    public void patch(IgniteConfiguration cfg);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/056bb33e/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatchers.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatchers.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatchers.java
new file mode 100644
index 0000000..474dfbc
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/ConfigurationPatchers.java
@@ -0,0 +1,104 @@
+/*
+ * 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.testframework.junits;
+
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+public class ConfigurationPatchers {
+    /**
+     * Default constructor.
+     */
+    private ConfigurationPatchers() {
+        // No-op.
+    }
+
+    /**
+     * @param patchers Patchers.
+     */
+    public static ConfigurationPatcher union(ConfigurationPatcher ... patchers) {
+        if (patchers.length == 0)
+            return EmptyPatcher.INSTANCE;
+
+        if (patchers.length == 1)
+            return patchers[0];
+
+        return new Union(Arrays.asList(patchers));
+    }
+
+    /**
+     *
+     */
+    public abstract static class DiscoveryPatcher implements ConfigurationPatcher {
+        /**
+         * @param disco Discovery.
+         */
+        protected abstract void patch(TcpDiscoverySpiAdapter disco);
+
+        /** {@inheritDoc} */
+        @Override public final void patch(IgniteConfiguration cfg) {
+            patch((TcpDiscoverySpiAdapter)cfg.getDiscoverySpi());
+        }
+    }
+
+    /**
+     *
+     */
+    public static class EmptyPatcher implements ConfigurationPatcher {
+        /** */
+        public static final EmptyPatcher INSTANCE = new EmptyPatcher();
+
+        /**
+         *
+         */
+        private EmptyPatcher() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void patch(IgniteConfiguration cfg) {
+            // No-op.
+        }
+    }
+
+    /**
+     *
+     */
+    public static class Union implements ConfigurationPatcher {
+        /** */
+        private final Collection<ConfigurationPatcher> patchers;
+
+        /**
+         * @param patchers Patchers.
+         */
+        public Union(Collection<ConfigurationPatcher> patchers) {
+            this.patchers = patchers;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void patch(IgniteConfiguration cfg) {
+            for (ConfigurationPatcher patcher : patchers)
+                patcher.patch(cfg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/056bb33e/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 2cae73a..98d4122 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -96,6 +96,9 @@ public abstract class GridAbstractTest extends TestCase {
     /** Starting grid name. */
     protected static ThreadLocal<String> startingGrid = new ThreadLocal<>();
 
+    /** */
+    private final ThreadLocal<ConfigurationPatcher> patchers = new ThreadLocal<>();
+
     /**
      *
      */
@@ -483,6 +486,8 @@ public abstract class GridAbstractTest extends TestCase {
             }
         }
 
+        patchers.remove();
+
         info(">>> Starting test: " + getName() + " <<<");
 
         try {
@@ -621,6 +626,24 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Started grid.
      * @throws Exception If anything failed.
      */
+    protected IgniteEx startGrid(int idx, ConfigurationPatcher ... patchers) throws Exception {
+        ConfigurationPatcher oldPatcher = addPatchers(patchers);
+
+        try {
+            return (IgniteEx)startGrid(getTestGridName(idx));
+        }
+        finally {
+            setPatcher(oldPatcher);
+        }
+    }
+
+    /**
+     * Starts new grid with given index.
+     *
+     * @param idx Index of the grid to start.
+     * @return Started grid.
+     * @throws Exception If anything failed.
+     */
     protected IgniteEx startGrid(int idx) throws Exception {
         return (IgniteEx)startGrid(getTestGridName(idx));
     }
@@ -644,6 +667,24 @@ public abstract class GridAbstractTest extends TestCase {
      * @return Started grid.
      * @throws Exception If failed.
      */
+    protected Ignite startGrid(String gridName, ConfigurationPatcher ... patchers) throws Exception {
+        ConfigurationPatcher oldPatchers = addPatchers(patchers);
+
+        try {
+            return startGrid(gridName, (GridSpringResourceContext)null);
+        }
+        finally {
+            setPatcher(oldPatchers);
+        }
+    }
+
+    /**
+     * Starts new grid with given name.
+     *
+     * @param gridName Grid name.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
     protected Ignite startGrid(String gridName) throws Exception {
         return startGrid(gridName, (GridSpringResourceContext)null);
     }
@@ -660,7 +701,12 @@ public abstract class GridAbstractTest extends TestCase {
         startingGrid.set(gridName);
 
         try {
-            return IgnitionEx.start(optimize(getConfiguration(gridName)), ctx);
+            IgniteConfiguration cfg = getConfiguration(gridName);
+
+            if (patchers.get() != null)
+                patchers.get().patch(cfg);
+
+            return IgnitionEx.start(optimize(cfg), ctx);
         }
         finally {
             startingGrid.set(null);
@@ -668,6 +714,32 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * @param patchers Patchers to add.
+     * @return Old patcher.
+     */
+    protected ConfigurationPatcher addPatchers(ConfigurationPatcher... patchers) {
+        ConfigurationPatcher oldPatchers = this.patchers.get();
+
+        ConfigurationPatcher newPatcher = ConfigurationPatchers.union(patchers);
+
+        this.patchers.set(oldPatchers == null ? newPatcher : ConfigurationPatchers.union(oldPatchers, newPatcher));
+
+        return oldPatchers;
+    }
+
+    /**
+     * @param patcher Patcher to set.
+     * @return Old patcher.
+     */
+    protected ConfigurationPatcher setPatcher(ConfigurationPatcher patcher) {
+        ConfigurationPatcher oldPatchers = patchers.get();
+
+        patchers.set(patcher);
+
+        return oldPatchers;
+    }
+
+    /**
      * Optimizes configuration to achieve better test performance.
      *
      * @param cfg Configuration.