You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2015/09/15 01:12:59 UTC

[18/41] ignite git commit: ignite-1250 JDBC driver: migration to embedded Ignite client node

ignite-1250 JDBC driver: migration to embedded Ignite client node

Signed-off-by: Valentin Kulichenko <va...@gmail.com>


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

Branch: refs/heads/ignite-1400
Commit: ebb9e2e9d3e05ba65c06ec301bee040b3a74fd3b
Parents: 1ff4a52
Author: Andrey Gura <ag...@gridgain.com>
Authored: Fri Sep 11 18:32:54 2015 -0700
Committer: Valentin Kulichenko <va...@gmail.com>
Committed: Fri Sep 11 18:32:54 2015 -0700

----------------------------------------------------------------------
 modules/clients/pom.xml                         |    7 +
 modules/clients/src/test/config/jdbc-config.xml |   55 +
 .../jdbc2/JdbcComplexQuerySelfTest.java         |  316 ++++
 .../internal/jdbc2/JdbcConnectionSelfTest.java  |  268 +++
 .../internal/jdbc2/JdbcEmptyCacheSelfTest.java  |  140 ++
 .../internal/jdbc2/JdbcLocalCachesSelfTest.java |  156 ++
 .../internal/jdbc2/JdbcMetadataSelfTest.java    |  334 ++++
 .../jdbc2/JdbcPreparedStatementSelfTest.java    |  730 +++++++++
 .../internal/jdbc2/JdbcResultSetSelfTest.java   |  751 +++++++++
 .../internal/jdbc2/JdbcStatementSelfTest.java   |  292 ++++
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   11 +
 .../org/apache/ignite/IgniteJdbcDriver.java     |  281 +++-
 .../apache/ignite/IgniteSystemProperties.java   |    5 +-
 .../ignite/internal/jdbc/JdbcConnection.java    |    4 +
 .../internal/jdbc/JdbcConnectionInfo.java       |   91 --
 .../internal/jdbc/JdbcDatabaseMetadata.java     |    4 +
 .../internal/jdbc/JdbcPreparedStatement.java    |    4 +
 .../ignite/internal/jdbc/JdbcResultSet.java     |    4 +
 .../internal/jdbc/JdbcResultSetMetadata.java    |    4 +
 .../ignite/internal/jdbc/JdbcStatement.java     |    4 +
 .../apache/ignite/internal/jdbc/JdbcUtils.java  |    4 +
 .../ignite/internal/jdbc2/JdbcConnection.java   |  777 +++++++++
 .../internal/jdbc2/JdbcDatabaseMetadata.java    | 1401 ++++++++++++++++
 .../internal/jdbc2/JdbcPreparedStatement.java   |  411 +++++
 .../ignite/internal/jdbc2/JdbcQueryTask.java    |  361 +++++
 .../ignite/internal/jdbc2/JdbcResultSet.java    | 1520 ++++++++++++++++++
 .../internal/jdbc2/JdbcResultSetMetadata.java   |  171 ++
 .../ignite/internal/jdbc2/JdbcStatement.java    |  456 ++++++
 .../apache/ignite/internal/jdbc2/JdbcUtils.java |  155 ++
 .../resources/META-INF/classnames.properties    |   12 +-
 .../yardstick/config/benchmark-query.properties |    5 +-
 modules/yardstick/config/ignite-base-config.xml |    2 +-
 modules/yardstick/config/ignite-jdbc-config.xml |   55 +
 33 files changed, 8606 insertions(+), 185 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/pom.xml
----------------------------------------------------------------------
diff --git a/modules/clients/pom.xml b/modules/clients/pom.xml
index 61f6694..6e690dc 100644
--- a/modules/clients/pom.xml
+++ b/modules/clients/pom.xml
@@ -56,6 +56,13 @@
 
         <dependency>
             <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
