You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ak...@apache.org on 2017/03/24 03:46:23 UTC

ignite git commit: IGNITE-4829 Spring context injected into JDBC driver client node.

Repository: ignite
Updated Branches:
  refs/heads/master 746ed2b07 -> 88b61246f


IGNITE-4829 Spring context injected into JDBC driver client node.


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

Branch: refs/heads/master
Commit: 88b61246f503ee6e737004d6b1ff6debe615b191
Parents: 746ed2b
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Fri Mar 24 10:46:56 2017 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Fri Mar 24 10:46:56 2017 +0700

----------------------------------------------------------------------
 modules/clients/pom.xml                         |   7 ++
 .../src/test/config/jdbc-config-cache-store.xml | 124 ++++++++++++++++++
 .../jdbc2/JdbcAbstractDmlStatementSelfTest.java |   3 +
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |  28 +++--
 .../internal/jdbc2/JdbcSpringSelfTest.java      | 125 +++++++++++++++++++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   1 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |  13 +-
 .../testframework/junits/GridAbstractTest.java  |  68 +++++++++-
 8 files changed, 351 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 022edb7..195204c 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -117,6 +117,13 @@
             <version>${spring.version}</version>
             <scope>test</scope>
         </dependency>
+
+        <dependency>
+            <groupId>com.h2database</groupId>
+            <artifactId>h2</artifactId>
+            <version>${h2.version}</version>
+            <scope>test</scope>
+        </dependency>
     </dependencies>
 
     <build>

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/src/test/config/jdbc-config-cache-store.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/config/jdbc-config-cache-store.xml b/modules/clients/src/test/config/jdbc-config-cache-store.xml
new file mode 100644
index 0000000..1e2db3b
--- /dev/null
+++ b/modules/clients/src/test/config/jdbc-config-cache-store.xml
@@ -0,0 +1,124 @@
+<?xml version="1.0" encoding="UTF-8"?>
+
+<!--
+  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.
+-->
+
+<!--
+    Ignite Spring configuration file.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xmlns:util="http://www.springframework.org/schema/util"
+       xsi:schemaLocation="http://www.springframework.org/schema/beans
+                           http://www.springframework.org/schema/beans/spring-beans.xsd
+                           http://www.springframework.org/schema/util
+                           http://www.springframework.org/schema/util/spring-util.xsd">
+
+    <!-- Data source beans for POJO store. -->
+    <bean id="dsTest" class="org.h2.jdbcx.JdbcDataSource">
+        <property name="URL" value="jdbc:h2:mem:demo-db"/>
+        <property name="user" value="sa"/>
+        <property name="password" value=""/>
+    </bean>
+
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <!-- JDBC driver should force true value -->
+        <property name="clientMode" value="false"/>
+
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="discoverySpi">
+            <bean class="org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi">
+                <property name="ipFinder">
+                    <bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder">
+                        <property name="addresses">
+                            <list>
+                                <value>127.0.0.1:47500..47501</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="peerClassLoadingEnabled" value="true"/>
+
+        <property name="cacheConfiguration">
+            <list>
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="custom-cache"/>
+                </bean>
+
+                <bean class="org.apache.ignite.configuration.CacheConfiguration">
+                    <property name="name" value="TestCacheWithStore"/>
+                    <property name="cacheMode" value="PARTITIONED"/>
+                    <property name="atomicityMode" value="ATOMIC"/>
+
+                    <property name="cacheStoreFactory">
+                        <bean class="org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreFactory">
+                            <property name="dataSourceBean" value="dsTest"/>
+                            <property name="dialect">
+                                <bean class="org.apache.ignite.cache.store.jdbc.dialect.H2Dialect"/>
+                            </property>
+
+                            <property name="types">
+                                <list>
+                                    <bean class="org.apache.ignite.cache.store.jdbc.JdbcType">
+                                        <property name="cacheName" value="TestCacheWithStore"/>
+                                        <property name="keyType" value="java.lang.Integer"/>
+                                        <property name="valueType" value="java.lang.String"/>
+                                        <property name="databaseSchema" value="public"/>
+                                        <property name="databaseTable" value="city"/>
+
+                                        <property name="keyFields">
+                                            <list>
+                                                <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
+                                                    <constructor-arg>
+                                                        <util:constant static-field="java.sql.Types.INTEGER"/>
+                                                    </constructor-arg>
+                                                    <constructor-arg value="id"/>
+                                                    <constructor-arg value="int"/>
+                                                    <constructor-arg value="id"/>
+                                                </bean>
+                                            </list>
+                                        </property>
+
+                                        <property name="valueFields">
+                                            <list>
+                                                <bean class="org.apache.ignite.cache.store.jdbc.JdbcTypeField">
+                                                    <constructor-arg>
+                                                        <util:constant static-field="java.sql.Types.VARCHAR"/>
+                                                    </constructor-arg>
+                                                    <constructor-arg value="name"/>
+                                                    <constructor-arg value="java.lang.String"/>
+                                                    <constructor-arg value="name"/>
+                                                </bean>
+                                            </list>
+                                        </property>
+                                    </bean>
+                                </list>
+                            </property>
+                        </bean>
+                    </property>
+
+                    <property name="readThrough" value="true"/>
+                    <property name="writeThrough" value="true"/>
+                </bean>
+            </list>
+        </property>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
index afaadd1..440f6d0 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcAbstractDmlStatementSelfTest.java
@@ -150,6 +150,9 @@ public abstract class JdbcAbstractDmlStatementSelfTest extends GridCommonAbstrac
         grid(0).cache(null).clear();
 
         assertEquals(0, grid(0).cache(null).size(CachePeekMode.ALL));
