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 2017/12/01 08:22:54 UTC

[5/9] ignite git commit: IGNITE-7070 Ignite PDS compatibility framework improvements - Fixes #3106.

IGNITE-7070 Ignite PDS compatibility framework improvements - Fixes #3106.

Signed-off-by: Alexey Goncharuk <al...@gmail.com>


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

Branch: refs/heads/ignite-zk
Commit: 83049d7ae3f583483c9cb320d4d51896552183a8
Parents: 3998fbb
Author: dpavlov <dp...@gridgain.com>
Authored: Thu Nov 30 16:21:30 2017 +0300
Committer: Alexey Goncharuk <al...@gmail.com>
Committed: Thu Nov 30 16:21:30 2017 +0300

----------------------------------------------------------------------
 modules/compatibility/pom.xml                   |   8 ++
 .../DummyPersistenceCompatibilityTest.java      | 108 +++++++++++------
 .../testframework/junits/Dependency.java        | 117 +++++++++++++++++++
 .../junits/IgniteCompatibilityAbstractTest.java |  72 +++++++++---
 .../junits/IgniteCompatibilityNodeRunner.java   | 110 +++++++++++++----
 .../testframework/util/MavenUtils.java          |  30 ++---
 6 files changed, 351 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/pom.xml
----------------------------------------------------------------------
diff --git a/modules/compatibility/pom.xml b/modules/compatibility/pom.xml
index d29e816..aea95cc 100644
--- a/modules/compatibility/pom.xml
+++ b/modules/compatibility/pom.xml
@@ -37,6 +37,10 @@
     <url>http://ignite.apache.org</url>
 
     <dependencies>
+        <!-- Note: when adding new Ignite module dependency please register it in
+        org.apache.ignite.compatibility.testframework.junits.IgniteCompatibilityAbstractTest.getDependencies()
+        method. This will allow to use original version dependency instead current code base. -->
+
         <dependency>
             <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-core</artifactId>
@@ -90,6 +94,10 @@
             <version>${spring.version}</version>
             <scope>test</scope>
         </dependency>
+
+        <!-- Note: when adding new Ignite module dependency please register it in
+        org.apache.ignite.compatibility.testframework.junits.IgniteCompatibilityAbstractTest.getDependencies()
+        method. This will allow to use original version dependency instead current code base. -->
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java
index b05d5a6..466b858 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/persistence/DummyPersistenceCompatibilityTest.java
@@ -22,10 +22,12 @@ import java.io.IOException;
 import java.io.ObjectInput;
 import java.io.ObjectOutput;
 import java.io.Serializable;
+import javax.cache.Cache;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicityMode;
 import org.apache.ignite.cache.CacheWriteSynchronizationMode;
+import org.apache.ignite.configuration.BinaryConfiguration;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.DataRegionConfiguration;
 import org.apache.ignite.configuration.DataStorageConfiguration;
@@ -96,12 +98,12 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib
     /**
      * Tests opportunity to read data from previous Ignite DB version.
      *
-     * @param ver 3-digits version of ignite
+     * @param igniteVer 3-digits version of ignite
      * @throws Exception If failed.
      */