+            <version>${project.version}</version>
+            <scope>test</scope>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.ignite</groupId>
             <artifactId>ignite-indexing</artifactId>
             <version>${project.version}</version>
             <scope>test</scope>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/config/jdbc-config.xml
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/config/jdbc-config.xml b/modules/clients/src/test/config/jdbc-config.xml
new file mode 100644
index 0000000..980eaf1
--- /dev/null
+++ b/modules/clients/src/test/config/jdbc-config.xml
@@ -0,0 +1,55 @@
+<?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"
+       xsi:schemaLocation="
+        http://www.springframework.org/schema/beans
+        http://www.springframework.org/schema/beans/spring-beans.xsd">
+    <bean id="grid.cfg" class="org.apache.ignite.configuration.IgniteConfiguration">
+        <property name="clientMode" value="true"/>
+
+        <property name="localHost" value="127.0.0.1"/>
+
+        <property name="marshaller">
+            <bean class="org.apache.ignite.marshaller.optimized.OptimizedMarshaller">
+                <property name="requireSerializable" value="false"/>
+            </bean>
+        </property>
+
+        <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..47549</value>
+                            </list>
+                        </property>
+                    </bean>
+                </property>
+            </bean>
+        </property>
+
+        <property name="peerClassLoadingEnabled" value="true"/>
+    </bean>
+</beans>

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcComplexQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcComplexQuerySelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcComplexQuerySelfTest.java
new file mode 100644
index 0000000..d126d34
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcComplexQuerySelfTest.java
@@ -0,0 +1,316 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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.io.*;
+import java.sql.*;
+
+import static org.apache.ignite.IgniteJdbcDriver.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests for complex queries (joins, etc.).
+ */
+public class JdbcComplexQuerySelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** JDBC URL. */
+    private static final String BASE_URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** Statement. */
+    private Statement stmt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setAtomicityMode(TRANSACTIONAL);
+        cache.setIndexedTypes(String.class, Organization.class, AffinityKey.class, Person.class);
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrids(3);
+
+        IgniteCache<String, Organization> orgCache = grid(0).cache(null);
+
+        assert orgCache != null;
+
+        orgCache.put("o1", new Organization(1, "A"));
+        orgCache.put("o2", new Organization(2, "B"));
+
+        IgniteCache<AffinityKey<String>, Person> personCache = grid(0).cache(null);
+
+        assert personCache != null;
+
+        personCache.put(new AffinityKey<>("p1", "o1"), new Person(1, "John White", 25, 1));
+        personCache.put(new AffinityKey<>("p2", "o1"), new Person(2, "Joe Black", 35, 1));
+        personCache.put(new AffinityKey<>("p3", "o2"), new Person(3, "Mike Green", 40, 2));
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stmt = DriverManager.getConnection(BASE_URL).createStatement();
+
+        assert stmt != null;
+        assert !stmt.isClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (stmt != null) {
+            stmt.getConnection().close();
+            stmt.close();
+
+            assert stmt.isClosed();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoin() throws Exception {
+        ResultSet rs = stmt.executeQuery(
+            "select p.id, p.name, o.name as orgName from Person p, Organization o where p.orgId = o.id");
+
+        assert rs != null;
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            int id = rs.getInt("id");
+
+            if (id == 1) {
+                assert "John White".equals(rs.getString("name"));
+                assert "A".equals(rs.getString("orgName"));
+            }
+            else if (id == 2) {
+                assert "Joe Black".equals(rs.getString("name"));
+                assert "A".equals(rs.getString("orgName"));
+            }
+            else if (id == 3) {
+                assert "Mike Green".equals(rs.getString("name"));
+                assert "B".equals(rs.getString("orgName"));
+            }
+            else
+                assert false : "Wrong ID: " + id;
+
+            cnt++;
+        }
+
+        assert cnt == 3;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWithoutAlias() throws Exception {
+        ResultSet rs = stmt.executeQuery(
+            "select p.id, p.name, o.name from Person p, Organization o where p.orgId = o.id");
+
+        assert rs != null;
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            int id = rs.getInt(1);
+
+            if (id == 1) {
+                assert "John White".equals(rs.getString("name"));
+                assert "John White".equals(rs.getString(2));
+                assert "A".equals(rs.getString(3));
+            }
+            else if (id == 2) {
+                assert "Joe Black".equals(rs.getString("name"));
+                assert "Joe Black".equals(rs.getString(2));
+                assert "A".equals(rs.getString(3));
+            }
+            else if (id == 3) {
+                assert "Mike Green".equals(rs.getString("name"));
+                assert "Mike Green".equals(rs.getString(2));
+                assert "B".equals(rs.getString(3));
+            }
+            else
+                assert false : "Wrong ID: " + id;
+
+            cnt++;
+        }
+
+        assert cnt == 3;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIn() throws Exception {
+        ResultSet rs = stmt.executeQuery("select name from Person where age in (25, 35)");
+
+        assert rs != null;
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assert "John White".equals(rs.getString("name")) ||
+                "Joe Black".equals(rs.getString("name"));
+
+            cnt++;
+        }
+
+        assert cnt == 2;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBetween() throws Exception {
+        ResultSet rs = stmt.executeQuery("select name from Person where age between 24 and 36");
+
+        assert rs != null;
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assert "John White".equals(rs.getString("name")) ||
+                "Joe Black".equals(rs.getString("name"));
+
+            cnt++;
+        }
+
+        assert cnt == 2;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCalculatedValue() throws Exception {
+        ResultSet rs = stmt.executeQuery("select age * 2 from Person");
+
+        assert rs != null;
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assert rs.getInt(1) == 50 ||
+                rs.getInt(1) == 70 ||
+                rs.getInt(1) == 80;
+
+            cnt++;
+        }
+
+        assert cnt == 3;
+    }
+
+    /**
+     * Person.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class Person implements Serializable {
+        /** ID. */
+        @QuerySqlField
+        private final int id;
+
+        /** Name. */
+        @QuerySqlField(index = false)
+        private final String name;
+
+        /** Age. */
+        @QuerySqlField
+        private final int age;
+
+        /** Organization ID. */
+        @QuerySqlField
+        private final int orgId;
+
+        /**
+         * @param id ID.
+         * @param name Name.
+         * @param age Age.
+         * @param orgId Organization ID.
+         */
+        private Person(int id, String name, int age, int orgId) {
+            assert !F.isEmpty(name);
+            assert age > 0;
+            assert orgId > 0;
+
+            this.id = id;
+            this.name = name;
+            this.age = age;
+            this.orgId = orgId;
+        }
+    }
+
+    /**
+     * Organization.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class Organization implements Serializable {
+        /** ID. */
+        @QuerySqlField
+        private final int id;
+
+        /** Name. */
+        @QuerySqlField(index = false)
+        private final String name;
+
+        /**
+         * @param id ID.
+         * @param name Name.
+         */
+        private Organization(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/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
new file mode 100644
index 0000000..951890e
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcConnectionSelfTest.java
@@ -0,0 +1,268 @@
+/*
+ * 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 org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+
+/**
+ * Connection test.
+ */
+public class JdbcConnectionSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** 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;
+
+    /** Daemon node flag. */
+    private boolean daemon;
+
+    /** Client node flag. */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(cacheConfiguration(null), cacheConfiguration(CUSTOM_CACHE_NAME));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setDaemon(daemon);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     * @throws Exception In case of error.
+     */
+    private CacheConfiguration cacheConfiguration(@Nullable String name) throws Exception {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(name);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(GRID_CNT);
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaults() throws Exception {
+        String url = CFG_URL_PREFIX + CFG_URL;
+
+        try (Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+        }
+
+        try (Connection conn = DriverManager.getConnection(url + '/')) {
+            assertNotNull(conn);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNodeId() throws Exception {
+        String url = CFG_URL_PREFIX + "nodeId=" + grid(0).localNode().id() + '@' + CFG_URL;
+
+        try (Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+        }
+
+        url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + ":nodeId=" + grid(0).localNode().id() + '@' + CFG_URL;
+
+        try (Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWrongNodeId() throws Exception {
+        UUID wrongId = UUID.randomUUID();
+
+        final String url = CFG_URL_PREFIX + "nodeId=" + wrongId + '@' + CFG_URL;
+
+        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?): " + wrongId
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientNodeId() throws Exception {
+        client = true;
+
+        IgniteEx client = (IgniteEx)startGrid();
+
+        UUID clientId = client.localNode().id();
+
+        final String url = CFG_URL_PREFIX + "nodeId=" + clientId + '@' + CFG_URL;
+
+        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
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDaemonNodeId() throws Exception {
+        daemon = true;
+
+        IgniteEx daemon = startGrid(GRID_CNT);
+
+        UUID daemonId = daemon.localNode().id();
+
+        final String url = CFG_URL_PREFIX + "nodeId=" + daemonId + '@' + CFG_URL;
+
+        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?): " + daemonId
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCustomCache() throws Exception {
+        String url = CFG_URL_PREFIX + "cache=" + CUSTOM_CACHE_NAME + '@' + CFG_URL;
+
+        try (Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWrongCache() throws Exception {
+        final String url = CFG_URL_PREFIX + "cache=wrongCacheName@" + CFG_URL;
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    try (Connection conn = DriverManager.getConnection(url)) {
+                        return conn;
+                    }
+                }
+            },
+            SQLException.class,
+            "Client is invalid. Probably cache name is wrong."
+        );
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClose() throws Exception {
+        String url = CFG_URL_PREFIX + CFG_URL;
+
+        try(final Connection conn = DriverManager.getConnection(url)) {
+            assertNotNull(conn);
+            assertFalse(conn.isClosed());
+
+            conn.close();
+
+            assertTrue(conn.isClosed());
+
+            GridTestUtils.assertThrows(
+                log,
+                new Callable<Object>() {
+                    @Override public Object call() throws Exception {
+                        conn.isValid(2);
+
+                        return null;
+                    }
+                },
+                SQLException.class,
+                "Connection is closed."
+            );
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcEmptyCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcEmptyCacheSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcEmptyCacheSelfTest.java
new file mode 100644
index 0000000..adf1368
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcEmptyCacheSelfTest.java
@@ -0,0 +1,140 @@
+/*
+ * 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 org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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.sql.*;
+
+import static org.apache.ignite.IgniteJdbcDriver.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Tests for empty cache.
+ */
+public class JdbcEmptyCacheSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** JDBC URL. */
+    private static final String BASE_URL =
+        CFG_URL_PREFIX + "cache=" + CACHE_NAME + "@modules/clients/src/test/config/jdbc-config.xml";
+
+    /** Statement. */
+    private Statement stmt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cache = defaultCacheConfiguration();
+
+        cache.setName(CACHE_NAME);
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(
+            Byte.class, Byte.class
+        );
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGrid();
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stmt = DriverManager.getConnection(BASE_URL).createStatement();
+
+        assert stmt != null;
+        assert !stmt.isClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (stmt != null) {
+            stmt.getConnection().close();
+            stmt.close();
+
+            assert stmt.isClosed();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSelectNumber() throws Exception {
+        ResultSet rs = stmt.executeQuery("select 1");
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assert rs.getInt(1) == 1;
+            assert "1".equals(rs.getString(1));
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSelectString() throws Exception {
+        ResultSet rs = stmt.executeQuery("select 'str'");
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            assertEquals("str", rs.getString(1));
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
new file mode 100644
index 0000000..a8988f9
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcLocalCachesSelfTest.java
@@ -0,0 +1,156 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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.sql.*;
+import java.util.*;
+
+import static org.apache.ignite.IgniteJdbcDriver.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Test JDBC with several local caches.
+ */
+public class JdbcLocalCachesSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "cache";
+
+    /** JDBC URL. */
+    private static final String BASE_URL =
+        CFG_URL_PREFIX + "cache=" + CACHE_NAME + "@modules/clients/src/test/config/jdbc-config.xml";
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration cache = defaultCacheConfiguration();
+
+        cache.setName(CACHE_NAME);
+        cache.setCacheMode(LOCAL);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(
+            String.class, Integer.class
+        );
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(2);
+
+        IgniteCache<Object, Object> cache1 = grid(0).cache(CACHE_NAME);
+
+        assert cache1 != null;
+
+        cache1.put("key1", 1);
+        cache1.put("key2", 2);
+
+        IgniteCache<Object, Object> cache2 = grid(1).cache(CACHE_NAME);
+
+        assert cache2 != null;
+
+        cache2.put("key1", 3);
+        cache2.put("key2", 4);
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCache1() throws Exception {
+        Properties cfg = new Properties();
+
+        cfg.setProperty(PROP_NODE_ID, grid(0).localNode().id().toString());
+
+        Connection conn = null;
+
+        try {
+            conn = DriverManager.getConnection(BASE_URL, cfg);
+
+            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
+
+            int cnt = 0;
+
+            while (rs.next())
+                assertEquals(++cnt, rs.getInt(1));
+
+            assertEquals(2, cnt);
+        }
+        finally {
+            if (conn != null)
+                conn.close();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testCache2() throws Exception {
+        Properties cfg = new Properties();
+
+        cfg.setProperty(PROP_NODE_ID, grid(1).localNode().id().toString());
+
+        Connection conn = null;
+
+        try {
+            conn = DriverManager.getConnection(BASE_URL, cfg);
+
+            ResultSet rs = conn.createStatement().executeQuery("select _val from Integer order by _val");
+
+            int cnt = 0;
+
+            while (rs.next())
+                assertEquals(++cnt + 2, rs.getInt(1));
+
+            assertEquals(2, cnt);
+        }
+        finally {
+            if (conn != null)
+                conn.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
new file mode 100644
index 0000000..f601dbc
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcMetadataSelfTest.java
@@ -0,0 +1,334 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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.io.*;
+import java.sql.*;
+import java.util.*;
+
+import static java.sql.Types.*;
+import static org.apache.ignite.IgniteJdbcDriver.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Metadata tests.
+ */
+public class JdbcMetadataSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** JDBC URL. */
+    private static final String BASE_URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(String.class, Organization.class, AffinityKey.class, Person.class);
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+
+        IgniteCache<String, Organization> orgCache = grid(0).cache(null);
+
+        orgCache.put("o1", new Organization(1, "A"));
+        orgCache.put("o2", new Organization(2, "B"));
+
+        IgniteCache<AffinityKey<String>, Person> personCache = grid(0).cache(null);
+
+        personCache.put(new AffinityKey<>("p1", "o1"), new Person("John White", 25, 1));
+        personCache.put(new AffinityKey<>("p2", "o1"), new Person("Joe Black", 35, 1));
+        personCache.put(new AffinityKey<>("p3", "o2"), new Person("Mike Green", 40, 2));
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testResultSetMetaData() throws Exception {
+        try (Connection conn = DriverManager.getConnection(BASE_URL)) {
+            Statement stmt = conn.createStatement();
+
+            ResultSet rs = stmt.executeQuery(
+                "select p.name, o.id as orgId from Person p, Organization o where p.orgId = o.id");
+
+            assertNotNull(rs);
+
+            ResultSetMetaData meta = rs.getMetaData();
+
+            assertNotNull(meta);
+
+            assertEquals(2, meta.getColumnCount());
+
+            assertTrue("Person".equalsIgnoreCase(meta.getTableName(1)));
+            assertTrue("name".equalsIgnoreCase(meta.getColumnName(1)));
+            assertTrue("name".equalsIgnoreCase(meta.getColumnLabel(1)));
+            assertEquals(VARCHAR, meta.getColumnType(1));
+            assertEquals("VARCHAR", meta.getColumnTypeName(1));
+            assertEquals("java.lang.String", meta.getColumnClassName(1));
+
+            assertTrue("Organization".equalsIgnoreCase(meta.getTableName(2)));
+            assertTrue("orgId".equalsIgnoreCase(meta.getColumnName(2)));
+            assertTrue("orgId".equalsIgnoreCase(meta.getColumnLabel(2)));
+            assertEquals(INTEGER, meta.getColumnType(2));
+            assertEquals("INTEGER", meta.getColumnTypeName(2));
+            assertEquals("java.lang.Integer", meta.getColumnClassName(2));
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetTables() throws Exception {
+        try (Connection conn = DriverManager.getConnection(BASE_URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            Collection<String> names = new ArrayList<>(2);
+
+            names.add("PERSON");
+            names.add("ORGANIZATION");
+
+            ResultSet rs = meta.getTables("", "PUBLIC", "%", new String[]{"TABLE"});
+
+            assertNotNull(rs);
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                assertEquals("TABLE", rs.getString("TABLE_TYPE"));
+                assertTrue(names.remove(rs.getString("TABLE_NAME")));
+
+                cnt++;
+            }
+
+            assertTrue(names.isEmpty());
+            assertEquals(2, cnt);
+
+            names.add("PERSON");
+            names.add("ORGANIZATION");
+
+            rs = meta.getTables("", "PUBLIC", "%", null);
+
+            assertNotNull(rs);
+
+            cnt = 0;
+
+            while (rs.next()) {
+                assertEquals("TABLE", rs.getString("TABLE_TYPE"));
+                assertTrue(names.remove(rs.getString("TABLE_NAME")));
+
+                cnt++;
+            }
+
+            assertTrue(names.isEmpty());
+            assertEquals(2, cnt);
+
+            rs = meta.getTables("", "PUBLIC", "", new String[]{"WRONG"});
+
+            assertFalse(rs.next());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testGetColumns() throws Exception {
+        try (Connection conn = DriverManager.getConnection(BASE_URL)) {
+            DatabaseMetaData meta = conn.getMetaData();
+
+            ResultSet rs = meta.getColumns("", "PUBLIC", "Person", "%");
+
+            assertNotNull(rs);
+
+            Collection<String> names = new ArrayList<>(2);
+
+            names.add("NAME");
+            names.add("AGE");
+            names.add("ORGID");
+            names.add("_KEY");
+            names.add("_VAL");
+
+            int cnt = 0;
+
+            while (rs.next()) {
+                String name = rs.getString("COLUMN_NAME");
+
+                assertTrue(names.remove(name));
+
+                if ("NAME".equals(name)) {
+                    assertEquals(VARCHAR, rs.getInt("DATA_TYPE"));
+                    assertEquals("VARCHAR", rs.getString("TYPE_NAME"));
+                    assertEquals(1, rs.getInt("NULLABLE"));
+                } else if ("AGE".equals(name) || "ORGID".equals(name)) {
+                    assertEquals(INTEGER, rs.getInt("DATA_TYPE"));
+                    assertEquals("INTEGER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                }
+                if ("_KEY".equals(name)) {
+                    assertEquals(OTHER, rs.getInt("DATA_TYPE"));
+                    assertEquals("OTHER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                }
+                if ("_VAL".equals(name)) {
+                    assertEquals(OTHER, rs.getInt("DATA_TYPE"));
+                    assertEquals("OTHER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                }
+
+                cnt++;
+            }
+
+            assertTrue(names.isEmpty());
+            assertEquals(5, cnt);
+
+            rs = meta.getColumns("", "PUBLIC", "Organization", "%");
+
+            assertNotNull(rs);
+
+            names.add("ID");
+            names.add("NAME");
+            names.add("_KEY");
+            names.add("_VAL");
+
+            cnt = 0;
+
+            while (rs.next()) {
+                String name = rs.getString("COLUMN_NAME");
+
+                assertTrue(names.remove(name));
+
+                if ("id".equals(name)) {
+                    assertEquals(INTEGER, rs.getInt("DATA_TYPE"));
+                    assertEquals("INTEGER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                } else if ("name".equals(name)) {
+                    assertEquals(VARCHAR, rs.getInt("DATA_TYPE"));
+                    assertEquals("VARCHAR", rs.getString("TYPE_NAME"));
+                    assertEquals(1, rs.getInt("NULLABLE"));
+                }
+                if ("_KEY".equals(name)) {
+                    assertEquals(VARCHAR, rs.getInt("DATA_TYPE"));
+                    assertEquals("VARCHAR", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                }
+                if ("_VAL".equals(name)) {
+                    assertEquals(OTHER, rs.getInt("DATA_TYPE"));
+                    assertEquals("OTHER", rs.getString("TYPE_NAME"));
+                    assertEquals(0, rs.getInt("NULLABLE"));
+                }
+
+                cnt++;
+            }
+
+            assertTrue(names.isEmpty());
+            assertEquals(4, cnt);
+        }
+    }
+
+    /**
+     * Person.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class Person implements Serializable {
+        /** Name. */
+        @QuerySqlField(index = false)
+        private final String name;
+
+        /** Age. */
+        @QuerySqlField
+        private final int age;
+
+        /** Organization ID. */
+        @QuerySqlField
+        private final int orgId;
+
+        /**
+         * @param name Name.
+         * @param age Age.
+         * @param orgId Organization ID.
+         */
+        private Person(String name, int age, int orgId) {
+            assert !F.isEmpty(name);
+            assert age > 0;
+            assert orgId > 0;
+
+            this.name = name;
+            this.age = age;
+            this.orgId = orgId;
+        }
+    }
+
+    /**
+     * Organization.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class Organization implements Serializable {
+        /** ID. */
+        @QuerySqlField
+        private final int id;
+
+        /** Name. */
+        @QuerySqlField(index = false)
+        private final String name;
+
+        /**
+         * @param id ID.
+         * @param name Name.
+         */
+        private Organization(int id, String name) {
+            this.id = id;
+            this.name = name;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
new file mode 100644
index 0000000..ea586b2
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcPreparedStatementSelfTest.java
@@ -0,0 +1,730 @@
+/*
+ * 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 org.apache.ignite.*;
+import org.apache.ignite.cache.query.annotations.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+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.io.*;
+import java.math.*;
+import java.net.*;
+import java.sql.*;
+import java.util.Date;
+
+import static java.sql.Types.*;
+import static org.apache.ignite.IgniteJdbcDriver.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ * Prepared statement test.
+ */
+public class JdbcPreparedStatementSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** JDBC URL. */
+    private static final String BASE_URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** Connection. */
+    private Connection conn;
+
+    /** Statement. */
+    private PreparedStatement stmt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(
+            Integer.class, TestObject.class
+        );
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+
+        IgniteCache<Integer, TestObject> cache = grid(0).cache(null);
+
+        assert cache != null;
+
+        TestObject o = new TestObject(1);
+
+        o.boolVal = true;
+        o.byteVal = 1;
+        o.shortVal = 1;
+        o.intVal = 1;
+        o.longVal = 1L;
+        o.floatVal = 1.0f;
+        o.doubleVal = 1.0d;
+        o.bigVal = new BigDecimal(1);
+        o.strVal = "str";
+        o.arrVal = new byte[] {1};
+        o.dateVal = new Date(1);
+        o.timeVal = new Time(1);
+        o.tsVal = new Timestamp(1);
+        o.urlVal = new URL("http://abc.com/");
+
+        cache.put(1, o);
+        cache.put(2, new TestObject(2));
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        conn = DriverManager.getConnection(BASE_URL);
+
+        assert conn != null;
+        assert !conn.isClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (stmt != null) {
+            stmt.close();
+
+            assert stmt.isClosed();
+        }
+
+        if (conn != null) {
+            conn.close();
+
+            assert conn.isClosed();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBoolean() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where boolVal is not distinct from ?");
+
+        stmt.setBoolean(1, true);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, BOOLEAN);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByte() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where byteVal is not distinct from ?");
+
+        stmt.setByte(1, (byte)1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, TINYINT);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where shortVal is not distinct from ?");
+
+        stmt.setShort(1, (short)1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, SMALLINT);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInteger() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where intVal is not distinct from ?");
+
+        stmt.setInt(1, 1);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, INTEGER);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where longVal is not distinct from ?");
+
+        stmt.setLong(1, 1L);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, BIGINT);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where floatVal is not distinct from ?");
+
+        stmt.setFloat(1, 1.0f);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, FLOAT);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where doubleVal is not distinct from ?");
+
+        stmt.setDouble(1, 1.0d);
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, DOUBLE);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBigDecimal() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where bigVal is not distinct from ?");
+
+        stmt.setBigDecimal(1, new BigDecimal(1));
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, OTHER);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testString() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where strVal is not distinct from ?");
+
+        stmt.setString(1, "str");
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, VARCHAR);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testArray() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where arrVal is not distinct from ?");
+
+        stmt.setBytes(1, new byte[] {1});
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, BINARY);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDate() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where dateVal is not distinct from ?");
+
+        stmt.setObject(1, new Date(1));
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, DATE);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTime() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where timeVal is not distinct from ?");
+
+        stmt.setTime(1, new Time(1));
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, TIME);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestamp() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where tsVal is not distinct from ?");
+
+        stmt.setTimestamp(1, new Timestamp(1));
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, TIMESTAMP);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUrl() throws Exception {
+        stmt = conn.prepareStatement("select * from TestObject where urlVal is not distinct from ?");
+
+        stmt.setURL(1, new URL("http://abc.com/"));
+
+        ResultSet rs = stmt.executeQuery();
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 1;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+
+        stmt.setNull(1, DATALINK);
+
+        rs = stmt.executeQuery();
+
+        cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0)
+                assert rs.getInt("id") == 2;
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * Test object.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject implements Serializable {
+        /** */
+        @QuerySqlField(index = false)
+        private final int id;
+
+        /** */
+        @QuerySqlField
+        private Boolean boolVal;
+
+        /** */
+        @QuerySqlField
+        private Byte byteVal;
+
+        /** */
+        @QuerySqlField
+        private Short shortVal;
+
+        /** */
+        @QuerySqlField
+        private Integer intVal;
+
+        /** */
+        @QuerySqlField
+        private Long longVal;
+
+        /** */
+        @QuerySqlField
+        private Float floatVal;
+
+        /** */
+        @QuerySqlField
+        private Double doubleVal;
+
+        /** */
+        @QuerySqlField
+        private BigDecimal bigVal;
+
+        /** */
+        @QuerySqlField
+        private String strVal;
+
+        /** */
+        @QuerySqlField
+        private byte[] arrVal;
+
+        /** */
+        @QuerySqlField
+        private Date dateVal;
+
+        /** */
+        @QuerySqlField
+        private Time timeVal;
+
+        /** */
+        @QuerySqlField
+        private Timestamp tsVal;
+
+        /** */
+        @QuerySqlField
+        private URL urlVal;
+
+        /**
+         * @param id ID.
+         */
+        private TestObject(int id) {
+            this.id = id;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/ebb9e2e9/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
new file mode 100644
index 0000000..3607f53
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/jdbc2/JdbcResultSetSelfTest.java
@@ -0,0 +1,751 @@
+/*
+ * 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.io.Serializable;
+import java.math.BigDecimal;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.sql.Date;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.concurrent.Callable;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.query.annotations.QuerySqlField;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.IgniteJdbcDriver.CFG_URL_PREFIX;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ * Result set test.
+ */
+@SuppressWarnings("FloatingPointEquality")
+public class JdbcResultSetSelfTest extends GridCommonAbstractTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** JDBC URL. */
+    private static final String BASE_URL = CFG_URL_PREFIX + "modules/clients/src/test/config/jdbc-config.xml";
+
+    /** SQL query. */
+    private static final String SQL =
+        "select id, boolVal, byteVal, shortVal, intVal, longVal, floatVal, " +
+            "doubleVal, bigVal, strVal, arrVal, dateVal, timeVal, tsVal, urlVal, f1, f2, f3, _val " +
+            "from TestObject where id = 1";
+
+    /** Statement. */
+    private Statement stmt;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        CacheConfiguration<?,?> cache = defaultCacheConfiguration();
+
+        cache.setCacheMode(PARTITIONED);
+        cache.setBackups(1);
+        cache.setWriteSynchronizationMode(FULL_SYNC);
+        cache.setIndexedTypes(
+            Integer.class, TestObject.class
+        );
+
+        cfg.setCacheConfiguration(cache);
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setConnectorConfiguration(new ConnectorConfiguration());
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        startGridsMultiThreaded(3);
+
+        IgniteCache<Integer, TestObject> cache = grid(0).cache(null);
+
+        assert cache != null;
+
+        TestObject o = createObjectWithData(1);
+
+        cache.put(1, o);
+        cache.put(2, new TestObject(2));
+        cache.put(3, new TestObject(3));
+
+        Class.forName("org.apache.ignite.IgniteJdbcDriver");
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stmt = DriverManager.getConnection(BASE_URL).createStatement();
+
+        assert stmt != null;
+        assert !stmt.isClosed();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        if (stmt != null) {
+            stmt.getConnection().close();
+            stmt.close();
+
+            assert stmt.isClosed();
+        }
+    }
+
+    /**
+     * @param id ID.
+     * @return Object.
+     * @throws MalformedURLException If URL in incorrect.
+     */
+    @SuppressWarnings("deprecation")
+    private TestObject createObjectWithData(int id) throws MalformedURLException {
+        TestObject o = new TestObject(id);
+
+        o.boolVal = true;
+        o.byteVal = 1;
+        o.shortVal = 1;
+        o.intVal = 1;
+        o.longVal = 1L;
+        o.floatVal = 1.0f;
+        o.doubleVal = 1.0d;
+        o.bigVal = new BigDecimal(1);
+        o.strVal = "str";
+        o.arrVal = new byte[] {1};
+        o.dateVal = new Date(1, 1, 1);
+        o.timeVal = new Time(1, 1, 1);
+        o.tsVal = new Timestamp(1);
+        o.urlVal = new URL("http://abc.com/");
+
+        return o;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBoolean() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getBoolean("boolVal");
+                assert rs.getBoolean(2);
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testByte() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getByte("byteVal") == 1;
+                assert rs.getByte(3) == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testShort() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getShort("shortVal") == 1;
+                assert rs.getShort(4) == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInteger() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getInt("intVal") == 1;
+                assert rs.getInt(5) == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testLong() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getLong("longVal") == 1;
+                assert rs.getLong(6) == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFloat() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getFloat("floatVal") == 1.0;
+                assert rs.getFloat(7) == 1.0;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDouble() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getDouble("doubleVal") == 1.0;
+                assert rs.getDouble(8) == 1.0;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBigDecimal() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getBigDecimal("bigVal").intValue() == 1;
+                assert rs.getBigDecimal(9).intValue() == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testString() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert "str".equals(rs.getString("strVal"));
+                assert "str".equals(rs.getString(10));
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testArray() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert Arrays.equals(rs.getBytes("arrVal"), new byte[] {1});
+                assert Arrays.equals(rs.getBytes(11), new byte[] {1});
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("deprecation")
+    public void testDate() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getDate("dateVal").equals(new Date(1, 1, 1));
+                assert rs.getDate(12).equals(new Date(1, 1, 1));
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("deprecation")
+    public void testTime() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getTime("timeVal").equals(new Time(1, 1, 1));
+                assert rs.getTime(13).equals(new Time(1, 1, 1));
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTimestamp() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert rs.getTimestamp("tsVal").getTime() == 1;
+                assert rs.getTimestamp(14).getTime() == 1;
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUrl() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        int cnt = 0;
+
+        while (rs.next()) {
+            if (cnt == 0) {
+                assert "http://abc.com/".equals(rs.getURL("urlVal").toString());
+                assert "http://abc.com/".equals(rs.getURL(15).toString());
+            }
+
+            cnt++;
+        }
+
+        assert cnt == 1;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testObject() throws Exception {
+        ResultSet rs = stmt.executeQuery(SQL);
+
+        TestObjectField f1 = new TestObjectField(100, "AAAA");
+        TestObjectField f2 = new TestObjectField(500, "BBBB");
+
+        TestObject o = createObjectWithData(1);
+
+        assertTrue(rs.next());
+
+        assertEquals(f1.toString(), rs.getObject("f1"));
+        assertEquals(f1.toString(), rs.getObject(16));
+
+        assertEquals(f2.toString(), rs.getObject("f2"));
+        assertEquals(f2.toString(), rs.getObject(17));
+
+        assertNull(rs.getObject("f3"));
+        assertTrue(rs.wasNull());
+        assertNull(rs.getObject(18));
+        assertTrue(rs.wasNull());
+
+        assertEquals(o.toString(), rs.getObject("_val"));
+        assertEquals(o.toString(), rs.getObject(19));
+
+        assertFalse(rs.next());
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNavigation() throws Exception {
+        ResultSet rs = stmt.executeQuery("select * from TestObject where id > 0");
+
+        assertTrue(rs.isBeforeFirst());
+        assertFalse(rs.isAfterLast());
+        assertFalse(rs.isFirst());
+        assertFalse(rs.isLast());
+        assertEquals(0, rs.getRow());
+
+        assertTrue(rs.next());
+
+        assertFalse(rs.isBeforeFirst());
+        assertFalse(rs.isAfterLast());
+        assertTrue(rs.isFirst());
+        assertFalse(rs.isLast());
+        assertEquals(1, rs.getRow());
+
+        assertTrue(rs.next());
+
+        assertFalse(rs.isBeforeFirst());
+        assertFalse(rs.isAfterLast());
+        assertFalse(rs.isFirst());
+        assertFalse(rs.isLast());
+        assertEquals(2, rs.getRow());
+
+        assertTrue(rs.next());
+
+        assertFalse(rs.isBeforeFirst());
+        assertFalse(rs.isAfterLast());
+        assertFalse(rs.isFirst());
+        assertTrue(rs.isLast());
+        assertEquals(3, rs.getRow());
+
+        assertFalse(rs.next());
+
+        assertFalse(rs.isBeforeFirst());
+        assertTrue(rs.isAfterLast());
+        assertFalse(rs.isFirst());
+        assertFalse(rs.isLast());
+        assertEquals(0, rs.getRow());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFetchSize() throws Exception {
+        stmt.setFetchSize(1);
+
+        ResultSet rs = stmt.executeQuery("select * from TestObject where id > 0");
+
+        assertTrue(rs.next());
+        assertTrue(rs.next());
+        assertTrue(rs.next());
+
+        stmt.setFetchSize(0);
+    }
+
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testFindColumn() throws Exception {
+        final ResultSet rs = stmt.executeQuery(SQL);
+
+        assertNotNull(rs);
+        assertTrue(rs.next());
+
+        assert rs.findColumn("id") == 1;
+
+        GridTestUtils.assertThrows(
+            log,
+            new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    rs.findColumn("wrong");
+
+                    return null;
+                }
+            },
+            SQLException.class,
+            "Column not found: wrong"
+        );
+    }
+
+    /**
+     * Test object.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    private static class TestObject implements Serializable {
+        /** */
+        @QuerySqlField
+        private final int id;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Boolean boolVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Byte byteVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Short shortVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Integer intVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Long longVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Float floatVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Double doubleVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private BigDecimal bigVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private String strVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private byte[] arrVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Date dateVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Time timeVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private Timestamp tsVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private URL urlVal;
+
+        /** */
+        @QuerySqlField(index = false)
+        private TestObjectField f1 = new TestObjectField(100, "AAAA");
+
+        /** */
+        @QuerySqlField(index = false)
+        private TestObjectField f2 = new TestObjectField(500, "BBBB");
+
+        /** */
+        @QuerySqlField(index = false)
+        private TestObjectField f3;
+
+        /**
+         * @param id ID.
+         */
+        private TestObject(int id) {
+            this.id = id;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestObject.class, this);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings({"BigDecimalEquals", "EqualsHashCodeCalledOnUrl", "RedundantIfStatement"})
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TestObject that = (TestObject)o;
+
+            if (id != that.id) return false;
+            if (!Arrays.equals(arrVal, that.arrVal)) return false;
+            if (bigVal != null ? !bigVal.equals(that.bigVal) : that.bigVal != null) return false;
+            if (boolVal != null ? !boolVal.equals(that.boolVal) : that.boolVal != null) return false;
+            if (byteVal != null ? !byteVal.equals(that.byteVal) : that.byteVal != null) return false;
+            if (dateVal != null ? !dateVal.equals(that.dateVal) : that.dateVal != null) return false;
+            if (doubleVal != null ? !doubleVal.equals(that.doubleVal) : that.doubleVal != null) return false;
+            if (f1 != null ? !f1.equals(that.f1) : that.f1 != null) return false;
+            if (f2 != null ? !f2.equals(that.f2) : that.f2 != null) return false;
+            if (f3 != null ? !f3.equals(that.f3) : that.f3 != null) return false;
+            if (floatVal != null ? !floatVal.equals(that.floatVal) : that.floatVal != null) return false;
+            if (intVal != null ? !intVal.equals(that.intVal) : that.intVal != null) return false;
+            if (longVal != null ? !longVal.equals(that.longVal) : that.longVal != null) return false;
+            if (shortVal != null ? !shortVal.equals(that.shortVal) : that.shortVal != null) return false;
+            if (strVal != null ? !strVal.equals(that.strVal) : that.strVal != null) return false;
+            if (timeVal != null ? !timeVal.equals(that.timeVal) : that.timeVal != null) return false;
+            if (tsVal != null ? !tsVal.equals(that.tsVal) : that.tsVal != null) return false;
+            if (urlVal != null ? !urlVal.equals(that.urlVal) : that.urlVal != null) return false;
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("EqualsHashCodeCalledOnUrl")
+        @Override public int hashCode() {
+            int res = id;
+
+            res = 31 * res + (boolVal != null ? boolVal.hashCode() : 0);
+            res = 31 * res + (byteVal != null ? byteVal.hashCode() : 0);
+            res = 31 * res + (shortVal != null ? shortVal.hashCode() : 0);
+            res = 31 * res + (intVal != null ? intVal.hashCode() : 0);
+            res = 31 * res + (longVal != null ? longVal.hashCode() : 0);
+            res = 31 * res + (floatVal != null ? floatVal.hashCode() : 0);
+            res = 31 * res + (doubleVal != null ? doubleVal.hashCode() : 0);
+            res = 31 * res + (bigVal != null ? bigVal.hashCode() : 0);
+            res = 31 * res + (strVal != null ? strVal.hashCode() : 0);
+            res = 31 * res + (arrVal != null ? Arrays.hashCode(arrVal) : 0);
+            res = 31 * res + (dateVal != null ? dateVal.hashCode() : 0);
+            res = 31 * res + (timeVal != null ? timeVal.hashCode() : 0);
+            res = 31 * res + (tsVal != null ? tsVal.hashCode() : 0);
+            res = 31 * res + (urlVal != null ? urlVal.hashCode() : 0);
+            res = 31 * res + (f1 != null ? f1.hashCode() : 0);
+            res = 31 * res + (f2 != null ? f2.hashCode() : 0);
+            res = 31 * res + (f3 != null ? f3.hashCode() : 0);
+
+            return res;
+        }
+    }
+
+    /**
+     * Test object field.
+     */
+    @SuppressWarnings("PackageVisibleField")
+    private static class TestObjectField implements Serializable {
+        /** */
+        final int a;
+
+        /** */
+        final String b;
+
+        /**
+         * @param a A.
+         * @param b B.
+         */
+        private TestObjectField(int a, String b) {
+            this.a = a;
+            this.b = b;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object o) {
+            if (this == o) return true;
+            if (o == null || getClass() != o.getClass()) return false;
+
+            TestObjectField that = (TestObjectField)o;
+
+            return a == that.a && !(b != null ? !b.equals(that.b) : that.b != null);
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = a;
+
+            res = 31 * res + (b != null ? b.hashCode() : 0);
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(TestObjectField.class, this);
+        }
+    }
+}