+
+        conn.close();
+        assertTrue(conn.isClosed());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
index 6c44076..15826fb 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
@@ -44,9 +44,6 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
     /** Custom cache name. */
     private static final String CUSTOM_CACHE_NAME = "custom-cache";
 
-    /** Ignite configuration URL. */
-    private static final String CFG_URL = "modules/clients/src/test/config/jdbc-config.xml";
-
     /** Grid count. */
     private static final int GRID_CNT = 2;
 
@@ -56,6 +53,13 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
     /** Client node flag. */
     private boolean client;
 
+    /**
+     * @return Config URL to use in test.
+     */
+    protected String configURL() {
+        return "modules/clients/src/test/config/jdbc-config.xml";
+    }
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
@@ -104,7 +108,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testDefaults() throws Exception {
-        String url = CFG_URL_PREFIX + CFG_URL;
+        String url = CFG_URL_PREFIX + configURL();
 
         try (Connection conn = DriverManager.getConnection(url)) {
             assertNotNull(conn);
@@ -121,13 +125,13 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testNodeId() throws Exception {
-        String url = CFG_URL_PREFIX + "nodeId=" + grid(0).localNode().id() + '@' + CFG_URL;
+        String url = CFG_URL_PREFIX + "nodeId=" + grid(0).localNode().id() + '@' + configURL();
 
         try (Connection conn = DriverManager.getConnection(url)) {
             assertNotNull(conn);
         }
 
-        url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + ":nodeId=" + grid(0).localNode().id() + '@' + CFG_URL;
+        url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + ":nodeId=" + grid(0).localNode().id() + '@' + configURL();
 
         try (Connection conn = DriverManager.getConnection(url)) {
             assertNotNull(conn);
@@ -140,7 +144,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
     public void testWrongNodeId() throws Exception {
         UUID wrongId = UUID.randomUUID();
 
-        final String url = CFG_URL_PREFIX + "nodeId=" + wrongId + '@' + CFG_URL;
+        final String url = CFG_URL_PREFIX + "nodeId=" + wrongId + '@' + configURL();
 
         GridTestUtils.assertThrows(
                 log,
@@ -166,7 +170,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
 
         UUID clientId = client.localNode().id();
 
-        final String url = CFG_URL_PREFIX + "nodeId=" + clientId + '@' + CFG_URL;
+        final String url = CFG_URL_PREFIX + "nodeId=" + clientId + '@' + configURL();
 
         GridTestUtils.assertThrows(
                 log,
@@ -192,7 +196,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
 
         UUID daemonId = daemon.localNode().id();
 
-        final String url = CFG_URL_PREFIX + "nodeId=" + daemonId + '@' + CFG_URL;
+        final String url = CFG_URL_PREFIX + "nodeId=" + daemonId + '@' + configURL();
 
         GridTestUtils.assertThrows(
             log,
@@ -212,7 +216,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCustomCache() throws Exception {
-        String url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + '@' + CFG_URL;
+        String url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + '@' + configURL();
 
         try (Connection conn = DriverManager.getConnection(url)) {
             assertNotNull(conn);
@@ -223,7 +227,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testWrongCache() throws Exception {
-        final String url = CFG_URL_PREFIX + "cache=wrongCacheName@" + CFG_URL;
+        final String url = CFG_URL_PREFIX + "cache=wrongCacheName@" + configURL();
 
         GridTestUtils.assertThrows(
             log,
@@ -243,7 +247,7 @@ public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testClose() throws Exception {
-        String url = CFG_URL_PREFIX + CFG_URL;
+        String url = CFG_URL_PREFIX + configURL();
 
         try(final Connection conn = DriverManager.getConnection(url)) {
             assertNotNull(conn);

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcSpringSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcSpringSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcSpringSelfTest.java
new file mode 100644
index 0000000..fb03ae8
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcSpringSelfTest.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.jdbc2;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.UUID;
+import java.util.concurrent.Callable;
+import javax.sql.DataSource;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteComponentType;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.processors.resource.GridResourceIoc;
+import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
+import org.apache.ignite.resources.SpringApplicationContextResource;
+import org.apache.ignite.testframework.GridTestUtils;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+
+/**
+ * Test of cluster and JDBC driver with config that contains cache with POJO store and datasource bean.
+ */
+public class JdbcSpringSelfTest extends JdbcConnectionSelfTest {
+    /** Grid count. */
+    private static final int GRID_CNT = 2;
+
+    /** {@inheritDoc} */
+    @Override protected String configURL() {
+        return "modules/clients/src/test/config/jdbc-config-cache-store.xml";
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setMarshaller(null); // In this test we are using default Marshaller.
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsWithSpringCtx(GRID_CNT, false, configURL());
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override public void testClientNodeId() throws Exception {
+        IgniteEx client = (IgniteEx) startGridWithSpringCtx(getTestIgniteInstanceName(), true, configURL());
+
+        UUID clientId = client.localNode().id();
+
+        final String url = CFG_URL_PREFIX + "nodeId=" + clientId + '@' + configURL();
+
+        GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        try (Connection conn = DriverManager.getConnection(url)) {
+                            return conn;
+                        }
+                    }
+                },
+                SQLException.class,
+                "Failed to establish connection with node (is it a server node?): " + clientId
+        );
+    }
+
+    /**
+     * Special class to test Spring context injection.
+     */
+    private static class TestInjectTarget {
+        /** */
+        @SpringApplicationContextResource
+        private Object appCtx;
+    }
+
+    /**
+     * Test that we have valid Spring context and also could create beans from it.
+     *
+     * @throws Exception If test failed.
+     */
+    public void testSpringBean() throws Exception {
+        String url = CFG_URL_PREFIX + configURL();
+
+        // Create connection.
+        try (Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+
+            TestInjectTarget target = new TestInjectTarget();
+
+            IgniteKernal kernal = (IgniteKernal)((JdbcConnection)conn).ignite();
+
+            // Inject Spring context to test object.
+            kernal.context().resource().inject(target, GridResourceIoc.AnnotationSet.GENERIC);
+
+            assertNotNull(target.appCtx);
+
+            IgniteSpringHelper spring = IgniteComponentType.SPRING.create(false);
+
+            // Load bean by name.
+            DataSource ds = spring.loadBeanFromAppContext(target.appCtx, "dsTest");
+
+            assertNotNull(ds);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
index 7395fcb..85e7d90 100644
--- a/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/suite/IgniteJdbcDriverTestSuite.java
@@ -52,6 +52,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
 
         // Ignite client node based driver tests
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcConnectionSelfTest.class));
+        suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcSpringSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcPreparedStatementSelfTest.class));
         suite.addTest(new TestSuite(org.apache.ignite.internal.jdbc2.JdbcResultSetSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
index 18c13ca..2220bfe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc2/JdbcConnection.java
@@ -219,8 +219,12 @@ public class JdbcConnection implements Connection {
 
                             ignite = Ignition.start();
                         }
-                        else
-                            ignite = Ignition.start(loadConfiguration(cfgUrl));
+                        else {
+                            IgniteBiTuple<IgniteConfiguration, ? extends GridSpringResourceContext> cfgAndCtx =
+                                loadConfiguration(cfgUrl);
+
+                            ignite = IgnitionEx.start(cfgAndCtx.get1(), cfgAndCtx.get2());
+                        }
 
                         fut.onDone(ignite);
                     }
@@ -241,8 +245,9 @@ public class JdbcConnection implements Connection {
 
     /**
      * @param cfgUrl Config URL.
+     * @return Ignite config and Spring context.
      */
-    private IgniteConfiguration loadConfiguration(String cfgUrl) {
+    private IgniteBiTuple<IgniteConfiguration, ? extends GridSpringResourceContext> loadConfiguration(String cfgUrl) {
         try {
             IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> cfgMap =
                 IgnitionEx.loadConfigurations(cfgUrl);
@@ -254,7 +259,7 @@ public class JdbcConnection implements Connection {
 
             cfg.setClientMode(true); // Force client mode.
 
-            return cfg;
+            return new IgniteBiTuple<>(cfg, cfgMap.getValue());
         }
         catch (IgniteCheckedException e) {
             throw new IgniteException(e);

http://git-wip-us.apache.org/repos/asf/ignite/blob/88b61246/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 5a76ddd..ddcf91f 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
@@ -71,6 +71,7 @@ import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.G;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.marshaller.Marshaller;
@@ -828,6 +829,68 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /**
+     * Starts new grid with given name.
+     *
+     * @param gridName Grid name.
+     * @param client Client mode.
+     * @param cfgUrl Config URL.
+     * @return Started grid.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithSpringCtx(String gridName, boolean client, String cfgUrl) throws Exception {
+        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> cfgMap =
+                IgnitionEx.loadConfigurations(cfgUrl);
+
+        IgniteConfiguration cfg = F.first(cfgMap.get1());
+
+        cfg.setIgniteInstanceName(gridName);
+        cfg.setClientMode(client);
+
+        return IgnitionEx.start(cfg, cfgMap.getValue());
+    }
+
+    /**
+     * Starts new node with given index.
+     *
+     * @param idx Index of the node to start.
+     * @param client Client mode.
+     * @param cfgUrl Config URL.
+     * @return Started node.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridWithSpringCtx(int idx, boolean client, String cfgUrl) throws Exception {
+        return startGridWithSpringCtx(getTestIgniteInstanceName(idx), client, cfgUrl);
+    }
+
+    /**
+     * Start specified amount of nodes.
+     *
+     * @param cnt Nodes count.
+     * @param client Client mode.
+     * @param cfgUrl Config URL.
+     * @return First started node.
+     * @throws Exception If failed.
+     */
+    protected Ignite startGridsWithSpringCtx(int cnt, boolean client, String cfgUrl) throws Exception {
+        assert cnt > 0;
+
+        Ignite ignite = null;
+
+        for (int i = 0; i < cnt; i++) {
+            if (ignite == null)
+                ignite = startGridWithSpringCtx(i, client, cfgUrl);
+            else
+                startGridWithSpringCtx(i, client, cfgUrl);
+        }
+
+        checkTopology(cnt);
+
+        assert ignite != null;
+
+        return ignite;
+    }
+
+    /**
      * Starts new grid at another JVM with given name.
      *
      * @param igniteInstanceName Ignite instance name.
@@ -1859,7 +1922,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @param store Store.
      */
     protected <T> Factory<T> singletonFactory(T store) {
-        return notSerializableProxy(new FactoryBuilder.SingletonFactory<T>(store), Factory.class);
+        return notSerializableProxy(new FactoryBuilder.SingletonFactory<>(store), Factory.class);
     }
 
     /**
@@ -2179,7 +2242,8 @@ public abstract class GridAbstractTest extends TestCase {
     }
 
     /** */
-    public static abstract class TestIgniteIdxCallable<R> implements Serializable {
+    public abstract static class TestIgniteIdxCallable<R> implements Serializable {
+        /** */
         @IgniteInstanceResource
         protected Ignite ignite;