-    private void doTestStartupWithOldVersion(String ver) throws Exception {
+    protected void doTestStartupWithOldVersion(String igniteVer, boolean compactFooter) throws Exception {
         try {
-            startGrid(1, ver, new ConfigurationClosure(), new PostStartupClosure());
+            startGrid(1, igniteVer, new ConfigurationClosure(compactFooter), new PostStartupClosure());
 
             stopAllGrids();
 
@@ -111,31 +113,67 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib
 
             ignite.active(true);
 
-            IgniteCache<Object, Object> cache = ignite.getOrCreateCache(TEST_CACHE_NAME);
-
-            for (int i = 0; i < 10; i++)
-                assertEquals("data" + i, cache.get(i));
-
-            assertEquals(cache.get("1"), "2");
-            assertEquals(cache.get(12), 2);
-            assertEquals(cache.get(13L), 2L);
-            assertEquals(cache.get(TestEnum.A), "Enum_As_Key");
-            assertEquals(cache.get("Enum_As_Value"), TestEnum.B);
-            assertEquals(cache.get(TestEnum.C), TestEnum.C);
-            assertEquals(cache.get("Serializable"), new TestSerializable(42));
-            assertEquals(cache.get(new TestSerializable(42)), "Serializable_As_Key");
-            assertEquals(cache.get("Externalizable"), new TestExternalizable(42));
-            assertEquals(cache.get(new TestExternalizable(42)), "Externalizable_As_Key");
-            assertEquals(cache.get("testStringContainer"),
-                new TestStringContainerToBePrinted("testStringContainer"));
+            validateResultingCacheData(ignite.getOrCreateCache(TEST_CACHE_NAME));
         }
         finally {
             stopAllGrids();
         }
     }
 
+    /**
+     * Tests opportunity to read data from previous Ignite DB version.
+     *
+     * @param igniteVer 3-digits version of ignite
+     * @throws Exception If failed.
+     */
+    protected void doTestStartupWithOldVersion(String igniteVer) throws Exception {
+        doTestStartupWithOldVersion(igniteVer, true);
+    }
+
+    /**
+     * @param cache to be filled by different keys and values. Results may be validated in {@link
+     * #validateResultingCacheData(Cache)}.
+     */
+    public static void saveCacheData(Cache<Object, Object> cache) {
+        for (int i = 0; i < 10; i++)
+            cache.put(i, "data" + i);
+
+        cache.put("1", "2");
+        cache.put(12, 2);
+        cache.put(13L, 2L);
+        cache.put(TestEnum.A, "Enum_As_Key");
+        cache.put("Enum_As_Value", TestEnum.B);
+        cache.put(TestEnum.C, TestEnum.C);
+        cache.put("Serializable", new TestSerializable(42));
+        cache.put(new TestSerializable(42), "Serializable_As_Key");
+        cache.put("Externalizable", new TestExternalizable(42));
+        cache.put(new TestExternalizable(42), "Externalizable_As_Key");
+        cache.put("testStringContainer", new TestStringContainerToBePrinted("testStringContainer"));
+    }
+
+    /**
+     * Asserts cache contained all expected values as it was saved before.
+     * @param cache cache should be filled using {@link #saveCacheData(Cache)}.
+     */
+    public static void validateResultingCacheData(Cache<Object, Object> cache) {
+        for (int i = 0; i < 10; i++)
+            assertEquals(cache.get(i), "data" + i);
+
+        assertEquals("2", cache.get("1"));
+        assertEquals(2, cache.get(12));
+        assertEquals(2L, cache.get(13L));
+        assertEquals("Enum_As_Key", cache.get(TestEnum.A));
+        assertEquals(TestEnum.B, cache.get("Enum_As_Value"));
+        assertEquals(TestEnum.C, cache.get(TestEnum.C));
+        assertEquals(new TestSerializable(42), cache.get("Serializable"));
+        assertEquals("Serializable_As_Key", cache.get(new TestSerializable(42)));
+        assertEquals(new TestExternalizable(42), cache.get("Externalizable"));
+        assertEquals("Externalizable_As_Key", cache.get(new TestExternalizable(42)));
+        assertEquals(new TestStringContainerToBePrinted("testStringContainer"), cache.get("testStringContainer"));
+    }
+
     /** */
-    private static class PostStartupClosure implements IgniteInClosure<Ignite> {
+    public static class PostStartupClosure implements IgniteInClosure<Ignite> {
         /** {@inheritDoc} */
         @Override public void apply(Ignite ignite) {
             ignite.active(true);
@@ -148,25 +186,18 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib
 
             IgniteCache<Object, Object> cache = ignite.createCache(cacheCfg);
 
-            for (int i = 0; i < 10; i++)
-                cache.put(i, "data" + i);
-
-            cache.put("1", "2");
-            cache.put(12, 2);
-            cache.put(13L, 2L);
-            cache.put(TestEnum.A, "Enum_As_Key");
-            cache.put("Enum_As_Value", TestEnum.B);
-            cache.put(TestEnum.C, TestEnum.C);
-            cache.put("Serializable", new TestSerializable(42));
-            cache.put(new TestSerializable(42), "Serializable_As_Key");
-            cache.put("Externalizable", new TestExternalizable(42));
-            cache.put(new TestExternalizable(42), "Externalizable_As_Key");
-            cache.put("testStringContainer", new TestStringContainerToBePrinted("testStringContainer"));
+            saveCacheData(cache);
         }
     }
 
     /** */
-    private static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
+    public static class ConfigurationClosure implements IgniteInClosure<IgniteConfiguration> {
+        private boolean compactFooter;
+
+        public ConfigurationClosure(boolean compactFooter) {
+            this.compactFooter = compactFooter;
+        }
+
         /** {@inheritDoc} */
         @Override public void apply(IgniteConfiguration cfg) {
             cfg.setLocalHost("127.0.0.1");
@@ -179,6 +210,9 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib
             cfg.setPeerClassLoadingEnabled(false);
 
             cfg.setPersistentStoreConfiguration(new PersistentStoreConfiguration());
+
+            if (!compactFooter)
+                cfg.setBinaryConfiguration(new BinaryConfiguration().setCompactFooter(compactFooter));
         }
     }
 
@@ -275,7 +309,7 @@ public class DummyPersistenceCompatibilityTest extends IgnitePersistenceCompatib
             if (o == null || getClass() != o.getClass())
                 return false;
 
-            TestExternalizable that = ( TestExternalizable)o;
+            TestExternalizable that = (TestExternalizable)o;
 
             return iVal == that.iVal;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/Dependency.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/Dependency.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/Dependency.java
new file mode 100644
index 0000000..56da0e9
--- /dev/null
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/Dependency.java
@@ -0,0 +1,117 @@
+/*
+ * 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.compatibility.testframework.junits;
+
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Module dependency: Should be filtered out from current test classpath for separate JVM classpath.
+ */
+public class Dependency {
+    /** Local module name. Folder name where module is located. */
+    private String locModuleName;
+
+    /** Group name. Null means ignite default group name. */
+    @Nullable
+    private String groupName;
+
+    /** Artifact name (artifact ID) without group name. */
+    private String artifactName;
+
+    /** Version. Null means default Ignite version is to be used. May be used for 3rd party dependencies. */
+    @Nullable
+    private String version;
+
+    /** Test flag. Test jar should have {@code true} value. Default is {@code false}. */
+    private boolean test;
+
+    /**
+     * Creates dependency.
+     *
+     * @param locModuleName Local module name. Folder name where module is located.
+     * @param artifactName Artifact name (artifact ID) without group name.
+     * @param test Test flag. Test jar should have {@code true} value. Default is {@code false}.
+     */
+    public Dependency(String locModuleName, String artifactName, boolean test) {
+        this.locModuleName = locModuleName;
+        this.artifactName = artifactName;
+        this.test = test;
+    }
+
+    /**
+     * Creates dependency.
+     *
+     * @param locModuleName Local module name. Folder name where module is located.
+     * @param artifactName Artifact name (artifact ID) without group name.
+     */
+    public Dependency(String locModuleName, String artifactName) {
+        this.locModuleName = locModuleName;
+        this.artifactName = artifactName;
+    }
+
+    /**
+     * @param locModuleName Local module name. Folder name where module is located.
+     * @param grpName Group name. Null means ignite default group name.
+     * @param artifactName Artifact name (artifact ID) without group na
+     * @param version Version. Null means default Ignite version is to be used. M
+     */
+    public Dependency(String locModuleName, @Nullable String grpName, String artifactName, @Nullable String version) {
+        this.locModuleName = locModuleName;
+        this.groupName = grpName;
+        this.artifactName = artifactName;
+        this.version = version;
+    }
+
+    /**
+     * @return path based on local module name to exclude from classpath
+     */
+    public String localPathTemplate() {
+        return "modules/" +
+            locModuleName +
+            "/target/" +
+            (test ? "test-classes" : "classes");
+    }
+
+    /**
+     * @return {@link #artifactName}
+     */
+    public String artifactName() {
+        return artifactName;
+    }
+
+    /**
+     * @return classifier or {@code} null depending on {@link #test} flag
+     */
+    @Nullable public String classifier() {
+        return test ? "tests" : null;
+    }
+
+    /**
+     * @return {@link #version}
+     */
+    @Nullable public String version() {
+        return version;
+    }
+
+    /**
+     * @return {@link #groupName}
+     */
+    @Nullable public String groupName() {
+        return groupName;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
index d637d31..321da12 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityAbstractTest.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.apache.ignite.testframework.junits.multijvm.IgniteProcessProxy;
+import org.jetbrains.annotations.NotNull;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -120,7 +121,7 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
      * stored via Maven.
      *
      * @param igniteInstanceName Instance name.
-     * @param ver Ignite version.
+     * @param ver Ignite version. Dots separated, 3-digit version.
      * @param cfgClo IgniteInClosure for post-configuration.
      * @param clo IgniteInClosure for actions on started Ignite.
      * @return Started grid.
@@ -165,29 +166,40 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
 
                 URLClassLoader ldr = (URLClassLoader)CLASS_LOADER;
 
-                StringBuilder pathBuilder = new StringBuilder();
-
-                String corePathTemplate = "modules/core/target/classes";
-                String coreTestsPathTemplate = "modules/core/target/test-classes";
+                final Collection<Dependency> dependencies = getDependencies(ver);
 
+                StringBuilder pathBuilder = new StringBuilder();
                 for (URL url : ldr.getURLs()) {
                     String path = url.getPath();
 
-                    if (!path.contains(corePathTemplate) && !path.contains(coreTestsPathTemplate))
+                    boolean excluded = false;
+                    for (Dependency next : dependencies) {
+                        if (path.contains(next.localPathTemplate())) {
+                            excluded = true;
+                            break;
+                        }
+                    }
+                    if (!excluded)
                         pathBuilder.append(path).append(File.pathSeparator);
                 }
 
-                String pathToArtifact = MavenUtils.getPathToIgniteCoreArtifact(ver);
+                for (Dependency next : dependencies) {
+                    final String artifactVer = next.version() != null ? next.version() : ver;
+                    final String grpName = next.groupName() != null ? next.groupName() : "org.apache.ignite";
+                    String pathToArtifact = MavenUtils.getPathToIgniteArtifact(grpName, next.artifactName(),
+                        artifactVer,  next.classifier());
 
-                pathBuilder.append(pathToArtifact).append(File.pathSeparator);
-
-                String pathToTestsArtifact = MavenUtils.getPathToIgniteCoreArtifact(ver, "tests");
-
-                pathBuilder.append(pathToTestsArtifact).append(File.pathSeparator);
+                    pathBuilder.append(pathToArtifact).append(File.pathSeparator);
+                }
 
                 filteredJvmArgs.add("-cp");
                 filteredJvmArgs.add(pathBuilder.toString());
 
+                final Collection<String> jvmParms = getJvmParms();
+
+                if (jvmParms != null)
+                    filteredJvmArgs.addAll(jvmParms);
+
                 return filteredJvmArgs;
             }
         };
@@ -201,7 +213,11 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
 
             log.addListener(nodeId, new LoggedJoinNodeClosure(nodeJoinedLatch, nodeId));
 
-            assert nodeJoinedLatch.await(NODE_JOIN_TIMEOUT, TimeUnit.MILLISECONDS) : "Node has not joined [id=" + nodeId + "]";
+            final long nodeJoinTimeout = getNodeJoinTimeout();
+            final boolean joined = nodeJoinedLatch.await(nodeJoinTimeout, TimeUnit.MILLISECONDS);
+
+            assertTrue("Node has not joined [id=" + nodeId + "]/" +
+                "or does not completed its startup during timeout: " + nodeJoinTimeout + " ms.", joined);
 
             log.removeListener(nodeId);
         }
@@ -212,6 +228,36 @@ public abstract class IgniteCompatibilityAbstractTest extends GridCommonAbstract
         return ignite;
     }
 
