You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2018/05/10 08:02:04 UTC

ignite git commit: IGNITE-6140: JDBC thin driver: implemented data source. This closes #2826.

Repository: ignite
Updated Branches:
  refs/heads/master 03a3e927c -> be25a50db


IGNITE-6140: JDBC thin driver: implemented data source. This closes #2826.


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

Branch: refs/heads/master
Commit: be25a50dbbd2c4a28c19d5fe8e81acba8afe00b7
Parents: 03a3e92
Author: tledkov-gridgain <tl...@gridgain.com>
Authored: Thu May 10 11:01:50 2018 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Thu May 10 11:01:50 2018 +0300

----------------------------------------------------------------------
 .../jdbc/suite/IgniteJdbcDriverTestSuite.java   |   2 +
 .../jdbc/thin/JdbcThinDataSourceSelfTest.java   | 429 +++++++++++++
 .../apache/ignite/IgniteJdbcThinDataSource.java | 614 +++++++++++++++++++
 .../org/apache/ignite/IgniteJdbcThinDriver.java |  40 +-
 .../jdbc/thin/ConnectionPropertiesImpl.java     |  20 +-
 5 files changed, 1092 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/be25a50d/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 a88ebe8..c0378b2 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
@@ -48,6 +48,7 @@ import org.apache.ignite.jdbc.thin.JdbcThinComplexQuerySelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionMultipleAddressesTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionSSLTest;
 import org.apache.ignite.jdbc.thin.JdbcThinConnectionSelfTest;
+import org.apache.ignite.jdbc.thin.JdbcThinDataSourceSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDeleteStatementSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDynamicIndexAtomicPartitionedNearSelfTest;
 import org.apache.ignite.jdbc.thin.JdbcThinDynamicIndexAtomicPartitionedSelfTest;
@@ -143,6 +144,7 @@ public class IgniteJdbcDriverTestSuite extends TestSuite {
         suite.addTest(new TestSuite(JdbcThinConnectionMultipleAddressesTest.class));
         suite.addTest(new TestSuite(JdbcThinTcpIoTest.class));
         suite.addTest(new TestSuite(JdbcThinConnectionSSLTest.class));
+        suite.addTest(new TestSuite(JdbcThinDataSourceSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinPreparedStatementSelfTest.class));
         suite.addTest(new TestSuite(JdbcThinResultSetSelfTest.class));
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/be25a50d/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
new file mode 100644
index 0000000..6040bed
--- /dev/null
+++ b/modules/clients/src/test/java/org/apache/ignite/jdbc/thin/JdbcThinDataSourceSelfTest.java
@@ -0,0 +1,429 @@
+/*
+ * 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.jdbc.thin;
+
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.util.HashMap;
+import java.util.Hashtable;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import javax.naming.Binding;
+import javax.naming.Context;
+import javax.naming.InitialContext;
+import javax.naming.Name;
+import javax.naming.NameClassPair;
+import javax.naming.NameParser;
+import javax.naming.NamingEnumeration;
+import javax.naming.NamingException;
+import javax.naming.spi.InitialContextFactory;
+import org.apache.ignite.IgniteJdbcThinDataSource;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.binary.BinaryMarshaller;
+import org.apache.ignite.internal.jdbc.thin.JdbcThinConnection;
+import org.apache.ignite.internal.jdbc.thin.JdbcThinTcpIo;
+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;
+
+/**
+ * DataSource test.
+ */
+@SuppressWarnings("ThrowableNotThrown")
+public class JdbcThinDataSourceSelfTest extends JdbcThinAbstractSelfTest {
+    /** IP finder. */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("deprecation")
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(cacheConfiguration(DEFAULT_CACHE_NAME));
+
+        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+
+        disco.setIpFinder(IP_FINDER);
+
+        cfg.setDiscoverySpi(disco);
+
+        cfg.setMarshaller(new BinaryMarshaller());
+
+        return cfg;
+    }
+
+    /**
+     * @param name Cache name.
+     * @return Cache configuration.
+     * @throws Exception In case of error.
+     */
+    private CacheConfiguration cacheConfiguration(String name) throws Exception {
+        CacheConfiguration cfg = defaultCacheConfiguration();
+
+        cfg.setName(name);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGridsMultiThreaded(2);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJndi() throws Exception {
+        IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1");
+
+        InitialContext ic = getInitialContext();
+
+        ic.bind("ds/test", ids);
+
+        IgniteJdbcThinDataSource ds = (IgniteJdbcThinDataSource)ic.lookup("ds/test");
+
+        assertTrue("Cannot looking up DataSource from JNDI", ds != null);
+
+        assertEquals(ids.getUrl(), ds.getUrl());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUrlCompose() throws Exception {
+        IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setAddresses("127.0.0.1:" + ClientConnectorConfiguration.DFLT_PORT);
+
+        assertEquals("jdbc:ignite:thin://127.0.0.1:10800/PUBLIC", ids.getUrl());
+
+        ids.setSchema("test");
+
+        assertEquals("jdbc:ignite:thin://127.0.0.1:10800/test", ids.getUrl());
+        assertEquals("jdbc:ignite:thin://127.0.0.1:10800/test", ids.getURL());
+
+        ids.setAddresses("127.0.0.1:" + ClientConnectorConfiguration.DFLT_PORT + ".."
+            + (ClientConnectorConfiguration.DFLT_PORT + 10), "myhost:10801..10802");
+
+        ids.setSchema("test2");
+        assertEquals("jdbc:ignite:thin://127.0.0.1:10800..10810,myhost:10801..10802/test2", ids.getURL());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testResetUrl() throws Exception {
+        IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1:10800/test?lazy=true");
+
+        assertEquals("test", ids.getSchema());
+        assertTrue(ids.isLazy());
+
+        ids.setUrl("jdbc:ignite:thin://mydomain.org,localhost?collocated=true");
+
+        assertNull(ids.getSchema());
+        assertFalse(ids.isLazy());
+        assertTrue(ids.isCollocated());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSqlHints() throws Exception {
+        IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1");
+
+        try (Connection conn = ids.getConnection()) {
+            JdbcThinTcpIo io = GridTestUtils.getFieldValue(conn, JdbcThinConnection.class, "cliIo");
+
+            assertFalse(io.connectionProperties().isAutoCloseServerCursor());
+            assertFalse(io.connectionProperties().isCollocated());
+            assertFalse(io.connectionProperties().isEnforceJoinOrder());
+            assertFalse(io.connectionProperties().isLazy());
+            assertFalse(io.connectionProperties().isDistributedJoins());
+            assertFalse(io.connectionProperties().isReplicatedOnly());
+        }
+
+        ids.setAutoCloseServerCursor(true);
+        ids.setCollocated(true);
+        ids.setEnforceJoinOrder(true);
+        ids.setLazy(true);
+        ids.setDistributedJoins(true);
+        ids.setReplicatedOnly(true);
+
+        try (Connection conn = ids.getConnection()) {
+            JdbcThinTcpIo io = GridTestUtils.getFieldValue(conn, JdbcThinConnection.class, "cliIo");
+
+            assertTrue(io.connectionProperties().isAutoCloseServerCursor());
+            assertTrue(io.connectionProperties().isCollocated());
+            assertTrue(io.connectionProperties().isEnforceJoinOrder());
+            assertTrue(io.connectionProperties().isLazy());
+            assertTrue(io.connectionProperties().isDistributedJoins());
+            assertTrue(io.connectionProperties().isReplicatedOnly());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTcpNoDelay() throws Exception {
+        IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1");
+
+        try (Connection conn = ids.getConnection()) {
+            JdbcThinTcpIo io = GridTestUtils.getFieldValue(conn, JdbcThinConnection.class, "cliIo");
+
+            assertTrue(io.connectionProperties().isTcpNoDelay());
+        }
+
+        ids.setTcpNoDelay(false);
+
+        try (Connection conn = ids.getConnection()) {
+            JdbcThinTcpIo io = GridTestUtils.getFieldValue(conn, JdbcThinConnection.class, "cliIo");
+
+            assertFalse(io.connectionProperties().isTcpNoDelay());
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSocketBuffers() throws Exception {
+        final IgniteJdbcThinDataSource ids = new IgniteJdbcThinDataSource();
+
+        ids.setUrl("jdbc:ignite:thin://127.0.0.1");
+        ids.setSocketReceiveBuffer(111);
+        ids.setSocketSendBuffer(111);
+
+        try (Connection conn = ids.getConnection()) {
+            JdbcThinTcpIo io = GridTestUtils.getFieldValue(conn, JdbcThinConnection.class, "cliIo");
+
+            assertEquals(111, io.connectionProperties().getSocketReceiveBuffer());
+            assertEquals(111, io.connectionProperties().getSocketSendBuffer());
+        }
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ids.setSocketReceiveBuffer(-1);
+
+                ids.getConnection();
+
+                return null;
+            }
+        }, SQLException.class, "Property cannot be lower than 0 [name=socketReceiveBuffer, value=-1]");
+
+        GridTestUtils.assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                ids.setSocketReceiveBuffer(1024);
+                ids.setSocketSendBuffer(-1);
+
+                ids.getConnection();
+
+                return null;
+            }
+        }, SQLException.class, "Property cannot be lower than 0 [name=socketSendBuffer, value=-1]");
+    }
+
+    /**
+     * Initial context creation testing purposes
+     * @return Initial context.
+     * @throws Exception On error.
+     */
+    private InitialContext getInitialContext() throws Exception {
+        Hashtable<String, String> env = new Hashtable<>();
+
+        env.put(Context.INITIAL_CONTEXT_FACTORY, JndiContextMockFactory.class.getName());
+
+        return new InitialContext(env);
+    }
+
+    /**
+     *
+     */
+    public static class JndiContextMockFactory implements InitialContextFactory {
+        /** {@inheritDoc} */
+        @Override public Context getInitialContext(Hashtable<?, ?> environment) throws NamingException {
+            return new JndiMockContext();
+        }
+    }
+
+    /**
+     *
+     */
+    public static class JndiMockContext implements Context {
+        /** Objects map. */
+        private Map<String, Object> map = new HashMap<String, Object>();
+
+        /** {@inheritDoc} */
+        @Override public Object lookup(Name name) throws NamingException {
+            return lookup(name.get(0));
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object lookup(String name) throws NamingException {
+            return map.get(name);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void bind(Name name, Object obj) throws NamingException {
+            rebind(name.get(0), obj);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void bind(String name, Object obj) throws NamingException {
+            rebind(name, obj);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rebind(Name name, Object obj) throws NamingException {
+            rebind(name.get(0), obj);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rebind(String name, Object obj) throws NamingException {
+            map.put(name, obj);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void unbind(Name name) throws NamingException {
+            unbind(name.get(0));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void unbind(String name) throws NamingException {
+            map.remove(name);
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rename(Name oldName, Name newName) throws NamingException {
+            rename(oldName.get(0), newName.get(0));
+        }
+
+        /** {@inheritDoc} */
+        @Override public void rename(String oldName, String newName) throws NamingException {
+            map.put(newName, map.remove(oldName));
+        }
+
+        /** {@inheritDoc} */
+        @Override public NamingEnumeration<NameClassPair> list(Name name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NamingEnumeration<NameClassPair> list(String name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NamingEnumeration<Binding> listBindings(Name name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NamingEnumeration<Binding> listBindings(String name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void destroySubcontext(Name name) throws NamingException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public void destroySubcontext(String name) throws NamingException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public Context createSubcontext(Name name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Context createSubcontext(String name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object lookupLink(Name name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object lookupLink(String name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NameParser getNameParser(Name name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NameParser getNameParser(String name) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Name composeName(Name name, Name prefix) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String composeName(String name, String prefix) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object addToEnvironment(String propName, Object propVal) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object removeFromEnvironment(String propName) throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Hashtable<?, ?> getEnvironment() throws NamingException {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void close() throws NamingException {
+        }
+
+        /** {@inheritDoc} */
+        @Override public String getNameInNamespace() throws NamingException {
+            return null;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/be25a50d/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
new file mode 100644
index 0000000..610ff99
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDataSource.java
@@ -0,0 +1,614 @@
+/*
+ * 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;
+
+import java.io.PrintWriter;
+import java.io.Serializable;
+import java.sql.Connection;
+import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
+import java.util.Properties;
+import java.util.logging.Logger;
+import javax.sql.DataSource;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.internal.jdbc.thin.ConnectionPropertiesImpl;
+import org.apache.ignite.internal.processors.odbc.SqlStateCode;
+import org.apache.ignite.internal.util.HostAndPortRange;
+import org.apache.ignite.internal.util.typedef.F;
+
+/**
+ * JDBC thin DataSource implementation.
+ */
+public class IgniteJdbcThinDataSource implements DataSource, Serializable {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Connection properties. */
+    private ConnectionPropertiesImpl props = new ConnectionPropertiesImpl();
+
+    /** Login timeout. */
+    private int loginTimeout;
+
+    /** {@inheritDoc} */
+    @Override public Connection getConnection() throws SQLException {
+        return getConnection(null, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Connection getConnection(String username, String pwd) throws SQLException {
+        Properties props =  this.props.storeToProperties();
+
+        if (!F.isEmpty(username))
+            props.put("user", username);
+
+        if (!F.isEmpty(pwd))
+            props.put("password", pwd);
+
+        return IgniteJdbcThinDriver.register().connect(getUrl(), props);
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public <T> T unwrap(Class<T> iface) throws SQLException {
+        if (!isWrapperFor(iface))
+            throw new SQLException("DataSource is not a wrapper for " + iface.getName());
+
+        return (T)this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isWrapperFor(Class<?> iface) throws SQLException {
+        return iface != null && iface.isAssignableFrom(IgniteJdbcThinDataSource.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PrintWriter getLogWriter() throws SQLException {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setLogWriter(PrintWriter out) throws SQLException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setLoginTimeout(int seconds) throws SQLException {
+        loginTimeout = seconds;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getLoginTimeout() throws SQLException {
+        return loginTimeout;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException {
+        return Logger.getLogger("org.apache.ignite");
+    }
+
+    /**
+     * Different application servers us different format (URL & url).
+     * @return Connection URL.
+     */
+    public String getURL() {
+        return getUrl();
+    }
+
+    /**
+     * Different application servers us different format (URL & url).
+     * @param url Connection URL.
+     * @throws SQLException On error whrn URL is invalid.
+     */
+    public void setURL(String url) throws SQLException {
+        setUrl(url);
+    }
+
+    /**
+     * @return Ignite nodes addresses.
+     */
+    public String[] getAddresses() {
+        HostAndPortRange[] addrs = props.getAddresses();
+
+        if (addrs == null)
+            return null;
+
+        String [] addrsStr = new String[addrs.length];
+
+        for (int i = 0; i < addrs.length; ++i)
+            addrsStr[i] = addrs[i].toString();
+
+        return addrsStr;
+    }
+
+    /**
+     * Sets the addresses of the Ignite nodes to connect;
+     * address string format: {@code host[:portRangeFrom[..portRangeTo]]}.
+     *
+     * Examples:
+     * <ul>
+     *     <li>"127.0.0.1"</li>
+     *     <li>"127.0.0.1:10800"</li>
+     *     <li>"127.0.0.1:10800..10810"</li>
+     *     <li>"mynode0.mydomain.org:10800..10810", "mynode1.mydomain.org:10800..10810", "127.0.0.1:10800"</li>
+     * <ul/>
+     *
+     * @param addrsStr Ignite nodes addresses.
+     * @throws SQLException On invalid addresses.
+     */
+    public void setAddresses(String... addrsStr) throws SQLException {
+        HostAndPortRange[] addrs = new HostAndPortRange[addrsStr.length];
+
+        for (int i = 0; i < addrs.length; ++i) {
+            try {
+                addrs[i] = HostAndPortRange.parse(addrsStr[i],
+                    ClientConnectorConfiguration.DFLT_PORT, ClientConnectorConfiguration.DFLT_PORT,
+                    "Invalid endpoint format (should be \"host[:portRangeFrom[..portRangeTo]]\")");
+            }
+            catch (IgniteCheckedException e) {
+                throw new SQLException(e.getMessage(), SqlStateCode.CLIENT_CONNECTION_FAILED, e);
+            }
+        }
+
+        props.setAddresses(addrs);
+    }
+
+    /**
+     * @return Schema name of the connection.
+     */
+    public String getSchema() {
+        return props.getSchema();
+    }
+
+    /**
+     * @param schema Schema name of the connection.
+     */
+    public void setSchema(String schema) {
+        props.setSchema(schema);
+    }
+
+    /**
+     * @return The URL of the connection.
+     */
+    public String getUrl() {
+        return props.getUrl();
+    }
+
+    /**
+     * @param url The URL of the connection.
+     * @throws SQLException On invalid URL.
+     */
+    public void setUrl(String url) throws SQLException {
+        props = new ConnectionPropertiesImpl();
+
+        props.setUrl(url);
+    }
+
+    /**
+     * @return Distributed joins flag.
+     */
+    public boolean isDistributedJoins() {
+        return props.isDistributedJoins();
+    }
+
+    /**
+     * @param distributedJoins Distributed joins flag.
+     */
+    public void setDistributedJoins(boolean distributedJoins) {
+        props.setDistributedJoins(distributedJoins);
+    }
+
+    /**
+     * @return Enforce join order flag.
+     */
+    public boolean isEnforceJoinOrder() {
+        return props.isEnforceJoinOrder();
+    }
+
+    /**
+     * @param enforceJoinOrder Enforce join order flag.
+     */
+    public void setEnforceJoinOrder(boolean enforceJoinOrder) {
+        props.setEnforceJoinOrder(enforceJoinOrder);
+    }
+
+    /**
+     * @return Collocated flag.
+     */
+    public boolean isCollocated() {
+        return props.isCollocated();
+    }
+
+    /**
+     * @param collocated Collocated flag.
+     */
+    public void setCollocated(boolean collocated) {
+        props.setCollocated(collocated);
+    }
+
+    /**
+     * @return Replicated only flag.
+     */
+    public boolean isReplicatedOnly() {
+        return props.isReplicatedOnly();
+    }
+
+    /**
+     * @param replicatedOnly Replicated only flag.
+     */
+    public void setReplicatedOnly(boolean replicatedOnly) {
+        props.setReplicatedOnly(replicatedOnly);
+    }
+
+    /**
+     * @return Auto close server cursors flag.
+     */
+    public boolean isAutoCloseServerCursor() {
+        return props.isAutoCloseServerCursor();
+    }
+
+    /**
+     * @param autoCloseServerCursor Auto close server cursors flag.
+     */
+    public void setAutoCloseServerCursor(boolean autoCloseServerCursor) {
+        props.setAutoCloseServerCursor(autoCloseServerCursor);
+    }
+
+    /**
+     * @return Socket send buffer size.
+     */
+    public int getSocketSendBuffer() {
+        return props.getSocketSendBuffer();
+    }
+
+    /**
+     * @param size Socket send buffer size.
+     * @throws SQLException On error.
+     */
+    public void setSocketSendBuffer(int size) throws SQLException {
+        props.setSocketSendBuffer(size);
+    }
+
+    /**
+     * @return Socket receive buffer size.
+     */
+    public int getSocketReceiveBuffer() {
+        return props.getSocketReceiveBuffer();
+    }
+
+    /**
+     * @param size Socket receive buffer size.
+     * @throws SQLException On error.
+     */
+    public void setSocketReceiveBuffer(int size) throws SQLException {
+        props.setSocketReceiveBuffer(size);
+    }
+
+    /**
+     * @return TCP no delay flag.
+     */
+    public boolean isTcpNoDelay() {
+        return props.isTcpNoDelay();
+    }
+
+    /**
+     * @param tcpNoDelay TCP no delay flag.
+     */
+    public void setTcpNoDelay(boolean tcpNoDelay) {
+        props.setTcpNoDelay(tcpNoDelay);
+    }
+
+    /**
+     * @return Lazy query execution flag.
+     */
+    public boolean isLazy() {
+        return props.isLazy();
+    }
+
+    /**
+     * @param lazy Lazy query execution flag.
+     */
+    public void setLazy(boolean lazy) {
+        props.setLazy(lazy);
+    }
+
+    /**
+     * @return Skip reducer on update flag.
+     */
+    public boolean isSkipReducerOnUpdate() {
+        return props.isSkipReducerOnUpdate();
+    }
+
+    /**
+     * @param skipReducerOnUpdate Skip reducer on update flag.
+     */
+    public void setSkipReducerOnUpdate(boolean skipReducerOnUpdate) {
+        props.setSkipReducerOnUpdate(skipReducerOnUpdate);
+    }
+
+    /**
+     * Gets SSL connection mode.
+     *
+     * @return Use SSL flag.
+     * @see #setSslMode(String).
+     */
+    public String getSslMode() {
+        return props.getSslMode();
+    }
+
+    /**
+     * Use SSL connection to Ignite node. In case set to {@code "require"} SSL context must be configured.
+     * {@link #setSslClientCertificateKeyStoreUrl} property and related properties must be set up
+     * or JSSE properties must be set up (see {@code javax.net.ssl.keyStore} and other {@code javax.net.ssl.*}
+     * properties)
+     *
+     * In case set to {@code "disable"} plain connection is used.
+     * Available modes: {@code "disable", "require"}. Default value is {@code "disable"}
+     *
+     * @param mode SSL mode.
+     */
+    public void setSslMode(String mode) {
+        props.setSslMode(mode);
+    }
+
+    /**
+     * Gets protocol for secure transport.
+     *
+     * @return SSL protocol name.
+     */
+    public String getSslProtocol() {
+        return props.getSslProtocol();
+    }
+
+    /**
+     * Sets protocol for secure transport. If not specified, TLS protocol will be used.
+     * Protocols implementations supplied by JSEE: SSLv3 (SSL), TLSv1 (TLS), TLSv1.1, TLSv1.2
+     *
+     * <p>See more at JSSE Reference Guide.
+     *
+     * @param sslProtocol SSL protocol name.
+     */
+    public void setSslProtocol(String sslProtocol) {
+        props.setSslProtocol(sslProtocol);
+    }
+
+    /**
+     * Gets algorithm that will be used to create a key manager.
+     *
+     * @return Key manager algorithm.
+     */
+    public String getSslKeyAlgorithm() {
+        return props.getSslKeyAlgorithm();
+    }
+
+    /**
+     * Sets key manager algorithm that will be used to create a key manager. Notice that in most cased default value
+     * suites well, however, on Android platform this value need to be set to <tt>X509<tt/>.
+     * Algorithms implementations supplied by JSEE: PKIX (X509 or SunPKIX), SunX509
+     *
+     * <p>See more at JSSE Reference Guide.
+     *
+     * @param keyAlgorithm Key algorithm name.
+     */
+    public void setSslKeyAlgorithm(String keyAlgorithm) {
+        props.setSslKeyAlgorithm(keyAlgorithm);
+    }
+
+    /**
+     * Gets the key store URL.
+     *
+     * @return Client certificate KeyStore URL.
+     */
+    public String getSslClientCertificateKeyStoreUrl() {
+        return props.getSslClientCertificateKeyStoreUrl();
+    }
+
+    /**
+     * Sets path to the key store file. This is a mandatory parameter since
+     * ssl context could not be initialized without key manager.
+     *
+     * In case {@link #getSslMode()} is {@code required} and key store URL isn't specified by Ignite properties
+     * (e.g. at JDBC URL) the JSSE property {@code javax.net.ssl.keyStore} will be used.
+     *
+     * @param url Client certificate KeyStore URL.
+     */
+    public void setSslClientCertificateKeyStoreUrl(String url) {
+        props.setSslClientCertificateKeyStoreUrl(url);
+    }
+
+    /**
+     * Gets key store password.
+     *
+     * @return Client certificate KeyStore password.
+     */
+    public String getSslClientCertificateKeyStorePassword() {
+        return props.getSslClientCertificateKeyStorePassword();
+    }
+
+    /**
+     * Sets key store password.
+     *
+     * In case {@link #getSslMode()} is {@code required}  and key store password isn't specified by Ignite properties
+     * (e.g. at JDBC URL) the JSSE property {@code javax.net.ssl.keyStorePassword} will be used.
+     *
+     * @param passwd Client certificate KeyStore password.
+     */
+    public void setSslClientCertificateKeyStorePassword(String passwd) {
+        props.setSslClientCertificateKeyStorePassword(passwd);
+    }
+
+    /**
+     * Gets key store type used for context creation.
+     *
+     * @return Client certificate KeyStore type.
+     */
+    public String getSslClientCertificateKeyStoreType() {
+        return props.getSslClientCertificateKeyStoreType();
+    }
+
+    /**
+     * Sets key store type used in context initialization.
+     *
+     * In case {@link #getSslMode()} is {@code required} and key store type isn't specified by Ignite properties
+     *  (e.g. at JDBC URL)the JSSE property {@code javax.net.ssl.keyStoreType} will be used.
+     * In case both Ignite properties and JSSE properties are not set the default 'JKS' type is used.
+     *
+     * <p>See more at JSSE Reference Guide.
+     *
+     * @param ksType Client certificate KeyStore type.
+     */
+    public void setSslClientCertificateKeyStoreType(String ksType) {
+        props.setSslClientCertificateKeyStoreType(ksType);
+    }
+
+    /**
+     * Gets the trust store URL.
+     *
+     * @return Trusted certificate KeyStore URL.
+     */
+    public String getSslTrustCertificateKeyStoreUrl() {
+        return props.getSslTrustCertificateKeyStoreUrl();
+    }
+
+    /**
+     * Sets path to the trust store file. This is an optional parameter,
+     * however one of the {@code setSslTrustCertificateKeyStoreUrl(String)}, {@link #setSslTrustAll(boolean)}
+     * properties must be set.
+     *
+     * In case {@link #getSslMode()} is {@code required} and trust store URL isn't specified by Ignite properties
+     * (e.g. at JDBC URL) the JSSE property {@code javax.net.ssl.trustStore} will be used.
+     *
+     * @param url Trusted certificate KeyStore URL.
+     */
+    public void setSslTrustCertificateKeyStoreUrl(String url) {
+        props.setSslTrustCertificateKeyStoreUrl(url);
+    }
+
+    /**
+     * Gets trust store password.
+     *
+     * @return Trusted certificate KeyStore password.
+     */
+    public String getSslTrustCertificateKeyStorePassword() {
+        return props.getSslTrustCertificateKeyStorePassword();
+    }
+
+    /**
+     * Sets trust store password.
+     *
+     * In case {@link #getSslMode()} is {@code required} and trust store password isn't specified by Ignite properties
+     * (e.g. at JDBC URL) the JSSE property {@code javax.net.ssl.trustStorePassword} will be used.
+     *
+     * @param passwd Trusted certificate KeyStore password.
+     */
+    public void setSslTrustCertificateKeyStorePassword(String passwd) {
+        props.setSslTrustCertificateKeyStorePassword(passwd);
+    }
+
+    /**
+     * Gets trust store type.
+     *
+     * @return Trusted certificate KeyStore type.
+     */
+    public String getSslTrustCertificateKeyStoreType() {
+        return props.getSslTrustCertificateKeyStoreType();
+    }
+
+    /**
+     * Sets trust store type.
+     *
+     * In case {@link #getSslMode()} is {@code required} and trust store type isn't specified by Ignite properties
+     * (e.g. at JDBC URL) the JSSE property {@code javax.net.ssl.trustStoreType} will be used.
+     * In case both Ignite properties and JSSE properties are not set the default 'JKS' type is used.
+     *
+     * @param ksType Trusted certificate KeyStore type.
+     */
+    public void setSslTrustCertificateKeyStoreType(String ksType) {
+        props.setSslTrustCertificateKeyStoreType(ksType);
+    }
+
+    /**
+     * Gets trust any server certificate flag.
+     *
+     * @return Trust all certificates flag.
+     */
+    public boolean isSslTrustAll() {
+        return props.isSslTrustAll();
+    }
+
+    /**
+     * Sets to {@code true} to trust any server certificate (revoked, expired or self-signed SSL certificates).
+     *
+     * <p> Defaults is {@code false}.
+     *
+     * Note: Do not enable this option in production you are ever going to use
+     * on a network you do not entirely trust. Especially anything going over the public internet.
+     *
+     * @param trustAll Trust all certificates flag.
+     */
+    public void setSslTrustAll(boolean trustAll) {
+        props.setSslTrustAll(trustAll);
+    }
+
+    /**
+     * Gets the class name of the custom implementation of the Factory&lt;SSLSocketFactory&gt;.
+     *
+     * @return Custom class name that implements Factory&lt;SSLSocketFactory&gt;.
+     */
+    public String getSslFactory() {
+        return props.getSslFactory();
+    }
+
+    /**
+     * Sets the class name of the custom implementation of the Factory&lt;SSLSocketFactory&gt;.
+     * If {@link #getSslMode()} is {@code required} and factory is specified the custom factory will be used
+     * instead of JSSE socket factory. So, other SSL properties will be ignored.
+     *
+     * @param sslFactory Custom class name that implements Factory&lt;SSLSocketFactory&gt;.
+     */
+    public void setSslFactory(String sslFactory) {
+        props.setSslFactory(sslFactory);
+    }
+
+    /**
+     * @param name User name to authentication.
+     */
+    public void setUsername(String name) {
+        props.setUsername(name);
+    }
+
+    /**
+     * @return User name to authentication.
+     */
+    public String getUsername() {
+        return props.getUsername();
+    }
+
+    /**
+     * @param passwd User's password.
+     */
+    public void setPassword(String passwd) {
+        props.setPassword(passwd);
+    }
+
+    /**
+     * @return User's password.
+     */
+    public String getPassword() {
+        return props.getPassword();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/be25a50d/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
index 0dbe536..ba4199f 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteJdbcThinDriver.java
@@ -127,24 +127,22 @@ import org.apache.ignite.internal.jdbc.thin.JdbcThinUtils;
  */
 @SuppressWarnings("JavadocReference")
 public class IgniteJdbcThinDriver implements Driver {
+    /** Driver instance. */
+    private static final Driver INSTANCE = new IgniteJdbcThinDriver();
+
+    /** Registered flag. */
+    private static volatile boolean registered;
+
+    static {
+        register();
+    }
+
     /** Major version. */
     private static final int MAJOR_VER = IgniteVersionUtils.VER.major();
 
     /** Minor version. */
     private static final int MINOR_VER = IgniteVersionUtils.VER.minor();
 
-    /*
-     * Register driver.
-     */
-    static {
-        try {
-            DriverManager.registerDriver(new IgniteJdbcThinDriver());
-        }
-        catch (SQLException e) {
-            throw new RuntimeException("Failed to register Ignite JDBC driver.", e);
-        }
-    }
-
     /** {@inheritDoc} */
     @Override public Connection connect(String url, Properties props) throws SQLException {
         if (!acceptsURL(url))
@@ -190,4 +188,22 @@ public class IgniteJdbcThinDriver implements Driver {
     @Override public Logger getParentLogger() throws SQLFeatureNotSupportedException {
         throw new SQLFeatureNotSupportedException("java.util.logging is not used.");
     }
+
+    /**
+     * @return Driver instance.
+     */
+    public static synchronized Driver register() {
+        try {
+            if (!registered) {
+                DriverManager.registerDriver(INSTANCE);
+
+                registered = true;
+            }
+        }
+        catch (SQLException e) {
+            throw new RuntimeException("Failed to register Ignite JDBC thin driver.", e);
+        }
+
+        return INSTANCE;
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/be25a50d/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
index 86dc298..9371a07 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/jdbc/thin/ConnectionPropertiesImpl.java
@@ -195,8 +195,12 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
 
             HostAndPortRange [] addrs = getAddresses();
 
-            for (int i = 0; i < addrs.length; i++)
+            for (int i = 0; i < addrs.length; i++) {
+                if (i > 0)
+                    sbUrl.append(',');
+
                 sbUrl.append(addrs[i].toString());
+            }
 
             if (!F.isEmpty(getSchema()))
                 sbUrl.append('/').append(getSchema());
@@ -692,6 +696,20 @@ public class ConnectionPropertiesImpl implements ConnectionProperties, Serializa
     }
 
     /**
+     * @return Properties set contains connection parameters.
+     */
+    public Properties storeToProperties() {
+        Properties props = new Properties();
+
+        for (ConnectionProperty prop : propsArray) {
+            if (prop.valueObject() != null)
+                props.setProperty(PROP_PREFIX + prop.getName(), prop.valueObject());
+        }
+
+        return props;
+    }
+
+    /**
      *
      */
     private interface PropertyValidator extends Serializable {