+    /**
+     * Total amount of milliseconds.
+     *
+     * @return timeout in ms.
+     */
+    protected long getNodeJoinTimeout() {
+        return NODE_JOIN_TIMEOUT;
+    }
+
+    /**
+     * @return list of actual module dependencies from pom.xml
+     */
+    @NotNull protected Collection<Dependency> getDependencies(String igniteVer) {
+        final Collection<Dependency> dependencies = new ArrayList<>();
+
+        dependencies.add(new Dependency("core", "ignite-core"));
+        dependencies.add(new Dependency("core", "ignite-core", true));
+
+        return dependencies;
+    }
+
+    /**
+     * Allows to setup JVM arguments for standalone JVM
+     *
+     * @return additional JVM arguments
+     */
+    protected Collection<String> getJvmParms() {
+        return new ArrayList<>();
+    }
+
     /** {@inheritDoc} */
     @Override protected Ignite startGrid(String igniteInstanceName, IgniteConfiguration cfg,
         GridSpringResourceContext ctx) throws Exception {

http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
index 3b5010b..7c9a511 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/junits/IgniteCompatibilityNodeRunner.java
@@ -22,6 +22,8 @@ import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.IOException;
+import java.net.URL;
+import java.net.URLClassLoader;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Paths;
@@ -64,43 +66,105 @@ public class IgniteCompatibilityNodeRunner extends IgniteNodeRunner {
      * @throws Exception In case of an error.
      */
     public static void main(String[] args) throws Exception {
-        X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
+        try {
+            X.println(GridJavaProcess.PID_MSG_PREFIX + U.jvmPid());
 
-        X.println("Starting Ignite Node... Args=" + Arrays.toString(args));
+            X.println("Starting Ignite Node... Args=" + Arrays.toString(args));
 
-        if (args.length < 3) {
-            throw new IllegalArgumentException("At least four arguments expected:" +
-                " [path/to/closure/file] [ignite-instance-name] [node-id] [sync-node-id] [optional/path/to/closure/file]");
-        }
+            if (args.length < 3) {
+                throw new IllegalArgumentException("At least four arguments expected:" +
+                    " [path/to/closure/file] [ignite-instance-name] [node-id] [sync-node-id] [optional/path/to/closure/file]");
+            }
+
+            final Thread watchdog = delayedDumpClasspath();
 
-        IgniteConfiguration cfg = CompatibilityTestsFacade.getConfiguration();
+            IgniteConfiguration cfg = CompatibilityTestsFacade.getConfiguration();
 
-        IgniteInClosure<IgniteConfiguration> cfgClo = readClosureFromFileAndDelete(args[0]);
+            IgniteInClosure<IgniteConfiguration> cfgClo = readClosureFromFileAndDelete(args[0]);
 
-        cfgClo.apply(cfg);
+            cfgClo.apply(cfg);
 
-        final UUID nodeId = UUID.fromString(args[2]);
-        final UUID syncNodeId = UUID.fromString(args[3]);
+            final UUID nodeId = UUID.fromString(args[2]);
+            final UUID syncNodeId = UUID.fromString(args[3]);
 
-        // Ignite instance name and id must be set according to arguments
-        // it's used for nodes managing: start, stop etc.
-        cfg.setIgniteInstanceName(args[1]);
-        cfg.setNodeId(nodeId);
+            // Ignite instance name and id must be set according to arguments
+            // it's used for nodes managing: start, stop etc.
+            cfg.setIgniteInstanceName(args[1]);
+            cfg.setNodeId(nodeId);
 
-        final Ignite ignite = Ignition.start(cfg);
+            final Ignite ignite = Ignition.start(cfg);
 
-        assert ignite.cluster().node(syncNodeId) != null : "Node has not joined [id=" + nodeId + "]";
+            assert ignite.cluster().node(syncNodeId) != null : "Node has not joined [id=" + nodeId + "]";
 
-        // It needs to set private static field 'ignite' of the IgniteNodeRunner class via reflection
-        GridTestUtils.setFieldValue(new IgniteNodeRunner(), "ignite", ignite);
+            // It needs to set private static field 'ignite' of the IgniteNodeRunner class via reflection
+            GridTestUtils.setFieldValue(new IgniteNodeRunner(), "ignite", ignite);
 
-        if (args.length == 5) {
-            IgniteInClosure<Ignite> clo = readClosureFromFileAndDelete(args[4]);
+            if (args.length == 5) {
+                IgniteInClosure<Ignite> clo = readClosureFromFileAndDelete(args[4]);
 
-            clo.apply(ignite);
+                clo.apply(ignite);
+            }
+
+            X.println(IgniteCompatibilityAbstractTest.SYNCHRONIZATION_LOG_MESSAGE + nodeId);
+            watchdog.interrupt();
+        }
+        catch (Throwable e) {
+            X.println("Dumping classpath, error occurred: " + e);
+            dumpClasspath();
+            throw e;
         }
+    }
 
-        X.println(IgniteCompatibilityAbstractTest.SYNCHRONIZATION_LOG_MESSAGE + nodeId);
+    /**
+     * Starts background watchdog thread which will dump main thread stacktrace and classpath dump if main thread
+     * will not respond with node startup finished.
+     *
+     * @return Thread to be interrupted.
+     */
+    private static Thread delayedDumpClasspath() {
+        final Thread mainThread = Thread.currentThread();
+        final Runnable target = new Runnable() {
+            @Override public void run() {
+                try {
+                    final int timeout = IgniteCompatibilityAbstractTest.NODE_JOIN_TIMEOUT - 1_000;
+                    if (timeout > 0)
+                        Thread.sleep(timeout);
+                }
+                catch (InterruptedException ignored) {
+                    //interrupt is correct behaviour
+                    return;
+                }
+
+                X.println("Ignite startup/Init closure/post configuration closure is probably hanging at");
+
+                for (StackTraceElement ste : mainThread.getStackTrace()) {
+                    X.println("\t" + ste.toString());
+                }
+
+                X.println("\nDumping classpath");
+                dumpClasspath();
+            }
+        };
+
+        final Thread thread = new Thread(target);
+        thread.setDaemon(true);
+        thread.start();
+
+        return thread;
+    }
+
+    /**
+     * Dumps classpath to output stream.
+     */
+    private static void dumpClasspath() {
+        final ClassLoader clsLdr = IgniteCompatibilityNodeRunner.class.getClassLoader();
+        if (clsLdr instanceof URLClassLoader) {
+            URLClassLoader ldr = (URLClassLoader)clsLdr;
+
+            for (URL url : ldr.getURLs()) {
+                X.println("Classpath url: [" + url.getPath() + "]");
+            }
+        }
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/83049d7a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/util/MavenUtils.java
----------------------------------------------------------------------
diff --git a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/util/MavenUtils.java b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/util/MavenUtils.java
index fe73e48..b2c798d 100644
--- a/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/util/MavenUtils.java
+++ b/modules/compatibility/src/test/java/org/apache/ignite/compatibility/testframework/util/MavenUtils.java
@@ -40,35 +40,23 @@ public class MavenUtils {
     private static String locRepPath = null;
 
     /**
-     * Gets a path to an artifact with given version and groupId=org.apache.ignite and artifactId=ignite-core.
-     *
+     * Gets a path to an artifact with given version and groupId=org.apache.ignite and artifactId={@code artifactName}.
+     * <br>
      * At first, artifact is looked for in the Maven local repository, if it isn't exists there, it will be downloaded
      * and stored via Maven.
-     *
-     * @param ver Version of ignite-core artifact.
-     * @return Path to the artifact.
-     * @throws Exception In case of an error.
-     * @see #getPathToArtifact(String)
-     */
-    public static String getPathToIgniteCoreArtifact(@NotNull String ver) throws Exception {
-        return getPathToIgniteCoreArtifact(ver, null);
-    }
-
-    /**
-     * Gets a path to an artifact with given version and groupId=org.apache.ignite and artifactId=ignite-core.
-     *
-     * At first, artifact is looked for in the Maven local repository, if it isn't exists there, it will be downloaded
-     * and stored via Maven.
-     *
-     * @param ver Version of ignite-core artifact.
+     * <br>
+     * @param groupName group name, e.g. 'org.apache.ignite'.
+     * @param ver Version of ignite or 3rd party library artifact.
      * @param classifier Artifact classifier.
      * @return Path to the artifact.
      * @throws Exception In case of an error.
      * @see #getPathToArtifact(String)
      */
-    public static String getPathToIgniteCoreArtifact(@NotNull String ver,
+    public static String getPathToIgniteArtifact(@NotNull String groupName,
+        @NotNull String artifactName, @NotNull String ver,
         @Nullable String classifier) throws Exception {
-        String artifact = "org.apache.ignite:ignite-core:" + ver;
+        String artifact = groupName +
+            ":" + artifactName + ":" + ver;
 
         if (classifier != null)
             artifact += ":jar:" + classifier;